diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java new file mode 100644 index 0000000..7464dbb --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/InstancePending.java @@ -0,0 +1,80 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.zookeeper; + +import java.util.concurrent.CountDownLatch; + +/** + * Placeholder of an instance which will be accessed by other threads + * but is not yet created. Thread safe. + */ +class InstancePending { + // Based on a subtle part of the Java Language Specification, + // in order to avoid a slight overhead of synchronization for each access. + + private final CountDownLatch pendingLatch = new CountDownLatch(1); + + /** Piggybacking on {@code pendingLatch}. */ + private InstanceHolder instanceHolder; + + private static class InstanceHolder { + // The JLS ensures the visibility of a final field and its contents + // unless they are exposed to another thread while the construction. + final T instance; + + InstanceHolder(T instance) { + this.instance = instance; + } + } + + /** + * Returns the instance given by the method {@link #prepare}. + * This is an interruptible blocking method + * and the interruption flag will be set just before returning if any. + */ + T get() { + InstanceHolder instanceHolder; + boolean interrupted = false; + + while ((instanceHolder = this.instanceHolder) == null) { + try { + pendingLatch.await(); + } catch (InterruptedException e) { + interrupted = true; + } + } + + if (interrupted) { + Thread.currentThread().interrupt(); + } + return instanceHolder.instance; + } + + /** + * Associates the given instance for the method {@link #get}. + * This method should be called once, and {@code instance} should be non-null. + * This method is expected to call as soon as possible + * because the method {@code get} is uninterruptibly blocked until this method is called. + */ + void prepare(T instance) { + assert instance != null; + instanceHolder = new InstanceHolder(instance); + pendingLatch.countDown(); + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java new file mode 100644 index 0000000..11d0e5d --- /dev/null +++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/PendingWatcher.java @@ -0,0 +1,53 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.zookeeper; + +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; + +/** + * Placeholder of a watcher which might be triggered before the instance is not yet created. + *

+ * {@code ZooKeeper} starts its event thread within its constructor (and that is an anti-pattern), + * and the watcher passed to the constructor might be called back by the event thread + * before you get the instance of {@code ZooKeeper} from the constructor. + * If your watcher calls methods of {@code ZooKeeper}, + * pass this placeholder to the constructor of the {@code ZooKeeper}, + * create your watcher using the instance of {@code ZooKeeper}, + * and then call the method {@code PendingWatcher.prepare}. + */ +class PendingWatcher implements Watcher { + private final InstancePending pending = new InstancePending(); + + @Override + public void process(WatchedEvent event) { + pending.get().process(event); + } + + /** + * Associates the substantial watcher of processing events. + * This method should be called once, and {@code watcher} should be non-null. + * This method is expected to call as soon as possible + * because the event processing, being invoked by the ZooKeeper event thread, + * is uninterruptibly blocked until this method is called. + */ + void prepare(Watcher watcher) { + pending.prepare(watcher); + } +} diff --git hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java index 205d397..dcbcf30 100644 --- hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java +++ hbase-client/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java @@ -73,7 +73,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { private String quorum; // zookeeper connection - private RecoverableZooKeeper recoverableZooKeeper; + private final RecoverableZooKeeper recoverableZooKeeper; // abortable in case of zk failure protected Abortable abortable; @@ -130,8 +130,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { private final Configuration conf; - private final Exception constructorCaller; - /* A pattern that matches a Kerberos name, borrowed from Hadoop's KerberosName */ private static final Pattern NAME_PATTERN = Pattern.compile("([^/@]*)(/([^/@]*))?@([^/@]*)"); @@ -162,13 +160,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { Abortable abortable, boolean canCreateBaseZNode) throws IOException, ZooKeeperConnectionException { this.conf = conf; - // Capture a stack trace now. Will print it out later if problem so we can - // distingush amongst the myriad ZKWs. - try { - throw new Exception("ZKW CONSTRUCTOR STACK TRACE FOR DEBUGGING"); - } catch (Exception e) { - this.constructorCaller = e; - } this.quorum = ZKConfig.getZKQuorumServersString(conf); this.prefix = identifier; // Identifier will get the sessionid appended later below down when we @@ -176,7 +167,9 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { this.identifier = identifier + "0x0"; this.abortable = abortable; setNodeNames(conf); - this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, this, identifier); + PendingWatcher pendingWatcher = new PendingWatcher(); + this.recoverableZooKeeper = ZKUtil.connect(conf, quorum, pendingWatcher, identifier); + pendingWatcher.prepare(this); if (canCreateBaseZNode) { createBaseZNodes(); } @@ -650,27 +643,6 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { private void connectionEvent(WatchedEvent event) { switch(event.getState()) { case SyncConnected: - // Now, this callback can be invoked before the this.zookeeper is set. - // Wait a little while. - long finished = System.currentTimeMillis() + - this.conf.getLong("hbase.zookeeper.watcher.sync.connected.wait", 2000); - while (System.currentTimeMillis() < finished) { - try { - Thread.sleep(1); - } catch (InterruptedException e) { - LOG.warn("Interrupted while sleeping"); - throw new RuntimeException("Interrupted while waiting for" + - " recoverableZooKeeper is set"); - } - if (this.recoverableZooKeeper != null) break; - } - - if (this.recoverableZooKeeper == null) { - LOG.error("ZK is null on connection event -- see stack trace " + - "for the stack trace when constructor was called on this zkw", - this.constructorCaller); - throw new NullPointerException("ZK is null"); - } this.identifier = this.prefix + "-0x" + Long.toHexString(this.recoverableZooKeeper.getSessionId()); // Update our identifier. Otherwise ignore. @@ -759,9 +731,7 @@ public class ZooKeeperWatcher implements Watcher, Abortable, Closeable { @Override public void close() { try { - if (recoverableZooKeeper != null) { - recoverableZooKeeper.close(); - } + recoverableZooKeeper.close(); } catch (InterruptedException e) { Thread.currentThread().interrupt(); } diff --git hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java new file mode 100644 index 0000000..667fed8 --- /dev/null +++ hbase-client/src/test/java/org/apache/hadoop/hbase/zookeeper/TestInstancePending.java @@ -0,0 +1,49 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.zookeeper; + +import java.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.hbase.testclassification.SmallTests; +import org.junit.Assert; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(SmallTests.class) +public class TestInstancePending { + @Test(timeout = 1000) + public void test() throws Exception { + final InstancePending pending = new InstancePending(); + final AtomicReference getResultRef = new AtomicReference(); + + new Thread() { + @Override + public void run() { + getResultRef.set(pending.get()); + } + }.start(); + + Thread.sleep(100); + Assert.assertNull(getResultRef.get()); + + pending.prepare("abc"); + Thread.sleep(100); + Assert.assertEquals("abc", getResultRef.get()); + } +} diff --git hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java index 6bc7508..7172f06 100644 --- hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java +++ hbase-rsgroup/src/main/java/org/apache/hadoop/hbase/rsgroup/RSGroupAdminEndpoint.java @@ -310,6 +310,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService HRegionInfo[] regions) throws IOException { } + @Deprecated @Override public void preCreateTableHandler(ObserverContext ctx, HTableDescriptor desc, @@ -317,27 +318,56 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override + public void preCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + } + + @Deprecated + @Override public void postCreateTableHandler(ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { } @Override + public void postCompletedCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + } + + @Override public void preDeleteTable(ObserverContext ctx, TableName tableName) throws IOException { } + @Deprecated @Override public void preDeleteTableHandler(ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void preDeleteTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + } + + @Deprecated + @Override public void postDeleteTableHandler(ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedDeleteTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + } + + @Override public void preTruncateTable(ObserverContext ctx, TableName tableName) throws IOException { } @@ -347,17 +377,31 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService TableName tableName) throws IOException { } + @Deprecated @Override public void preTruncateTableHandler(ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void preTruncateTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + } + + @Deprecated + @Override public void postTruncateTableHandler(ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedTruncateTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + } + + @Override public void preModifyTable(ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { @@ -369,6 +413,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService HTableDescriptor htd) throws IOException { } + @Deprecated @Override public void preModifyTableHandler(ObserverContext ctx, TableName tableName, @@ -376,12 +421,27 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override + public void preModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException { + } + + @Deprecated + @Override public void postModifyTableHandler(ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { } @Override + public void postCompletedModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException { + } + + @Override public void preAddColumn(ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException { @@ -405,6 +465,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService HColumnDescriptor columnFamily) throws IOException { } + @Deprecated @Override public void preAddColumnHandler(ObserverContext ctx, TableName tableName, @@ -412,11 +473,13 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override - public void preAddColumnFamilyHandler(ObserverContext ctx, - TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void preAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } + @Deprecated @Override public void postAddColumnHandler(ObserverContext ctx, TableName tableName, @@ -424,9 +487,10 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override - public void postAddColumnFamilyHandler(ObserverContext ctx, - TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void postCompletedAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } @Override @@ -453,6 +517,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } + @Deprecated @Override public void preModifyColumnHandler(ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException { @@ -460,12 +525,15 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override - public void preModifyColumnFamilyHandler(ObserverContext ctx, - TableName tableName, HColumnDescriptor columnFamily) + public void preModifyColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } + @Deprecated @Override public void postModifyColumnHandler(ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws @@ -474,8 +542,10 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override - public void postModifyColumnFamilyHandler(ObserverContext ctx, - TableName tableName, HColumnDescriptor columnFamily) + public void postCompletedModifyColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } @@ -504,6 +574,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } + @Deprecated @Override public void preDeleteColumnHandler(ObserverContext ctx, TableName tableName, byte[] columnFamily) throws IOException { @@ -511,12 +582,15 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override - public void preDeleteColumnFamilyHandler(ObserverContext ctx, - TableName tableName, byte[] columnFamily) throws + public void preDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { } + @Deprecated @Override public void postDeleteColumnHandler(ObserverContext ctx, TableName tableName, byte[] columnFamily) throws IOException { @@ -524,8 +598,10 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override - public void postDeleteColumnFamilyHandler(ObserverContext ctx, - TableName tableName, byte[] columnFamily) throws + public void postCompletedDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { } @@ -542,6 +618,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } + @Deprecated @Override public void preEnableTableHandler(ObserverContext ctx, TableName tableName) throws IOException { @@ -549,12 +626,27 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override + public void preEnableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + + } + + @Deprecated + @Override public void postEnableTableHandler(ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedEnableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + + } + + @Override public void preDisableTable(ObserverContext ctx, TableName tableName) throws IOException { @@ -566,6 +658,7 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } + @Deprecated @Override public void preDisableTableHandler(ObserverContext ctx, TableName tableName) throws IOException { @@ -573,12 +666,27 @@ public class RSGroupAdminEndpoint extends RSGroupAdminService } @Override + public void preDisableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + + } + + @Deprecated + @Override public void postDisableTableHandler(ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedDisableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + + } + + @Override public void preMove(ObserverContext ctx, HRegionInfo region, ServerName srcServer, ServerName destServer) throws IOException { diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java index 74d9fe1..5a69708 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterAndRegionObserver.java @@ -67,6 +67,7 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver HRegionInfo regionA, HRegionInfo regionB) throws IOException { } + @Deprecated @Override public void preCreateTableHandler( final ObserverContext ctx, @@ -74,12 +75,27 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override + public void preCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + } + + @Deprecated + @Override public void postCreateTableHandler( final ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { } @Override + public void postCompletedCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + } + + @Override public void preDeleteTable(ObserverContext ctx, TableName tableName) throws IOException { } @@ -89,6 +105,7 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver TableName tableName) throws IOException { } + @Deprecated @Override public void preDeleteTableHandler( final ObserverContext ctx, TableName tableName) @@ -96,12 +113,25 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override + public void preDeleteTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException{ + } + + @Deprecated + @Override public void postDeleteTableHandler( final ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedDeleteTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preTruncateTable(ObserverContext ctx, TableName tableName) throws IOException { } @@ -111,6 +141,7 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver TableName tableName) throws IOException { } + @Deprecated @Override public void preTruncateTableHandler( final ObserverContext ctx, TableName tableName) @@ -118,22 +149,37 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override + public void preTruncateTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Deprecated + @Override public void postTruncateTableHandler( final ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedTruncateTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preModifyTable(ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { } + @Deprecated @Override public void postModifyTableHandler( ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { } + @Deprecated @Override public void preModifyTableHandler( ObserverContext ctx, TableName tableName, @@ -141,6 +187,20 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override + public void preModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException { + } + + @Override + public void postCompletedModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException { + } + + @Override public void postModifyTable(ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { } @@ -225,9 +285,10 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override - public void preAddColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void preAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } @Deprecated @@ -238,9 +299,10 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override - public void postAddColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void postCompletedAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } @Deprecated @@ -273,9 +335,10 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override - public void preModifyColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void preModifyColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } @Deprecated @@ -286,9 +349,10 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override - public void postModifyColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void postCompletedModifyColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { } @Deprecated @@ -321,9 +385,10 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override - public void preDeleteColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - byte[] columnFamily) throws IOException { + public void preDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { } @Deprecated @@ -334,9 +399,10 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override - public void postDeleteColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - byte[] columnFamily) throws IOException { + public void postCompletedDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { } @@ -350,6 +416,7 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver TableName tableName) throws IOException { } + @Deprecated @Override public void preEnableTableHandler( ObserverContext ctx, TableName tableName) @@ -357,12 +424,24 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override + public void preEnableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + @Deprecated + @Override public void postEnableTableHandler( ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedEnableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preDisableTable(ObserverContext ctx, TableName tableName) throws IOException { } @@ -372,6 +451,7 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver TableName tableName) throws IOException { } + @Deprecated @Override public void preDisableTableHandler( ObserverContext ctx, TableName tableName) @@ -379,12 +459,25 @@ public class BaseMasterAndRegionObserver extends BaseRegionObserver } @Override + public void preDisableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Deprecated + @Override public void postDisableTableHandler( ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedDisableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preAbortProcedure( ObserverContext ctx, final ProcedureExecutor procEnv, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java index 3574bbd..71dab06 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/BaseMasterObserver.java @@ -56,6 +56,20 @@ public class BaseMasterObserver implements MasterObserver { HTableDescriptor desc, HRegionInfo[] regions) throws IOException { } + /** + * Called before a new table is created by + * {@link org.apache.hadoop.hbase.master.HMaster}. Called as part of create + * table handler and it is async to the create RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param desc the HTableDescriptor for the table + * @param regions the initial regions created for the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preCreateTableAction(ObserverContext, HTableDescriptor, HRegionInfo[])}. + */ + @Deprecated @Override public void preCreateTableHandler( final ObserverContext ctx, @@ -63,12 +77,39 @@ public class BaseMasterObserver implements MasterObserver { } @Override + public void preCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + } + + /** + * Called after the createTable operation has been requested. Called as part + * of create table RPC call. Called as part of create table handler and + * it is async to the create RPC call. + * @param ctx the environment to interact with the framework and master + * @param desc the HTableDescriptor for the table + * @param regions the initial regions created for the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedCreateTableAction(ObserverContext, HTableDescriptor, HRegionInfo[])} + */ + @Deprecated + @Override public void postCreateTableHandler( final ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { } @Override + public void postCompletedCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + } + + @Override public void preDispatchMerge(final ObserverContext ctx, HRegionInfo regionA, HRegionInfo regionB) throws IOException { } @@ -88,6 +129,19 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName) throws IOException { } + /** + * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a + * table. Called as part of delete table handler and + * it is async to the delete RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preDeleteTableAction(ObserverContext, TableName)}. + */ + @Deprecated @Override public void preDeleteTableHandler( final ObserverContext ctx, TableName tableName) @@ -95,12 +149,37 @@ public class BaseMasterObserver implements MasterObserver { } @Override + public void preDeleteTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException{ + } + + /** + * Called after {@link org.apache.hadoop.hbase.master.HMaster} deletes a + * table. Called as part of delete table handler and it is async to the + * delete RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedDeleteTableAction(ObserverContext, TableName)}. + */ + @Deprecated + @Override public void postDeleteTableHandler( final ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedDeleteTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preTruncateTable(ObserverContext ctx, TableName tableName) throws IOException { } @@ -110,6 +189,19 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName) throws IOException { } + /** + * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a + * table. Called as part of truncate table handler and it is sync + * to the truncate RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preTruncateTableAction(ObserverContext, TableName)}. + */ + @Deprecated @Override public void preTruncateTableHandler( final ObserverContext ctx, TableName tableName) @@ -117,31 +209,96 @@ public class BaseMasterObserver implements MasterObserver { } @Override + public void preTruncateTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + /** + * Called after {@link org.apache.hadoop.hbase.master.HMaster} truncates a + * table. Called as part of truncate table handler and it is sync to the + * truncate RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedTruncateTableAction(ObserverContext, TableName)}. + */ + @Deprecated + @Override public void postTruncateTableHandler( final ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedTruncateTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preModifyTable(ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { } @Override - public void postModifyTableHandler( + public void postModifyTable(ObserverContext ctx, + TableName tableName, HTableDescriptor htd) throws IOException { + } + + /** + * Called prior to modifying a table's properties. Called as part of modify + * table handler and it is async to the modify table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param htd the HTableDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preModifyTableAction(ObserverContext, TableName, HTableDescriptor)}. + */ + @Deprecated + @Override + public void preModifyTableHandler( ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { } @Override - public void preModifyTableHandler( + public void preModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException { + } + + /** + * Called after to modifying a table's properties. Called as part of modify + * table handler and it is async to the modify table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param htd the HTableDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #postCompletedModifyTableAction(ObserverContext, TableName, HTableDescriptor)}. + */ + @Deprecated + @Override + public void postModifyTableHandler( ObserverContext ctx, TableName tableName, HTableDescriptor htd) throws IOException { } @Override - public void postModifyTable(ObserverContext ctx, - TableName tableName, HTableDescriptor htd) throws IOException { + public void postCompletedModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException { } @Override @@ -198,6 +355,17 @@ public class BaseMasterObserver implements MasterObserver { List descriptors) throws IOException { } + /** + * Called prior to adding a new column family to the table. Called as part of + * add column RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #preAddColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. + */ @Deprecated @Override public void preAddColumn(ObserverContext ctx, @@ -209,6 +377,17 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName, HColumnDescriptor columnFamily) throws IOException { } + /** + * Called after the new column family has been created. Called as part of + * add column RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #postAddColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. + */ @Deprecated @Override public void postAddColumn(ObserverContext ctx, @@ -220,6 +399,17 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName, HColumnDescriptor columnFamily) throws IOException { } + /** + * Called prior to adding a new column family to the table. Called as part of + * add column handler. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). Use + * {@link #preAddColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}. + */ @Deprecated @Override public void preAddColumnHandler( @@ -228,11 +418,23 @@ public class BaseMasterObserver implements MasterObserver { } @Override - public void preAddColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { - } - + public void preAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { + } + + /** + * Called after the new column family has been created. Called as part of + * add column handler. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). Use + * {@link #postCompletedAddColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}. + */ @Deprecated @Override public void postAddColumnHandler( @@ -241,11 +443,23 @@ public class BaseMasterObserver implements MasterObserver { } @Override - public void postAddColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { - } - + public void postCompletedAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { + } + + /** + * Called prior to modifying a column family's attributes. Called as part of + * modify column RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #preModifyColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. + */ @Deprecated @Override public void preModifyColumn(ObserverContext ctx, @@ -257,6 +471,17 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName, HColumnDescriptor columnFamily) throws IOException { } + /** + * Called after the column family has been updated. Called as part of modify + * column RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #postModifyColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. + */ @Deprecated @Override public void postModifyColumn(ObserverContext ctx, @@ -268,6 +493,17 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName, HColumnDescriptor columnFamily) throws IOException { } + /** + * Called prior to modifying a column family's attributes. Called as part of + * modify column handler. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #preModifyColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}. + */ @Deprecated @Override public void preModifyColumnHandler( @@ -276,11 +512,23 @@ public class BaseMasterObserver implements MasterObserver { } @Override - public void preModifyColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { - } - + public void preModifyColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { + } + + /** + * Called after the column family has been updated. Called as part of modify + * column handler. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). Use + * {@link #postCompletedModifyColumnFamilyAction(ObserverContext,TableName,HColumnDescriptor)}. + */ @Deprecated @Override public void postModifyColumnHandler( @@ -289,11 +537,23 @@ public class BaseMasterObserver implements MasterObserver { } @Override - public void postModifyColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { - } - + public void postCompletedModifyColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { + } + + /** + * Called prior to deleting the entire column family. Called as part of + * delete column RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the column family + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #preDeleteColumnFamily(ObserverContext, TableName, byte[])}. + */ @Deprecated @Override public void preDeleteColumn(ObserverContext ctx, @@ -305,6 +565,17 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName, byte[] columnFamily) throws IOException { } + /** + * Called after the column family has been deleted. Called as part of delete + * column RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the column family + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #postDeleteColumnFamily(ObserverContext, TableName, byte[])}. + */ @Deprecated @Override public void postDeleteColumn(ObserverContext ctx, @@ -316,6 +587,17 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName, byte[] columnFamily) throws IOException { } + /** + * Called prior to deleting the entire column family. Called as part of + * delete column handler. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the column family + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #preDeleteColumnFamilyAction(ObserverContext, TableName, byte[])}. + */ @Deprecated @Override public void preDeleteColumnHandler( @@ -324,11 +606,23 @@ public class BaseMasterObserver implements MasterObserver { } @Override - public void preDeleteColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - byte[] columnFamily) throws IOException { - } - + public void preDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { + } + + /** + * Called after the column family has been deleted. Called as part of + * delete column handler. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param columnFamily the column family + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #postCompletedDeleteColumnFamilyAction(ObserverContext, TableName, byte[])}. + */ @Deprecated @Override public void postDeleteColumnHandler( @@ -337,9 +631,10 @@ public class BaseMasterObserver implements MasterObserver { } @Override - public void postDeleteColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - byte[] columnFamily) throws IOException { + public void postCompletedDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { } @@ -353,6 +648,18 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName) throws IOException { } + /** + * Called prior to enabling a table. Called as part of enable table handler + * and it is async to the enable table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preEnableTableAction(ObserverContext, TableName)}. + */ + @Deprecated @Override public void preEnableTableHandler( ObserverContext ctx, TableName tableName) @@ -360,12 +667,35 @@ public class BaseMasterObserver implements MasterObserver { } @Override + public void preEnableTableAction( + ObserverContext ctx, final TableName tableName) + throws IOException { + } + + /** + * Called after the enableTable operation has been requested. Called as part + * of enable table handler and it is async to the enable table RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedEnableTableAction(ObserverContext, TableName)}. + */ + @Deprecated + @Override public void postEnableTableHandler( ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedEnableTableAction( + ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preDisableTable(ObserverContext ctx, TableName tableName) throws IOException { } @@ -375,6 +705,18 @@ public class BaseMasterObserver implements MasterObserver { TableName tableName) throws IOException { } + /** + * Called prior to disabling a table. Called as part of disable table handler + * and it is asyn to the disable table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preDisableTableAction(ObserverContext, TableName)}. + */ + @Deprecated @Override public void preDisableTableHandler( ObserverContext ctx, TableName tableName) @@ -382,12 +724,35 @@ public class BaseMasterObserver implements MasterObserver { } @Override + public void preDisableTableAction( + ObserverContext ctx, final TableName tableName) + throws IOException { + } + + /** + * Called after the disableTable operation has been requested. Called as part + * of disable table handler and it is asyn to the disable table RPC call. + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedDisableTableAction(ObserverContext, TableName)}. + */ + @Deprecated + @Override public void postDisableTableHandler( ObserverContext ctx, TableName tableName) throws IOException { } @Override + public void postCompletedDisableTableAction( + ObserverContext ctx, final TableName tableName) + throws IOException { + } + + @Override public void preAbortProcedure( ObserverContext ctx, final ProcedureExecutor procEnv, diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java index 4b43cfe..9e04c50 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/MasterObserver.java @@ -59,7 +59,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param desc the HTableDescriptor for the table * @param regions the initial regions created for the table - * @throws IOException + * @throws IOException if something went wrong */ void preCreateTable(final ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException; @@ -70,10 +70,11 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param desc the HTableDescriptor for the table * @param regions the initial regions created for the table - * @throws IOException + * @throws IOException if something went wrong */ void postCreateTable(final ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException; + /** * Called before a new table is created by * {@link org.apache.hadoop.hbase.master.HMaster}. Called as part of create @@ -82,8 +83,12 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param desc the HTableDescriptor for the table * @param regions the initial regions created for the table - * @throws IOException + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preCreateTableAction(ObserverContext, HTableDescriptor, HRegionInfo[])}. */ + @Deprecated void preCreateTableHandler(final ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException; @@ -94,17 +99,61 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param desc the HTableDescriptor for the table * @param regions the initial regions created for the table - * @throws IOException + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedCreateTableAction(ObserverContext, HTableDescriptor, HRegionInfo[])} */ + @Deprecated void postCreateTableHandler(final ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException; /** + * Called before a new table is created by + * {@link org.apache.hadoop.hbase.master.HMaster}. Called as part of create + * table procedure and it is async to the create RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #preCreateTableHandler(ObserverContext, HTableDescriptor, HRegionInfo[])} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param desc the HTableDescriptor for the table + * @param regions the initial regions created for the table + * @throws IOException if something went wrong + */ + void preCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException; + + /** + * Called after the createTable operation has been requested. Called as part + * of create table RPC call. Called as part of create table procedure and + * it is async to the create RPC call. + * + * Implementation note: This replaces the deprecated + * {@link #postCreateTableHandler(ObserverContext, HTableDescriptor, HRegionInfo[])} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param desc the HTableDescriptor for the table + * @param regions the initial regions created for the table + * @throws IOException if something went wrong + */ + void postCompletedCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException; + + /** * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a * table. Called as part of delete table RPC call. * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void preDeleteTable(final ObserverContext ctx, TableName tableName) throws IOException; @@ -114,6 +163,7 @@ public interface MasterObserver extends Coprocessor { * of delete table RPC call. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void postDeleteTable(final ObserverContext ctx, TableName tableName) throws IOException; @@ -125,7 +175,12 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preDeleteTableAction(ObserverContext, TableName)}. */ + @Deprecated void preDeleteTableHandler( final ObserverContext ctx, TableName tableName) throws IOException; @@ -137,11 +192,51 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedDeleteTableAction(ObserverContext, TableName)}. */ + @Deprecated void postDeleteTableHandler( final ObserverContext ctx, TableName tableName) throws IOException; + /** + * Called before {@link org.apache.hadoop.hbase.master.HMaster} deletes a + * table. Called as part of delete table procedure and + * it is async to the delete RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #preDeleteTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void preDeleteTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException; + + /** + * Called after {@link org.apache.hadoop.hbase.master.HMaster} deletes a + * table. Called as part of delete table procedure and it is async to the + * delete RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #postDeleteTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void postCompletedDeleteTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException; /** * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a @@ -149,6 +244,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void preTruncateTable(final ObserverContext ctx, TableName tableName) throws IOException; @@ -160,6 +256,7 @@ public interface MasterObserver extends Coprocessor { * truncate operation is terminated. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void postTruncateTable(final ObserverContext ctx, TableName tableName) throws IOException; @@ -171,7 +268,12 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preTruncateTableAction(ObserverContext, TableName)}. */ + @Deprecated void preTruncateTableHandler( final ObserverContext ctx, TableName tableName) throws IOException; @@ -183,18 +285,60 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedTruncateTableAction(ObserverContext, TableName)}. */ + @Deprecated void postTruncateTableHandler( final ObserverContext ctx, TableName tableName) throws IOException; /** + * Called before {@link org.apache.hadoop.hbase.master.HMaster} truncates a + * table. Called as part of truncate table procedure and it is async + * to the truncate RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #preTruncateTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void preTruncateTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException; + + /** + * Called after {@link org.apache.hadoop.hbase.master.HMaster} truncates a + * table. Called as part of truncate table procedure and it is async to the + * truncate RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #postTruncateTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void postCompletedTruncateTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException; + + /** * Called prior to modifying a table's properties. Called as part of modify * table RPC call. * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param htd the HTableDescriptor + * @throws IOException if something went wrong */ void preModifyTable(final ObserverContext ctx, final TableName tableName, HTableDescriptor htd) throws IOException; @@ -205,6 +349,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param htd the HTableDescriptor + * @throws IOException if something went wrong */ void postModifyTable(final ObserverContext ctx, final TableName tableName, HTableDescriptor htd) throws IOException; @@ -216,7 +361,12 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param htd the HTableDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preModifyTableAction(ObserverContext, TableName, HTableDescriptor)}. */ + @Deprecated void preModifyTableHandler( final ObserverContext ctx, final TableName tableName, HTableDescriptor htd) throws IOException; @@ -228,17 +378,61 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param htd the HTableDescriptor + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-13645). + * Use {@link #postCompletedModifyTableAction(ObserverContext, TableName, HTableDescriptor)}. */ + @Deprecated void postModifyTableHandler( final ObserverContext ctx, final TableName tableName, HTableDescriptor htd) throws IOException; /** + * Called prior to modifying a table's properties. Called as part of modify + * table procedure and it is async to the modify table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #preModifyTableHandler(ObserverContext, TableName, HTableDescriptor)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param htd the HTableDescriptor + * @throws IOException if something went wrong + */ + void preModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException; + + /** + * Called after to modifying a table's properties. Called as part of modify + * table procedure and it is async to the modify table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #postModifyTableHandler(ObserverContext, TableName, HTableDescriptor)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @param htd the HTableDescriptor + * @throws IOException if something went wrong + */ + void postCompletedModifyTableAction( + final ObserverContext ctx, + final TableName tableName, + final HTableDescriptor htd) throws IOException; + + /** * Called prior to adding a new column family to the table. Called as part of * add column RPC call. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 * (HBASE-13645). * Use {@link #preAddColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. @@ -258,6 +452,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ void preAddColumnFamily(final ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException; @@ -268,6 +463,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 * (HBASE-13645). * Use {@link #postAddColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. @@ -287,6 +483,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ void postAddColumnFamily(final ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException; @@ -297,9 +494,10 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 - * (HBASE-13645). Use - * {@link #preAddColumnFamilyHandler(ObserverContext, TableName, HColumnDescriptor)}. + * (HBASE-13645). Use + * {@link #preAddColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}. */ @Deprecated void preAddColumnHandler( @@ -308,7 +506,7 @@ public interface MasterObserver extends Coprocessor { /** * Called prior to adding a new column family to the table. Called as part of - * add column handler. + * add column procedure. * * Implementation note: This replaces the deprecated * {@link #preAddColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method. @@ -317,11 +515,12 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ - void preAddColumnFamilyHandler( + void preAddColumnFamilyAction( final ObserverContext ctx, - TableName tableName, HColumnDescriptor columnFamily) throws IOException; - + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException; /** * Called after the new column family has been created. Called as part of @@ -329,9 +528,10 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 - * (HBASE-13645). Use - * {@link #postAddColumnFamilyHandler(ObserverContext, TableName, HColumnDescriptor)}. + * (HBASE-13645). Use + * {@link #postCompletedAddColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}. */ @Deprecated void postAddColumnHandler( @@ -340,7 +540,7 @@ public interface MasterObserver extends Coprocessor { /** * Called after the new column family has been created. Called as part of - * add column handler. + * add column procedure. * * Implementation note: This replaces the deprecated * {@link #postAddColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method. @@ -349,10 +549,12 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ - void postAddColumnFamilyHandler( + void postCompletedAddColumnFamilyAction( final ObserverContext ctx, - TableName tableName, HColumnDescriptor columnFamily) throws IOException; + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException; /** * Called prior to modifying a column family's attributes. Called as part of @@ -360,6 +562,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 * (HBASE-13645). * Use {@link #preModifyColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. @@ -379,6 +582,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ void preModifyColumnFamily(final ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException; @@ -389,6 +593,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 * (HBASE-13645). * Use {@link #postModifyColumnFamily(ObserverContext, TableName, HColumnDescriptor)}. @@ -408,6 +613,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ void postModifyColumnFamily(final ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException; @@ -418,9 +624,10 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 - * (HBASE-13645). - * Use {@link #preModifyColumnFamilyHandler(ObserverContext, TableName, HColumnDescriptor)}. + * (HBASE-13645). + * Use {@link #preModifyColumnFamilyAction(ObserverContext, TableName, HColumnDescriptor)}. */ @Deprecated void preModifyColumnHandler( @@ -429,7 +636,7 @@ public interface MasterObserver extends Coprocessor { /** * Called prior to modifying a column family's attributes. Called as part of - * modify column handler. + * modify column procedure. * * Implementation note: This replaces the deprecated * {@link #preModifyColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method. @@ -438,10 +645,12 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ - void preModifyColumnFamilyHandler( + void preModifyColumnFamilyAction( final ObserverContext ctx, - TableName tableName, HColumnDescriptor columnFamily) throws IOException; + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException; /** * Called after the column family has been updated. Called as part of modify @@ -449,9 +658,10 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 - * (HBASE-13645). - * Use {@link #postModifyColumnFamilyHandler(ObserverContext, TableName, HColumnDescriptor)}. + * (HBASE-13645). Use + * {@link #postCompletedModifyColumnFamilyAction(ObserverContext,TableName,HColumnDescriptor)}. */ @Deprecated void postModifyColumnHandler( @@ -460,7 +670,7 @@ public interface MasterObserver extends Coprocessor { /** * Called after the column family has been updated. Called as part of modify - * column handler. + * column procedure. * * Implementation note: This replaces the deprecated * {@link #postModifyColumnHandler(ObserverContext, TableName, HColumnDescriptor)} method. @@ -469,10 +679,12 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the HColumnDescriptor + * @throws IOException if something went wrong */ - void postModifyColumnFamilyHandler( + void postCompletedModifyColumnFamilyAction( final ObserverContext ctx, - TableName tableName, HColumnDescriptor columnFamily) throws IOException; + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException; /** * Called prior to deleting the entire column family. Called as part of @@ -480,6 +692,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column family + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 * (HBASE-13645). * Use {@link #preDeleteColumnFamily(ObserverContext, TableName, byte[])}. @@ -499,6 +712,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column + * @throws IOException if something went wrong */ void preDeleteColumnFamily(final ObserverContext ctx, final TableName tableName, final byte[] columnFamily) throws IOException; @@ -509,6 +723,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column family + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 * (HBASE-13645). * Use {@link #postDeleteColumnFamily(ObserverContext, TableName, byte[])}. @@ -528,6 +743,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column family + * @throws IOException if something went wrong */ void postDeleteColumnFamily(final ObserverContext ctx, final TableName tableName, final byte[] columnFamily) throws IOException; @@ -538,9 +754,10 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column family + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 * (HBASE-13645). - * Use {@link #preDeleteColumnFamilyHandler(ObserverContext, TableName, byte[])}. + * Use {@link #preDeleteColumnFamilyAction(ObserverContext, TableName, byte[])}. */ @Deprecated void preDeleteColumnHandler( @@ -549,7 +766,7 @@ public interface MasterObserver extends Coprocessor { /** * Called prior to deleting the entire column family. Called as part of - * delete column handler. + * delete column procedure. * * Implementation note: This replaces the deprecated * {@link #preDeleteColumnHandler(ObserverContext, TableName, byte[])} method. @@ -558,8 +775,9 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column family + * @throws IOException if something went wrong */ - void preDeleteColumnFamilyHandler( + void preDeleteColumnFamilyAction( final ObserverContext ctx, final TableName tableName, final byte[] columnFamily) throws IOException; @@ -569,9 +787,10 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column family + * @throws IOException if something went wrong * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 - * (HBASE-13645). - * Use {@link #postDeleteColumnFamilyHandler(ObserverContext, TableName, byte[])}. + * (HBASE-13645). + * Use {@link #postCompletedDeleteColumnFamilyAction(ObserverContext, TableName, byte[])}. */ @Deprecated void postDeleteColumnHandler( @@ -580,7 +799,7 @@ public interface MasterObserver extends Coprocessor { /** * Called after the column family has been deleted. Called as part of - * delete column handler. + * delete column procedure. * * Implementation note: This replaces the deprecated * {@link #postDeleteColumnHandler(ObserverContext, TableName, byte[])} method. @@ -589,8 +808,9 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param columnFamily the column family + * @throws IOException if something went wrong */ - void postDeleteColumnFamilyHandler( + void postCompletedDeleteColumnFamilyAction( final ObserverContext ctx, final TableName tableName, final byte[] columnFamily) throws IOException; @@ -599,6 +819,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void preEnableTable(final ObserverContext ctx, final TableName tableName) throws IOException; @@ -608,6 +829,7 @@ public interface MasterObserver extends Coprocessor { * of enable table RPC call. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void postEnableTable(final ObserverContext ctx, final TableName tableName) throws IOException; @@ -618,7 +840,12 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preEnableTableAction(ObserverContext, TableName)}. */ + @Deprecated void preEnableTableHandler( final ObserverContext ctx, final TableName tableName) throws IOException; @@ -628,17 +855,56 @@ public interface MasterObserver extends Coprocessor { * of enable table handler and it is async to the enable table RPC call. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedEnableTableAction(ObserverContext, TableName)}. */ + @Deprecated void postEnableTableHandler( final ObserverContext ctx, final TableName tableName) throws IOException; /** + * Called prior to enabling a table. Called as part of enable table procedure + * and it is async to the enable table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #preEnableTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void preEnableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException; + + /** + * Called after the enableTable operation has been requested. Called as part + * of enable table procedure and it is async to the enable table RPC call. + * + * Implementation note: This replaces the deprecated + * {@link #postEnableTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void postCompletedEnableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException; + + /** * Called prior to disabling a table. Called as part of disable table RPC * call. * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void preDisableTable(final ObserverContext ctx, final TableName tableName) throws IOException; @@ -648,6 +914,7 @@ public interface MasterObserver extends Coprocessor { * of disable table RPC call. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong */ void postDisableTable(final ObserverContext ctx, final TableName tableName) throws IOException; @@ -658,7 +925,12 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #preDisableTableAction(ObserverContext, TableName)}. */ + @Deprecated void preDisableTableHandler( final ObserverContext ctx, final TableName tableName) throws IOException; @@ -668,15 +940,55 @@ public interface MasterObserver extends Coprocessor { * of disable table handler and it is asyn to the disable table RPC call. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table + * @throws IOException if something went wrong + * @deprecated As of release 2.0.0, this will be removed in HBase 3.0.0 + * (HBASE-15575). + * Use {@link #postCompletedDisableTableAction(ObserverContext, TableName)}. */ + @Deprecated void postDisableTableHandler( final ObserverContext ctx, final TableName tableName) throws IOException; /** + * Called prior to disabling a table. Called as part of disable table procedure + * and it is asyn to the disable table RPC call. + * It can't bypass the default action, e.g., ctx.bypass() won't have effect. + * + * Implementation note: This replaces the deprecated + * {@link #preDisableTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void preDisableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException; + + /** + * Called after the disableTable operation has been requested. Called as part + * of disable table procedure and it is asyn to the disable table RPC call. + * + * Implementation note: This replaces the deprecated + * {@link #postDisableTableHandler(ObserverContext, TableName)} method. + * Make sure to implement only one of the two as both are called. + * + * @param ctx the environment to interact with the framework and master + * @param tableName the name of the table + * @throws IOException if something went wrong + */ + void postCompletedDisableTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException; + + /** * Called before a abortProcedure request has been processed. * @param ctx the environment to interact with the framework and master - * @throws IOException + * @param procEnv procedure executor + * @param procId the Id of the procedure + * @throws IOException if something went wrong */ public void preAbortProcedure( ObserverContext ctx, @@ -686,6 +998,7 @@ public interface MasterObserver extends Coprocessor { /** * Called after a abortProcedure request has been processed. * @param ctx the environment to interact with the framework and master + * @throws IOException if something went wrong */ public void postAbortProcedure(ObserverContext ctx) throws IOException; @@ -693,7 +1006,7 @@ public interface MasterObserver extends Coprocessor { /** * Called before a listProcedures request has been processed. * @param ctx the environment to interact with the framework and master - * @throws IOException + * @throws IOException if something went wrong */ void preListProcedures(ObserverContext ctx) throws IOException; @@ -702,6 +1015,7 @@ public interface MasterObserver extends Coprocessor { * Called after a listProcedures request has been processed. * @param ctx the environment to interact with the framework and master * @param procInfoList the list of procedures about to be returned + * @throws IOException if something went wrong */ void postListProcedures( ObserverContext ctx, @@ -872,7 +1186,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor for the snapshot * @param hTableDescriptor the hTableDescriptor of the table to snapshot - * @throws IOException + * @throws IOException if something went wrong */ void preSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) @@ -884,7 +1198,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor for the snapshot * @param hTableDescriptor the hTableDescriptor of the table to snapshot - * @throws IOException + * @throws IOException if something went wrong */ void postSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) @@ -895,7 +1209,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor of the snapshot to list - * @throws IOException + * @throws IOException if something went wrong */ void preListSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot) throws IOException; @@ -905,7 +1219,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor of the snapshot to list - * @throws IOException + * @throws IOException if something went wrong */ void postListSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot) throws IOException; @@ -917,7 +1231,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor for the snapshot * @param hTableDescriptor the hTableDescriptor of the table to create - * @throws IOException + * @throws IOException if something went wrong */ void preCloneSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) @@ -929,7 +1243,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor for the snapshot * @param hTableDescriptor the hTableDescriptor of the table to create - * @throws IOException + * @throws IOException if something went wrong */ void postCloneSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) @@ -942,7 +1256,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor for the snapshot * @param hTableDescriptor the hTableDescriptor of the table to restore - * @throws IOException + * @throws IOException if something went wrong */ void preRestoreSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) @@ -954,7 +1268,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor for the snapshot * @param hTableDescriptor the hTableDescriptor of the table to restore - * @throws IOException + * @throws IOException if something went wrong */ void postRestoreSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot, final HTableDescriptor hTableDescriptor) @@ -966,7 +1280,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor of the snapshot to delete - * @throws IOException + * @throws IOException if something went wrong */ void preDeleteSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot) throws IOException; @@ -976,7 +1290,7 @@ public interface MasterObserver extends Coprocessor { * Called as part of deleteSnapshot RPC call. * @param ctx the environment to interact with the framework and master * @param snapshot the SnapshotDescriptor of the snapshot to delete - * @throws IOException + * @throws IOException if something went wrong */ void postDeleteSnapshot(final ObserverContext ctx, final SnapshotDescription snapshot) throws IOException; @@ -987,7 +1301,7 @@ public interface MasterObserver extends Coprocessor { * @param tableNamesList the list of table names, or null if querying for all * @param descriptors an empty list, can be filled with what to return if bypassing * @param regex regular expression used for filtering the table names - * @throws IOException + * @throws IOException if something went wrong */ void preGetTableDescriptors(ObserverContext ctx, List tableNamesList, List descriptors, @@ -999,7 +1313,7 @@ public interface MasterObserver extends Coprocessor { * @param tableNamesList the list of table names, or null if querying for all * @param descriptors the list of descriptors about to be returned * @param regex regular expression used for filtering the table names - * @throws IOException + * @throws IOException if something went wrong */ void postGetTableDescriptors(ObserverContext ctx, List tableNamesList, List descriptors, @@ -1010,7 +1324,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param descriptors an empty list, can be filled with what to return if bypassing * @param regex regular expression used for filtering the table names - * @throws IOException + * @throws IOException if something went wrong */ void preGetTableNames(ObserverContext ctx, List descriptors, String regex) throws IOException; @@ -1020,7 +1334,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param descriptors the list of descriptors about to be returned * @param regex regular expression used for filtering the table names - * @throws IOException + * @throws IOException if something went wrong */ void postGetTableNames(ObserverContext ctx, List descriptors, String regex) throws IOException; @@ -1033,7 +1347,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param ns the NamespaceDescriptor for the table - * @throws IOException + * @throws IOException if something went wrong */ void preCreateNamespace(final ObserverContext ctx, NamespaceDescriptor ns) throws IOException; @@ -1041,7 +1355,7 @@ public interface MasterObserver extends Coprocessor { * Called after the createNamespace operation has been requested. * @param ctx the environment to interact with the framework and master * @param ns the NamespaceDescriptor for the table - * @throws IOException + * @throws IOException if something went wrong */ void postCreateNamespace(final ObserverContext ctx, NamespaceDescriptor ns) throws IOException; @@ -1052,6 +1366,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param namespace the name of the namespace + * @throws IOException if something went wrong */ void preDeleteNamespace(final ObserverContext ctx, String namespace) throws IOException; @@ -1060,6 +1375,7 @@ public interface MasterObserver extends Coprocessor { * Called after the deleteNamespace operation has been requested. * @param ctx the environment to interact with the framework and master * @param namespace the name of the namespace + * @throws IOException if something went wrong */ void postDeleteNamespace(final ObserverContext ctx, String namespace) throws IOException; @@ -1069,6 +1385,7 @@ public interface MasterObserver extends Coprocessor { * It can't bypass the default action, e.g., ctx.bypass() won't have effect. * @param ctx the environment to interact with the framework and master * @param ns the NamespaceDescriptor + * @throws IOException if something went wrong */ void preModifyNamespace(final ObserverContext ctx, NamespaceDescriptor ns) throws IOException; @@ -1077,6 +1394,7 @@ public interface MasterObserver extends Coprocessor { * Called after the modifyNamespace operation has been requested. * @param ctx the environment to interact with the framework and master * @param ns the NamespaceDescriptor + * @throws IOException if something went wrong */ void postModifyNamespace(final ObserverContext ctx, NamespaceDescriptor ns) throws IOException; @@ -1085,7 +1403,7 @@ public interface MasterObserver extends Coprocessor { * Called before a getNamespaceDescriptor request has been processed. * @param ctx the environment to interact with the framework and master * @param namespace the name of the namespace - * @throws IOException + * @throws IOException if something went wrong */ void preGetNamespaceDescriptor(ObserverContext ctx, String namespace) throws IOException; @@ -1094,7 +1412,7 @@ public interface MasterObserver extends Coprocessor { * Called after a getNamespaceDescriptor request has been processed. * @param ctx the environment to interact with the framework and master * @param ns the NamespaceDescriptor - * @throws IOException + * @throws IOException if something went wrong */ void postGetNamespaceDescriptor(ObserverContext ctx, NamespaceDescriptor ns) throws IOException; @@ -1103,7 +1421,7 @@ public interface MasterObserver extends Coprocessor { * Called before a listNamespaceDescriptors request has been processed. * @param ctx the environment to interact with the framework and master * @param descriptors an empty list, can be filled with what to return if bypassing - * @throws IOException + * @throws IOException if something went wrong */ void preListNamespaceDescriptors(ObserverContext ctx, List descriptors) throws IOException; @@ -1112,7 +1430,7 @@ public interface MasterObserver extends Coprocessor { * Called after a listNamespaceDescriptors request has been processed. * @param ctx the environment to interact with the framework and master * @param descriptors the list of descriptors about to be returned - * @throws IOException + * @throws IOException if something went wrong */ void postListNamespaceDescriptors(ObserverContext ctx, List descriptors) throws IOException; @@ -1122,7 +1440,7 @@ public interface MasterObserver extends Coprocessor { * Called before the table memstore is flushed to disk. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table - * @throws IOException + * @throws IOException if something went wrong */ void preTableFlush(final ObserverContext ctx, final TableName tableName) throws IOException; @@ -1131,7 +1449,7 @@ public interface MasterObserver extends Coprocessor { * Called after the table memstore is flushed to disk. * @param ctx the environment to interact with the framework and master * @param tableName the name of the table - * @throws IOException + * @throws IOException if something went wrong */ void postTableFlush(final ObserverContext ctx, final TableName tableName) throws IOException; @@ -1141,7 +1459,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param userName the name of user * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void preSetUserQuota(final ObserverContext ctx, final String userName, final Quotas quotas) throws IOException; @@ -1151,7 +1469,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param userName the name of user * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void postSetUserQuota(final ObserverContext ctx, final String userName, final Quotas quotas) throws IOException; @@ -1162,7 +1480,7 @@ public interface MasterObserver extends Coprocessor { * @param userName the name of user * @param tableName the name of the table * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void preSetUserQuota(final ObserverContext ctx, final String userName, final TableName tableName, final Quotas quotas) throws IOException; @@ -1173,7 +1491,7 @@ public interface MasterObserver extends Coprocessor { * @param userName the name of user * @param tableName the name of the table * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void postSetUserQuota(final ObserverContext ctx, final String userName, final TableName tableName, final Quotas quotas) throws IOException; @@ -1184,7 +1502,7 @@ public interface MasterObserver extends Coprocessor { * @param userName the name of user * @param namespace the name of the namespace * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void preSetUserQuota(final ObserverContext ctx, final String userName, final String namespace, final Quotas quotas) throws IOException; @@ -1195,7 +1513,7 @@ public interface MasterObserver extends Coprocessor { * @param userName the name of user * @param namespace the name of the namespace * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void postSetUserQuota(final ObserverContext ctx, final String userName, final String namespace, final Quotas quotas) throws IOException; @@ -1205,7 +1523,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void preSetTableQuota(final ObserverContext ctx, final TableName tableName, final Quotas quotas) throws IOException; @@ -1215,7 +1533,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param tableName the name of the table * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void postSetTableQuota(final ObserverContext ctx, final TableName tableName, final Quotas quotas) throws IOException; @@ -1225,7 +1543,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param namespace the name of the namespace * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void preSetNamespaceQuota(final ObserverContext ctx, final String namespace, final Quotas quotas) throws IOException; @@ -1235,7 +1553,7 @@ public interface MasterObserver extends Coprocessor { * @param ctx the environment to interact with the framework and master * @param namespace the name of the namespace * @param quotas the quota settings - * @throws IOException + * @throws IOException if something went wrong */ void postSetNamespaceQuota(final ObserverContext ctx, final String namespace, final Quotas quotas) throws IOException; diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java index ff3d792..c2cab33 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java @@ -1082,7 +1082,7 @@ public class HMaster extends HRegionServer implements MasterServices { // at a time. To do concurrency, would need fencing of enable/disable of // tables. // Any time changing this maxThreads to > 1, pls see the comment at - // AccessController#postCreateTableHandler + // AccessController#postCompletedCreateTableAction this.service.startExecutorService(ExecutorType.MASTER_TABLE_OPERATIONS, 1); startProcedureExecutor(); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java index 14e8c20..459fd01 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterCoprocessorHost.java @@ -240,24 +240,26 @@ public class MasterCoprocessorHost }); } - public void preCreateTableHandler(final HTableDescriptor htd, final HRegionInfo[] regions) + public void preCreateTableAction(final HTableDescriptor htd, final HRegionInfo[] regions) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preCreateTableHandler(ctx, htd, regions); + oserver.preCreateTableAction(ctx, htd, regions); } }); } - public void postCreateTableHandler(final HTableDescriptor htd, final HRegionInfo[] regions) - throws IOException { + public void postCompletedCreateTableAction( + final HTableDescriptor htd, final HRegionInfo[] regions) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postCreateTableHandler(ctx, htd, regions); + oserver.postCompletedCreateTableAction(ctx, htd, regions); } }); } @@ -282,22 +284,24 @@ public class MasterCoprocessorHost }); } - public void preDeleteTableHandler(final TableName tableName) throws IOException { + public void preDeleteTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preDeleteTableHandler(ctx, tableName); + oserver.preDeleteTableAction(ctx, tableName); } }); } - public void postDeleteTableHandler(final TableName tableName) throws IOException { + public void postCompletedDeleteTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postDeleteTableHandler(ctx, tableName); + oserver.postCompletedDeleteTableAction(ctx, tableName); } }); } @@ -322,22 +326,24 @@ public class MasterCoprocessorHost }); } - public void preTruncateTableHandler(final TableName tableName) throws IOException { + public void preTruncateTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preTruncateTableHandler(ctx, tableName); + oserver.preTruncateTableAction(ctx, tableName); } }); } - public void postTruncateTableHandler(final TableName tableName) throws IOException { + public void postCompletedTruncateTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postTruncateTableHandler(ctx, tableName); + oserver.postCompletedTruncateTableAction(ctx, tableName); } }); } @@ -364,24 +370,26 @@ public class MasterCoprocessorHost }); } - public void preModifyTableHandler(final TableName tableName, final HTableDescriptor htd) + public void preModifyTableAction(final TableName tableName, final HTableDescriptor htd) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preModifyTableHandler(ctx, tableName, htd); + oserver.preModifyTableAction(ctx, tableName, htd); } }); } - public void postModifyTableHandler(final TableName tableName, final HTableDescriptor htd) + public void postCompletedModifyTableAction(final TableName tableName, final HTableDescriptor htd) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postModifyTableHandler(ctx, tableName, htd); + oserver.postCompletedModifyTableAction(ctx, tableName, htd); } }); } @@ -410,27 +418,30 @@ public class MasterCoprocessorHost }); } - public boolean preAddColumnHandler(final TableName tableName, - final HColumnDescriptor columnFamily) + public boolean preAddColumnFamilyAction( + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preAddColumnHandler(ctx, tableName, columnFamily); - oserver.preAddColumnFamilyHandler(ctx, tableName, columnFamily); + oserver.preAddColumnFamilyAction(ctx, tableName, columnFamily); } }); } - public void postAddColumnHandler(final TableName tableName, final HColumnDescriptor columnFamily) + public void postCompletedAddColumnFamilyAction( + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postAddColumnHandler(ctx, tableName, columnFamily); - oserver.postAddColumnFamilyHandler(ctx, tableName, columnFamily); + oserver.postCompletedAddColumnFamilyAction(ctx, tableName, columnFamily); } }); } @@ -459,26 +470,28 @@ public class MasterCoprocessorHost }); } - public boolean preModifyColumnHandler(final TableName tableName, + public boolean preModifyColumnFamilyAction( + final TableName tableName, final HColumnDescriptor columnFamily) throws IOException { return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preModifyColumnHandler(ctx, tableName, columnFamily); - oserver.preModifyColumnFamilyHandler(ctx, tableName, columnFamily); + oserver.preModifyColumnFamilyAction(ctx, tableName, columnFamily); } }); } - public void postModifyColumnHandler(final TableName tableName, + public void postCompletedModifyColumnFamilyAction( + final TableName tableName, final HColumnDescriptor columnFamily) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postModifyColumnHandler(ctx, tableName, columnFamily); - oserver.postModifyColumnFamilyHandler(ctx, tableName, columnFamily); + oserver.postCompletedModifyColumnFamilyAction(ctx, tableName, columnFamily); } }); } @@ -507,26 +520,28 @@ public class MasterCoprocessorHost }); } - public boolean preDeleteColumnHandler(final TableName tableName, final byte[] columnFamily) + public boolean preDeleteColumnFamilyAction( + final TableName tableName, + final byte[] columnFamily) throws IOException { return execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preDeleteColumnHandler(ctx, tableName, columnFamily); - oserver.preDeleteColumnFamilyHandler(ctx, tableName, columnFamily); + oserver.preDeleteColumnFamilyAction(ctx, tableName, columnFamily); } }); } - public void postDeleteColumnHandler(final TableName tableName, final byte[] columnFamily) - throws IOException { + public void postCompletedDeleteColumnFamilyAction( + final TableName tableName, final byte[] columnFamily) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postDeleteColumnHandler(ctx, tableName, columnFamily); - oserver.postDeleteColumnFamilyHandler(ctx, tableName, columnFamily); + oserver.postCompletedDeleteColumnFamilyAction(ctx, tableName, columnFamily); } }); } @@ -551,22 +566,24 @@ public class MasterCoprocessorHost }); } - public void preEnableTableHandler(final TableName tableName) throws IOException { + public void preEnableTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preEnableTableHandler(ctx, tableName); + oserver.preEnableTableAction(ctx, tableName); } }); } - public void postEnableTableHandler(final TableName tableName) throws IOException { + public void postCompletedEnableTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postEnableTableHandler(ctx, tableName); + oserver.postCompletedEnableTableAction(ctx, tableName); } }); } @@ -591,22 +608,24 @@ public class MasterCoprocessorHost }); } - public void preDisableTableHandler(final TableName tableName) throws IOException { + public void preDisableTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.preDisableTableHandler(ctx, tableName); + oserver.preDisableTableAction(ctx, tableName); } }); } - public void postDisableTableHandler(final TableName tableName) throws IOException { + public void postCompletedDisableTableAction(final TableName tableName) throws IOException { execOperation(coprocessors.isEmpty() ? null : new CoprocessorOperation() { @Override public void call(MasterObserver oserver, ObserverContext ctx) throws IOException { oserver.postDisableTableHandler(ctx, tableName); + oserver.postCompletedDisableTableAction(ctx, tableName); } }); } diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java index d34f25e..bfa9b5f 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/DisableTableHandler.java @@ -126,11 +126,11 @@ public class DisableTableHandler extends EventHandler { MasterCoprocessorHost cpHost = ((HMaster) this.server) .getMasterCoprocessorHost(); if (cpHost != null) { - cpHost.preDisableTableHandler(this.tableName); + cpHost.preDisableTableAction(this.tableName); } handleDisableTable(); if (cpHost != null) { - cpHost.postDisableTableHandler(this.tableName); + cpHost.postCompletedDisableTableAction(this.tableName); } } catch (IOException e) { LOG.error("Error trying to disable table " + this.tableName, e); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java index fd3d4c7..7edf3f1 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/handler/EnableTableHandler.java @@ -135,11 +135,11 @@ public class EnableTableHandler extends EventHandler { MasterCoprocessorHost cpHost = ((HMaster) this.server) .getMasterCoprocessorHost(); if (cpHost != null) { - cpHost.preEnableTableHandler(this.tableName); + cpHost.preEnableTableAction(this.tableName); } handleEnableTable(); if (cpHost != null) { - cpHost.postEnableTableHandler(this.tableName); + cpHost.postCompletedEnableTableAction(this.tableName); } } catch (IOException | InterruptedException e) { LOG.error("Error trying to enable the table " + this.tableName, e); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java index 2571829..4a6c67d 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/AddColumnFamilyProcedure.java @@ -383,10 +383,10 @@ public class AddColumnFamilyProcedure public Void run() throws Exception { switch (state) { case ADD_COLUMN_FAMILY_PRE_OPERATION: - cpHost.preAddColumnHandler(tableName, cfDescriptor); + cpHost.preAddColumnFamilyAction(tableName, cfDescriptor); break; case ADD_COLUMN_FAMILY_POST_OPERATION: - cpHost.postAddColumnHandler(tableName, cfDescriptor); + cpHost.postCompletedAddColumnFamilyAction(tableName, cfDescriptor); break; default: throw new UnsupportedOperationException(this + " unhandled state=" + state); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java index 2a84a15..c008b36 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CloneSnapshotProcedure.java @@ -376,7 +376,7 @@ public class CloneSnapshotProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.preCreateTableHandler(hTableDescriptor, null); + cpHost.preCreateTableAction(hTableDescriptor, null); return null; } }); @@ -398,7 +398,7 @@ public class CloneSnapshotProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.postCreateTableHandler(hTableDescriptor, regions); + cpHost.postCompletedCreateTableAction(hTableDescriptor, regions); return null; } }); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java index 51fc74c..110bccd 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/CreateTableProcedure.java @@ -311,7 +311,7 @@ public class CreateTableProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.preCreateTableHandler(hTableDescriptor, regions); + cpHost.preCreateTableAction(hTableDescriptor, regions); return null; } }); @@ -327,7 +327,7 @@ public class CreateTableProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.postCreateTableHandler(hTableDescriptor, regions); + cpHost.postCompletedCreateTableAction(hTableDescriptor, regions); return null; } }); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java index 442ed72..1e510b3 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteColumnFamilyProcedure.java @@ -408,10 +408,10 @@ public class DeleteColumnFamilyProcedure public Void run() throws Exception { switch (state) { case DELETE_COLUMN_FAMILY_PRE_OPERATION: - cpHost.preDeleteColumnHandler(tableName, familyName); + cpHost.preDeleteColumnFamilyAction(tableName, familyName); break; case DELETE_COLUMN_FAMILY_POST_OPERATION: - cpHost.postDeleteColumnHandler(tableName, familyName); + cpHost.postCompletedDeleteColumnFamilyAction(tableName, familyName); break; default: throw new UnsupportedOperationException(this + " unhandled state=" + state); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java index ff22b88..4c92fb5 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DeleteTableProcedure.java @@ -269,7 +269,7 @@ public class DeleteTableProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.preDeleteTableHandler(tableName); + cpHost.preDeleteTableAction(tableName); return null; } }); @@ -287,7 +287,7 @@ public class DeleteTableProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.postDeleteTableHandler(tableName); + cpHost.postCompletedDeleteTableAction(tableName); return null; } }); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java index 005069d..6a65690 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/DisableTableProcedure.java @@ -464,10 +464,10 @@ public class DisableTableProcedure public Void run() throws Exception { switch (state) { case DISABLE_TABLE_PRE_OPERATION: - cpHost.preDisableTableHandler(tableName); + cpHost.preDisableTableAction(tableName); break; case DISABLE_TABLE_POST_OPERATION: - cpHost.postDisableTableHandler(tableName); + cpHost.postCompletedDisableTableAction(tableName); break; default: throw new UnsupportedOperationException(this + " unhandled state=" + state); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java index d8af419..862231e 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/EnableTableProcedure.java @@ -566,10 +566,10 @@ public class EnableTableProcedure public Void run() throws Exception { switch (state) { case ENABLE_TABLE_PRE_OPERATION: - cpHost.preEnableTableHandler(getTableName()); + cpHost.preEnableTableAction(getTableName()); break; case ENABLE_TABLE_POST_OPERATION: - cpHost.postEnableTableHandler(getTableName()); + cpHost.postCompletedEnableTableAction(getTableName()); break; default: throw new UnsupportedOperationException(this + " unhandled state=" + state); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java index 0479c36..0b7d8a2 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyColumnFamilyProcedure.java @@ -364,10 +364,10 @@ public class ModifyColumnFamilyProcedure public Void run() throws Exception { switch (state) { case MODIFY_COLUMN_FAMILY_PRE_OPERATION: - cpHost.preModifyColumnHandler(tableName, cfDescriptor); + cpHost.preModifyColumnFamilyAction(tableName, cfDescriptor); break; case MODIFY_COLUMN_FAMILY_POST_OPERATION: - cpHost.postModifyColumnHandler(tableName, cfDescriptor); + cpHost.postCompletedModifyColumnFamilyAction(tableName, cfDescriptor); break; default: throw new UnsupportedOperationException(this + " unhandled state=" + state); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java index 8e907ec..8f088da 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/ModifyTableProcedure.java @@ -477,10 +477,10 @@ public class ModifyTableProcedure public Void run() throws Exception { switch (state) { case MODIFY_TABLE_PRE_OPERATION: - cpHost.preModifyTableHandler(getTableName(), modifiedHTableDescriptor); + cpHost.preModifyTableAction(getTableName(), modifiedHTableDescriptor); break; case MODIFY_TABLE_POST_OPERATION: - cpHost.postModifyTableHandler(getTableName(), modifiedHTableDescriptor); + cpHost.postCompletedModifyTableAction(getTableName(), modifiedHTableDescriptor); break; default: throw new UnsupportedOperationException(this + " unhandled state=" + state); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java index 40f8157..9abed52 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/procedure/TruncateTableProcedure.java @@ -264,7 +264,7 @@ public class TruncateTableProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.preTruncateTableHandler(tableName); + cpHost.preTruncateTableAction(tableName); return null; } }); @@ -280,7 +280,7 @@ public class TruncateTableProcedure user.doAs(new PrivilegedExceptionAction() { @Override public Void run() throws Exception { - cpHost.postTruncateTableHandler(tableName); + cpHost.postCompletedTruncateTableAction(tableName); return null; } }); diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java index e866f29..f21d8e2 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/security/access/AccessController.java @@ -1005,8 +1005,10 @@ public class AccessController extends BaseMasterAndRegionObserver } @Override - public void postCreateTableHandler(final ObserverContext c, - HTableDescriptor desc, HRegionInfo[] regions) throws IOException { + public void postCompletedCreateTableAction( + final ObserverContext c, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { // When AC is used, it should be configured as the 1st CP. // In Master, the table operations like create, are handled by a Thread pool but the max size // for this pool is 1. So if multiple CPs create tables on startup, these creations will happen diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java index 65c774e..06f8792 100644 --- hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java +++ hbase-server/src/main/java/org/apache/hadoop/hbase/wal/BoundedGroupingStrategy.java @@ -46,7 +46,7 @@ public class BoundedGroupingStrategy implements RegionGroupingStrategy{ String idStr = Bytes.toString(identifier); String groupName = groupNameCache.get(idStr); if (null == groupName) { - groupName = groupNames[counter.getAndIncrement() % groupNames.length]; + groupName = groupNames[getAndIncrAtomicInteger(counter, groupNames.length)]; String extantName = groupNameCache.putIfAbsent(idStr, groupName); if (extantName != null) { return extantName; @@ -55,6 +55,18 @@ public class BoundedGroupingStrategy implements RegionGroupingStrategy{ return groupName; } + // Non-blocking incrementing & resetting of AtomicInteger. + private int getAndIncrAtomicInteger(AtomicInteger atomicInt, int reset) { + for (;;) { + int current = atomicInt.get(); + int next = (current + 1); + if (next == reset) { + next = 0; + } + if (atomicInt.compareAndSet(current, next)) return current; + } + } + @Override public void init(Configuration config, String providerId) { int regionGroupNumber = config.getInt(NUM_REGION_GROUPS, DEFAULT_NUM_REGION_GROUPS); diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java index 5257cee..73917cd 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/TestMasterObserver.java @@ -152,24 +152,24 @@ public class TestMasterObserver { private boolean postRestoreSnapshotCalled; private boolean preDeleteSnapshotCalled; private boolean postDeleteSnapshotCalled; - private boolean preCreateTableHandlerCalled; - private boolean postCreateTableHandlerCalled; - private boolean preDeleteTableHandlerCalled; - private boolean postDeleteTableHandlerCalled; - private boolean preTruncateTableHandlerCalled; - private boolean postTruncateTableHandlerCalled; - private boolean preAddColumnHandlerCalled; - private boolean postAddColumnHandlerCalled; - private boolean preModifyColumnHandlerCalled; - private boolean postModifyColumnHandlerCalled; - private boolean preDeleteColumnHandlerCalled; - private boolean postDeleteColumnHandlerCalled; - private boolean preEnableTableHandlerCalled; - private boolean postEnableTableHandlerCalled; - private boolean preDisableTableHandlerCalled; - private boolean postDisableTableHandlerCalled; - private boolean preModifyTableHandlerCalled; - private boolean postModifyTableHandlerCalled; + private boolean preCreateTableActionCalled; + private boolean postCompletedCreateTableActionCalled; + private boolean preDeleteTableActionCalled; + private boolean postCompletedDeleteTableActionCalled; + private boolean preTruncateTableActionCalled; + private boolean postCompletedTruncateTableActionCalled; + private boolean preAddColumnFamilyActionCalled; + private boolean postCompletedAddColumnFamilyActionCalled; + private boolean preModifyColumnFamilyActionCalled; + private boolean postCompletedModifyColumnFamilyActionCalled; + private boolean preDeleteColumnFamilyActionCalled; + private boolean postCompletedDeleteColumnFamilyActionCalled; + private boolean preEnableTableActionCalled; + private boolean postCompletedEnableTableActionCalled; + private boolean preDisableTableActionCalled; + private boolean postCompletedDisableTableActionCalled; + private boolean preModifyTableActionCalled; + private boolean postCompletedModifyTableActionCalled; private boolean preGetTableDescriptorsCalled; private boolean postGetTableDescriptorsCalled; private boolean postGetTableNamesCalled; @@ -236,24 +236,24 @@ public class TestMasterObserver { postRestoreSnapshotCalled = false; preDeleteSnapshotCalled = false; postDeleteSnapshotCalled = false; - preCreateTableHandlerCalled = false; - postCreateTableHandlerCalled = false; - preDeleteTableHandlerCalled = false; - postDeleteTableHandlerCalled = false; - preTruncateTableHandlerCalled = false; - postTruncateTableHandlerCalled = false; - preModifyTableHandlerCalled = false; - postModifyTableHandlerCalled = false; - preAddColumnHandlerCalled = false; - postAddColumnHandlerCalled = false; - preModifyColumnHandlerCalled = false; - postModifyColumnHandlerCalled = false; - preDeleteColumnHandlerCalled = false; - postDeleteColumnHandlerCalled = false; - preEnableTableHandlerCalled = false; - postEnableTableHandlerCalled = false; - preDisableTableHandlerCalled = false; - postDisableTableHandlerCalled = false; + preCreateTableActionCalled = false; + postCompletedCreateTableActionCalled = false; + preDeleteTableActionCalled = false; + postCompletedDeleteTableActionCalled = false; + preTruncateTableActionCalled = false; + postCompletedTruncateTableActionCalled = false; + preModifyTableActionCalled = false; + postCompletedModifyTableActionCalled = false; + preAddColumnFamilyActionCalled = false; + postCompletedAddColumnFamilyActionCalled = false; + preModifyColumnFamilyActionCalled = false; + postCompletedModifyColumnFamilyActionCalled = false; + preDeleteColumnFamilyActionCalled = false; + postCompletedDeleteColumnFamilyActionCalled = false; + preEnableTableActionCalled = false; + postCompletedEnableTableActionCalled = false; + preDisableTableActionCalled = false; + postCompletedDisableTableActionCalled = false; preGetTableDescriptorsCalled = false; postGetTableDescriptorsCalled = false; postGetTableNamesCalled = false; @@ -962,109 +962,169 @@ public class TestMasterObserver { return preDeleteSnapshotCalled && postDeleteSnapshotCalled; } + @Deprecated @Override public void preCreateTableHandler( ObserverContext env, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { + } + + @Override + public void preCreateTableAction( + final ObserverContext env, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { if (bypass) { env.bypass(); } - preCreateTableHandlerCalled = true; + preCreateTableActionCalled = true; } + @Deprecated @Override public void postCreateTableHandler( ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { - postCreateTableHandlerCalled = true; + } + + @Override + public void postCompletedCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + postCompletedCreateTableActionCalled = true; tableCreationLatch.countDown(); } - public boolean wasPreCreateTableHandlerCalled(){ - return preCreateTableHandlerCalled; + public boolean wasPreCreateTableActionCalled(){ + return preCreateTableActionCalled; } - public boolean wasCreateTableHandlerCalled() { - return preCreateTableHandlerCalled && postCreateTableHandlerCalled; + public boolean wasCreateTableActionCalled() { + return preCreateTableActionCalled && postCompletedCreateTableActionCalled; } - public boolean wasCreateTableHandlerCalledOnly() { - return preCreateTableHandlerCalled && !postCreateTableHandlerCalled; + public boolean wasCreateTableActionCalledOnly() { + return preCreateTableActionCalled && !postCompletedCreateTableActionCalled; } + @Deprecated @Override public void preDeleteTableHandler( ObserverContext env, TableName tableName) throws IOException { + } + + @Deprecated + @Override + public void postDeleteTableHandler( + ObserverContext ctx, TableName tableName) + throws IOException { + } + + @Override + public void preDeleteTableAction( + final ObserverContext env, final TableName tableName) + throws IOException { if (bypass) { env.bypass(); } - preDeleteTableHandlerCalled = true; + preDeleteTableActionCalled = true; } @Override - public void postDeleteTableHandler( - ObserverContext ctx, TableName tableName) + public void postCompletedDeleteTableAction( + final ObserverContext ctx, final TableName tableName) throws IOException { - postDeleteTableHandlerCalled = true; + postCompletedDeleteTableActionCalled = true; tableDeletionLatch.countDown(); } - public boolean wasDeleteTableHandlerCalled() { - return preDeleteTableHandlerCalled && postDeleteTableHandlerCalled; + public boolean wasDeleteTableActionCalled() { + return preDeleteTableActionCalled && postCompletedDeleteTableActionCalled; } - public boolean wasDeleteTableHandlerCalledOnly() { - return preDeleteTableHandlerCalled && !postDeleteTableHandlerCalled; + public boolean wasDeleteTableActionCalledOnly() { + return preDeleteTableActionCalled && !postCompletedDeleteTableActionCalled; } + @Deprecated @Override public void preTruncateTableHandler( ObserverContext env, TableName tableName) throws IOException { + } + + @Deprecated + @Override + public void postTruncateTableHandler( + ObserverContext ctx, TableName tableName) + throws IOException { + } + + @Override + public void preTruncateTableAction( + final ObserverContext env, final TableName tableName) + throws IOException { if (bypass) { env.bypass(); } - preTruncateTableHandlerCalled = true; + preTruncateTableActionCalled = true; } @Override - public void postTruncateTableHandler( - ObserverContext ctx, TableName tableName) + public void postCompletedTruncateTableAction( + final ObserverContext ctx, final TableName tableName) throws IOException { - postTruncateTableHandlerCalled = true; + postCompletedTruncateTableActionCalled = true; } - public boolean wasTruncateTableHandlerCalled() { - return preTruncateTableHandlerCalled && postTruncateTableHandlerCalled; + public boolean wasTruncateTableActionCalled() { + return preTruncateTableActionCalled && postCompletedTruncateTableActionCalled; } - public boolean wasTruncateTableHandlerCalledOnly() { - return preTruncateTableHandlerCalled && !postTruncateTableHandlerCalled; + public boolean wasTruncateTableActionCalledOnly() { + return preTruncateTableActionCalled && !postCompletedTruncateTableActionCalled; } + @Deprecated @Override public void preModifyTableHandler( ObserverContext env, TableName tableName, HTableDescriptor htd) throws IOException { - if (bypass) { - env.bypass(); - } - preModifyTableHandlerCalled = true; } + @Deprecated @Override public void postModifyTableHandler( ObserverContext env, TableName tableName, HTableDescriptor htd) throws IOException { - postModifyTableHandlerCalled = true; } - public boolean wasModifyTableHandlerCalled() { - return preModifyColumnHandlerCalled && postModifyColumnHandlerCalled; + @Override + public void preModifyTableAction( + final ObserverContext env, + final TableName tableName, + final HTableDescriptor htd) throws IOException { + if (bypass) { + env.bypass(); + } + preModifyTableActionCalled = true; + } + + @Override + public void postCompletedModifyTableAction( + final ObserverContext env, + final TableName tableName, + final HTableDescriptor htd) throws IOException { + postCompletedModifyTableActionCalled = true; + } + + public boolean wasModifyTableActionCalled() { + return preModifyColumnFamilyActionCalled && postCompletedModifyColumnFamilyActionCalled; } - public boolean wasModifyTableHandlerCalledOnly() { - return preModifyColumnHandlerCalled && !postModifyColumnHandlerCalled; + public boolean wasModifyTableActionCalledOnly() { + return preModifyColumnFamilyActionCalled && !postCompletedModifyColumnFamilyActionCalled; } @Deprecated @@ -1075,13 +1135,14 @@ public class TestMasterObserver { } @Override - public void preAddColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void preAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { if (bypass) { ctx.bypass(); } - preAddColumnHandlerCalled = true; + preAddColumnFamilyActionCalled = true; } @Deprecated @@ -1092,18 +1153,19 @@ public class TestMasterObserver { } @Override - public void postAddColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { - postAddColumnHandlerCalled = true; + public void postCompletedAddColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { + postCompletedAddColumnFamilyActionCalled = true; } - public boolean wasAddColumnHandlerCalled() { - return preAddColumnHandlerCalled && postAddColumnHandlerCalled; + public boolean wasAddColumnFamilyActionCalled() { + return preAddColumnFamilyActionCalled && postCompletedAddColumnFamilyActionCalled; } - public boolean preAddColumnHandlerCalledOnly() { - return preAddColumnHandlerCalled && !postAddColumnHandlerCalled; + public boolean preAddColumnFamilyActionCalledOnly() { + return preAddColumnFamilyActionCalled && !postCompletedAddColumnFamilyActionCalled; } @Deprecated @@ -1114,13 +1176,14 @@ public class TestMasterObserver { } @Override - public void preModifyColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - HColumnDescriptor columnFamily) throws IOException { + public void preModifyColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final HColumnDescriptor columnFamily) throws IOException { if (bypass) { ctx.bypass(); } - preModifyColumnHandlerCalled = true; + preModifyColumnFamilyActionCalled = true; } @Deprecated @@ -1131,18 +1194,18 @@ public class TestMasterObserver { } @Override - public void postModifyColumnFamilyHandler( + public void postCompletedModifyColumnFamilyAction( ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException { - postModifyColumnHandlerCalled = true; + postCompletedModifyColumnFamilyActionCalled = true; } - public boolean wasModifyColumnHandlerCalled() { - return preModifyColumnHandlerCalled && postModifyColumnHandlerCalled; + public boolean wasModifyColumnFamilyActionCalled() { + return preModifyColumnFamilyActionCalled && postCompletedModifyColumnFamilyActionCalled; } - public boolean preModifyColumnHandlerCalledOnly() { - return preModifyColumnHandlerCalled && !postModifyColumnHandlerCalled; + public boolean preModifyColumnFamilyActionCalledOnly() { + return preModifyColumnFamilyActionCalled && !postCompletedModifyColumnFamilyActionCalled; } @Deprecated @@ -1153,13 +1216,14 @@ public class TestMasterObserver { } @Override - public void preDeleteColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - byte[] columnFamily) throws IOException { + public void preDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { if (bypass) { ctx.bypass(); } - preDeleteColumnHandlerCalled = true; + preDeleteColumnFamilyActionCalled = true; } @Deprecated @@ -1170,68 +1234,95 @@ public class TestMasterObserver { } @Override - public void postDeleteColumnFamilyHandler( - ObserverContext ctx, TableName tableName, - byte[] columnFamily) throws IOException { - postDeleteColumnHandlerCalled = true; + public void postCompletedDeleteColumnFamilyAction( + final ObserverContext ctx, + final TableName tableName, + final byte[] columnFamily) throws IOException { + postCompletedDeleteColumnFamilyActionCalled = true; } - public boolean wasDeleteColumnHandlerCalled() { - return preDeleteColumnHandlerCalled && postDeleteColumnHandlerCalled; + public boolean wasDeleteColumnFamilyActionCalled() { + return preDeleteColumnFamilyActionCalled && postCompletedDeleteColumnFamilyActionCalled; } - public boolean preDeleteColumnHandlerCalledOnly() { - return preDeleteColumnHandlerCalled && !postDeleteColumnHandlerCalled; + public boolean preDeleteColumnFamilyActionCalledOnly() { + return preDeleteColumnFamilyActionCalled && !postCompletedDeleteColumnFamilyActionCalled; } + @Deprecated @Override public void preEnableTableHandler( ObserverContext env, TableName tableName) throws IOException { - if (bypass) { - env.bypass(); - } - preEnableTableHandlerCalled = true; } + @Deprecated @Override public void postEnableTableHandler( ObserverContext ctx, TableName tableName) throws IOException { - postEnableTableHandlerCalled = true; } - public boolean wasEnableTableHandlerCalled() { - return preEnableTableHandlerCalled && postEnableTableHandlerCalled; + @Override + public void preEnableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + if (bypass) { + ctx.bypass(); + } + preEnableTableActionCalled = true; + } + + @Override + public void postCompletedEnableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + postCompletedEnableTableActionCalled = true; } - public boolean preEnableTableHandlerCalledOnly() { - return preEnableTableHandlerCalled && !postEnableTableHandlerCalled; + public boolean wasEnableTableActionCalled() { + return preEnableTableActionCalled && postCompletedEnableTableActionCalled; + } + + public boolean preEnableTableActionCalledOnly() { + return preEnableTableActionCalled && !postCompletedEnableTableActionCalled; } @Override public void preDisableTableHandler( ObserverContext env, TableName tableName) throws IOException { - if (bypass) { - env.bypass(); - } - preDisableTableHandlerCalled = true; } @Override public void postDisableTableHandler( ObserverContext ctx, TableName tableName) throws IOException { - postDisableTableHandlerCalled = true; } - public boolean wasDisableTableHandlerCalled() { - return preDisableTableHandlerCalled && postDisableTableHandlerCalled; + @Override + public void preDisableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + if (bypass) { + ctx.bypass(); + } + preDisableTableActionCalled = true; + } + + @Override + public void postCompletedDisableTableAction( + final ObserverContext ctx, final TableName tableName) + throws IOException { + postCompletedDisableTableActionCalled = true; + } + + public boolean wasDisableTableActionCalled() { + return preDisableTableActionCalled && postCompletedDisableTableActionCalled; } - public boolean preDisableTableHandlerCalledOnly() { - return preDisableTableHandlerCalled && !postDisableTableHandlerCalled; + public boolean preDisableTableActionCalledOnly() { + return preDisableTableActionCalled && !postCompletedDisableTableActionCalled; } @Override @@ -1445,9 +1536,9 @@ public class TestMasterObserver { assertTrue("Test table should be created", cp.wasCreateTableCalled()); tableCreationLatch.await(); assertTrue("Table pre create handler called.", cp - .wasPreCreateTableHandlerCalled()); + .wasPreCreateTableActionCalled()); assertTrue("Table create handler should be called.", - cp.wasCreateTableHandlerCalled()); + cp.wasCreateTableActionCalled()); RegionLocator regionLocator = connection.getRegionLocator(htd.getTableName()); List regions = regionLocator.getAllRegionLocations(); @@ -1464,7 +1555,7 @@ public class TestMasterObserver { assertTrue("Coprocessor should have been called on table disable", cp.wasDisableTableCalled()); assertTrue("Disable table handler should be called.", - cp.wasDisableTableHandlerCalled()); + cp.wasDisableTableActionCalled()); // enable assertFalse(cp.wasEnableTableCalled()); @@ -1474,7 +1565,7 @@ public class TestMasterObserver { assertTrue("Coprocessor should have been called on table enable", cp.wasEnableTableCalled()); assertTrue("Enable table handler should be called.", - cp.wasEnableTableHandlerCalled()); + cp.wasEnableTableActionCalled()); admin.disableTable(tableName); assertTrue(admin.isTableDisabled(tableName)); @@ -1511,7 +1602,7 @@ public class TestMasterObserver { assertTrue("Coprocessor should have been called on table delete", cp.wasDeleteTableCalled()); assertTrue("Delete table handler should be called.", - cp.wasDeleteTableHandlerCalled()); + cp.wasDeleteTableActionCalled()); // turn off bypass, run the tests again cp.enableBypass(false); @@ -1521,19 +1612,19 @@ public class TestMasterObserver { assertTrue("Test table should be created", cp.wasCreateTableCalled()); tableCreationLatch.await(); assertTrue("Table pre create handler called.", cp - .wasPreCreateTableHandlerCalled()); + .wasPreCreateTableActionCalled()); assertTrue("Table create handler should be called.", - cp.wasCreateTableHandlerCalled()); + cp.wasCreateTableActionCalled()); // disable assertFalse(cp.wasDisableTableCalled()); - assertFalse(cp.wasDisableTableHandlerCalled()); + assertFalse(cp.wasDisableTableActionCalled()); admin.disableTable(tableName); assertTrue(admin.isTableDisabled(tableName)); assertTrue("Coprocessor should have been called on table disable", cp.wasDisableTableCalled()); assertTrue("Disable table handler should be called.", - cp.wasDisableTableHandlerCalled()); + cp.wasDisableTableActionCalled()); // modify table htd.setMaxFileSize(512 * 1024 * 1024); @@ -1545,7 +1636,7 @@ public class TestMasterObserver { assertTrue("New column family should have been added to test table", cp.wasAddColumnCalled()); assertTrue("Add column handler should be called.", - cp.wasAddColumnHandlerCalled()); + cp.wasAddColumnFamilyActionCalled()); // modify a column family HColumnDescriptor hcd = new HColumnDescriptor(TEST_FAMILY2); @@ -1554,17 +1645,17 @@ public class TestMasterObserver { assertTrue("Second column family should be modified", cp.wasModifyColumnCalled()); assertTrue("Modify table handler should be called.", - cp.wasModifyColumnHandlerCalled()); + cp.wasModifyColumnFamilyActionCalled()); // enable assertFalse(cp.wasEnableTableCalled()); - assertFalse(cp.wasEnableTableHandlerCalled()); + assertFalse(cp.wasEnableTableActionCalled()); admin.enableTable(tableName); assertTrue(admin.isTableEnabled(tableName)); assertTrue("Coprocessor should have been called on table enable", cp.wasEnableTableCalled()); assertTrue("Enable table handler should be called.", - cp.wasEnableTableHandlerCalled()); + cp.wasEnableTableActionCalled()); // disable again admin.disableTable(tableName); @@ -1573,7 +1664,7 @@ public class TestMasterObserver { // delete column assertFalse("No column family deleted yet", cp.wasDeleteColumnCalled()); assertFalse("Delete table column handler should not be called.", - cp.wasDeleteColumnHandlerCalled()); + cp.wasDeleteColumnFamilyActionCalled()); admin.deleteColumnFamily(tableName, TEST_FAMILY2); HTableDescriptor tableDesc = admin.getTableDescriptor(tableName); assertNull("'"+Bytes.toString(TEST_FAMILY2)+"' should have been removed", @@ -1581,19 +1672,19 @@ public class TestMasterObserver { assertTrue("Coprocessor should have been called on column delete", cp.wasDeleteColumnCalled()); assertTrue("Delete table column handler should be called.", - cp.wasDeleteColumnHandlerCalled()); + cp.wasDeleteColumnFamilyActionCalled()); // delete table assertFalse("No table deleted yet", cp.wasDeleteTableCalled()); assertFalse("Delete table handler should not be called.", - cp.wasDeleteTableHandlerCalled()); + cp.wasDeleteTableActionCalled()); deleteTable(admin, tableName); assertFalse("Test table should have been deleted", admin.tableExists(tableName)); assertTrue("Coprocessor should have been called on table delete", cp.wasDeleteTableCalled()); assertTrue("Delete table handler should be called.", - cp.wasDeleteTableHandlerCalled()); + cp.wasDeleteTableActionCalled()); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java index 14e4844..573fdcb 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestTableLockManager.java @@ -104,18 +104,19 @@ public class TestTableLockManager { public static class TestLockTimeoutExceptionMasterObserver extends BaseMasterObserver { @Override - public void preDeleteColumnFamilyHandler(ObserverContext ctx, + public void preDeleteColumnFamilyAction(ObserverContext ctx, TableName tableName, byte[] columnFamily) throws IOException { deleteColumn.countDown(); } @Override - public void postDeleteColumnFamilyHandler(ObserverContext ctx, + public void postCompletedDeleteColumnFamilyAction( + ObserverContext ctx, TableName tableName, byte[] columnFamily) throws IOException { Threads.sleep(10000); } @Override - public void preAddColumnFamilyHandler(ObserverContext ctx, + public void preAddColumnFamilyAction(ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException { fail("Add column should have timeouted out for acquiring the table lock"); } @@ -169,14 +170,15 @@ public class TestTableLockManager { public static class TestAlterAndDisableMasterObserver extends BaseMasterObserver { @Override - public void preAddColumnFamilyHandler(ObserverContext ctx, + public void preAddColumnFamilyAction(ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException { LOG.debug("addColumn called"); addColumn.countDown(); } @Override - public void postAddColumnFamilyHandler(ObserverContext ctx, + public void postCompletedAddColumnFamilyAction( + ObserverContext ctx, TableName tableName, HColumnDescriptor columnFamily) throws IOException { Threads.sleep(6000); try { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java index a260d50..a01ccc4 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/handler/TestEnableTableHandler.java @@ -192,17 +192,21 @@ public class TestEnableTableHandler { volatile CountDownLatch tableDeletionLatch = null; @Override - public void postCreateTableHandler(final ObserverContext ctx, - HTableDescriptor desc, HRegionInfo[] regions) throws IOException { - // the AccessController test, some times calls only and directly the postCreateTableHandler() + public void postCompletedCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + // the AccessController test, some times calls only and directly the + // postCompletedCreateTableAction() if (tableCreationLatch != null) { tableCreationLatch.countDown(); } } @Override - public void postDeleteTableHandler(final ObserverContext ctx, - TableName tableName) + public void postCompletedDeleteTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { // the AccessController test, some times calls only and directly the postDeleteTableHandler() if (tableDeletionLatch != null) { diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java index 40732f6..d8b1b2e 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/namespace/TestNamespaceAuditor.java @@ -513,7 +513,7 @@ public class TestNamespaceAuditor { byte[] columnFamily = Bytes.toBytes("info"); HTableDescriptor tableDescOne = new HTableDescriptor(tableOne); tableDescOne.addFamily(new HColumnDescriptor(columnFamily)); - MasterSyncObserver.throwExceptionInPreCreateTableHandler = true; + MasterSyncObserver.throwExceptionInPreCreateTableAction = true; try { try { ADMIN.createTable(tableDescOne); @@ -527,7 +527,7 @@ public class TestNamespaceAuditor { assertEquals("First table creation failed in namespace so number of tables in namespace " + "should be 0.", 0, nstate.getTables().size()); - MasterSyncObserver.throwExceptionInPreCreateTableHandler = false; + MasterSyncObserver.throwExceptionInPreCreateTableAction = false; try { ADMIN.createTable(tableDescOne); } catch (Exception e) { @@ -539,7 +539,7 @@ public class TestNamespaceAuditor { assertEquals("First table was created successfully so table size in namespace should " + "be one now.", 1, nstate.getTables().size()); } finally { - MasterSyncObserver.throwExceptionInPreCreateTableHandler = false; + MasterSyncObserver.throwExceptionInPreCreateTableAction = false; if (ADMIN.tableExists(tableOne)) { ADMIN.disableTable(tableOne); deleteTable(tableOne); @@ -660,7 +660,7 @@ public class TestNamespaceAuditor { public static class MasterSyncObserver extends BaseMasterObserver { volatile CountDownLatch tableDeletionLatch; - static boolean throwExceptionInPreCreateTableHandler; + static boolean throwExceptionInPreCreateTableAction; @Override public void preDeleteTable(ObserverContext ctx, @@ -669,16 +669,16 @@ public class TestNamespaceAuditor { } @Override - public void postDeleteTableHandler( - final ObserverContext ctx, TableName tableName) - throws IOException { + public void postCompletedDeleteTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { tableDeletionLatch.countDown(); } @Override - public void preCreateTableHandler(ObserverContext ctx, + public void preCreateTableAction(ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { - if (throwExceptionInPreCreateTableHandler) { + if (throwExceptionInPreCreateTableAction) { throw new IOException("Throw exception as it is demanded."); } } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java index 22a9748..f0e7ac9 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/security/access/SecureTestUtil.java @@ -628,20 +628,22 @@ public class SecureTestUtil { volatile CountDownLatch tableDeletionLatch = null; @Override - public void postCreateTableHandler( + public void postCompletedCreateTableAction( final ObserverContext ctx, HTableDescriptor desc, HRegionInfo[] regions) throws IOException { - // the AccessController test, some times calls only and directly the postCreateTableHandler() + // the AccessController test, some times calls only and directly the + // postCompletedCreateTableAction() if (tableCreationLatch != null) { tableCreationLatch.countDown(); } } @Override - public void postDeleteTableHandler( - final ObserverContext ctx, TableName tableName) - throws IOException { - // the AccessController test, some times calls only and directly the postDeleteTableHandler() + public void postCompletedDeleteTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + // the AccessController test, some times calls only and directly the + // postCompletedDeleteTableAction() if (tableDeletionLatch != null) { tableDeletionLatch.countDown(); } diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java index 35560f5..1dfb526 100644 --- hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java +++ hbase-server/src/test/java/org/apache/hadoop/hbase/util/BaseTestHBaseFsck.java @@ -608,19 +608,23 @@ public class BaseTestHBaseFsck { volatile CountDownLatch tableDeletionLatch = null; @Override - public void postCreateTableHandler(final ObserverContext ctx, - HTableDescriptor desc, HRegionInfo[] regions) throws IOException { - // the AccessController test, some times calls only and directly the postCreateTableHandler() + public void postCompletedCreateTableAction( + final ObserverContext ctx, + final HTableDescriptor desc, + final HRegionInfo[] regions) throws IOException { + // the AccessController test, some times calls only and directly the + // postCompletedCreateTableAction() if (tableCreationLatch != null) { tableCreationLatch.countDown(); } } @Override - public void postDeleteTableHandler(final ObserverContext ctx, - TableName tableName) - throws IOException { - // the AccessController test, some times calls only and directly the postDeleteTableHandler() + public void postCompletedDeleteTableAction( + final ObserverContext ctx, + final TableName tableName) throws IOException { + // the AccessController test, some times calls only and directly the + // postCompletedDeleteTableAction() if (tableDeletionLatch != null) { tableDeletionLatch.countDown(); } diff --git hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java index 057853f..249831e 100644 --- hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java +++ hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/SparkSQLPushDownFilter.java @@ -24,6 +24,8 @@ import org.apache.commons.logging.LogFactory; import org.apache.hadoop.hbase.Cell; import org.apache.hadoop.hbase.exceptions.DeserializationException; import org.apache.hadoop.hbase.filter.FilterBase; +import org.apache.hadoop.hbase.spark.datasources.BytesEncoder; +import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder; import org.apache.hadoop.hbase.spark.protobuf.generated.FilterProtos; import org.apache.hadoop.hbase.util.ByteStringer; import org.apache.hadoop.hbase.util.Bytes; @@ -56,21 +58,25 @@ public class SparkSQLPushDownFilter extends FilterBase{ static final byte[] rowKeyFamily = new byte[0]; static final byte[] rowKeyQualifier = Bytes.toBytes("key"); + String encoder; + public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, byte[][] valueFromQueryArray, HashMap> - currentCellToColumnIndexMap) { + currentCellToColumnIndexMap, String encoder) { this.dynamicLogicExpression = dynamicLogicExpression; this.valueFromQueryArray = valueFromQueryArray; this.currentCellToColumnIndexMap = currentCellToColumnIndexMap; + this.encoder = encoder; } public SparkSQLPushDownFilter(DynamicLogicExpression dynamicLogicExpression, byte[][] valueFromQueryArray, - MutableList fields) { + MutableList fields, String encoder) { this.dynamicLogicExpression = dynamicLogicExpression; this.valueFromQueryArray = valueFromQueryArray; + this.encoder = encoder; //generate family qualifier to index mapping this.currentCellToColumnIndexMap = @@ -184,9 +190,12 @@ public class SparkSQLPushDownFilter extends FilterBase{ throw new DeserializationException(e); } + String encoder = proto.getEncoder(); + BytesEncoder enc = JavaBytesEncoder.create(encoder); + //Load DynamicLogicExpression DynamicLogicExpression dynamicLogicExpression = - DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression()); + DynamicLogicExpressionBuilder.build(proto.getDynamicLogicExpression(), enc); //Load valuesFromQuery final List valueFromQueryArrayList = proto.getValueFromQueryArrayList(); @@ -225,7 +234,7 @@ public class SparkSQLPushDownFilter extends FilterBase{ } return new SparkSQLPushDownFilter(dynamicLogicExpression, - valueFromQueryArray, currentCellToColumnIndexMap); + valueFromQueryArray, currentCellToColumnIndexMap, encoder); } /** @@ -256,6 +265,8 @@ public class SparkSQLPushDownFilter extends FilterBase{ builder.addCellToColumnMapping(columnMappingBuilder.build()); } } + builder.setEncoder(encoder); + return builder.build().toByteArray(); } diff --git hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/protobuf/generated/FilterProtos.java hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/protobuf/generated/FilterProtos.java index 1968d32..c909e90 100644 --- hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/protobuf/generated/FilterProtos.java +++ hbase-spark/src/main/java/org/apache/hadoop/hbase/spark/protobuf/generated/FilterProtos.java @@ -783,6 +783,21 @@ public final class FilterProtos { */ org.apache.hadoop.hbase.spark.protobuf.generated.FilterProtos.SQLPredicatePushDownCellToColumnMappingOrBuilder getCellToColumnMappingOrBuilder( int index); + + // required string encoder = 4; + /** + * required string encoder = 4; + */ + boolean hasEncoder(); + /** + * required string encoder = 4; + */ + java.lang.String getEncoder(); + /** + * required string encoder = 4; + */ + com.google.protobuf.ByteString + getEncoderBytes(); } /** * Protobuf type {@code hbase.pb.SQLPredicatePushDownFilter} @@ -856,6 +871,11 @@ public final class FilterProtos { cellToColumnMapping_.add(input.readMessage(org.apache.hadoop.hbase.spark.protobuf.generated.FilterProtos.SQLPredicatePushDownCellToColumnMapping.PARSER, extensionRegistry)); break; } + case 34: { + bitField0_ |= 0x00000002; + encoder_ = input.readBytes(); + break; + } } } } catch (com.google.protobuf.InvalidProtocolBufferException e) { @@ -1004,10 +1024,54 @@ public final class FilterProtos { return cellToColumnMapping_.get(index); } + // required string encoder = 4; + public static final int ENCODER_FIELD_NUMBER = 4; + private java.lang.Object encoder_; + /** + * required string encoder = 4; + */ + public boolean hasEncoder() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + /** + * required string encoder = 4; + */ + public java.lang.String getEncoder() { + java.lang.Object ref = encoder_; + if (ref instanceof java.lang.String) { + return (java.lang.String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + java.lang.String s = bs.toStringUtf8(); + if (bs.isValidUtf8()) { + encoder_ = s; + } + return s; + } + } + /** + * required string encoder = 4; + */ + public com.google.protobuf.ByteString + getEncoderBytes() { + java.lang.Object ref = encoder_; + if (ref instanceof java.lang.String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + encoder_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + private void initFields() { dynamicLogicExpression_ = ""; valueFromQueryArray_ = java.util.Collections.emptyList(); cellToColumnMapping_ = java.util.Collections.emptyList(); + encoder_ = ""; } private byte memoizedIsInitialized = -1; public final boolean isInitialized() { @@ -1018,6 +1082,10 @@ public final class FilterProtos { memoizedIsInitialized = 0; return false; } + if (!hasEncoder()) { + memoizedIsInitialized = 0; + return false; + } for (int i = 0; i < getCellToColumnMappingCount(); i++) { if (!getCellToColumnMapping(i).isInitialized()) { memoizedIsInitialized = 0; @@ -1040,6 +1108,9 @@ public final class FilterProtos { for (int i = 0; i < cellToColumnMapping_.size(); i++) { output.writeMessage(3, cellToColumnMapping_.get(i)); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(4, getEncoderBytes()); + } getUnknownFields().writeTo(output); } @@ -1066,6 +1137,10 @@ public final class FilterProtos { size += com.google.protobuf.CodedOutputStream .computeMessageSize(3, cellToColumnMapping_.get(i)); } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(4, getEncoderBytes()); + } size += getUnknownFields().getSerializedSize(); memoizedSerializedSize = size; return size; @@ -1098,6 +1173,11 @@ public final class FilterProtos { .equals(other.getValueFromQueryArrayList()); result = result && getCellToColumnMappingList() .equals(other.getCellToColumnMappingList()); + result = result && (hasEncoder() == other.hasEncoder()); + if (hasEncoder()) { + result = result && getEncoder() + .equals(other.getEncoder()); + } result = result && getUnknownFields().equals(other.getUnknownFields()); return result; @@ -1123,6 +1203,10 @@ public final class FilterProtos { hash = (37 * hash) + CELL_TO_COLUMN_MAPPING_FIELD_NUMBER; hash = (53 * hash) + getCellToColumnMappingList().hashCode(); } + if (hasEncoder()) { + hash = (37 * hash) + ENCODER_FIELD_NUMBER; + hash = (53 * hash) + getEncoder().hashCode(); + } hash = (29 * hash) + getUnknownFields().hashCode(); memoizedHashCode = hash; return hash; @@ -1243,6 +1327,8 @@ public final class FilterProtos { } else { cellToColumnMappingBuilder_.clear(); } + encoder_ = ""; + bitField0_ = (bitField0_ & ~0x00000008); return this; } @@ -1289,6 +1375,10 @@ public final class FilterProtos { } else { result.cellToColumnMapping_ = cellToColumnMappingBuilder_.build(); } + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000002; + } + result.encoder_ = encoder_; result.bitField0_ = to_bitField0_; onBuilt(); return result; @@ -1346,6 +1436,11 @@ public final class FilterProtos { } } } + if (other.hasEncoder()) { + bitField0_ |= 0x00000008; + encoder_ = other.encoder_; + onChanged(); + } this.mergeUnknownFields(other.getUnknownFields()); return this; } @@ -1355,6 +1450,10 @@ public final class FilterProtos { return false; } + if (!hasEncoder()) { + + return false; + } for (int i = 0; i < getCellToColumnMappingCount(); i++) { if (!getCellToColumnMapping(i).isInitialized()) { @@ -1769,6 +1868,80 @@ public final class FilterProtos { return cellToColumnMappingBuilder_; } + // required string encoder = 4; + private java.lang.Object encoder_ = ""; + /** + * required string encoder = 4; + */ + public boolean hasEncoder() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + /** + * required string encoder = 4; + */ + public java.lang.String getEncoder() { + java.lang.Object ref = encoder_; + if (!(ref instanceof java.lang.String)) { + java.lang.String s = ((com.google.protobuf.ByteString) ref) + .toStringUtf8(); + encoder_ = s; + return s; + } else { + return (java.lang.String) ref; + } + } + /** + * required string encoder = 4; + */ + public com.google.protobuf.ByteString + getEncoderBytes() { + java.lang.Object ref = encoder_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8( + (java.lang.String) ref); + encoder_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + /** + * required string encoder = 4; + */ + public Builder setEncoder( + java.lang.String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + encoder_ = value; + onChanged(); + return this; + } + /** + * required string encoder = 4; + */ + public Builder clearEncoder() { + bitField0_ = (bitField0_ & ~0x00000008); + encoder_ = getDefaultInstance().getEncoder(); + onChanged(); + return this; + } + /** + * required string encoder = 4; + */ + public Builder setEncoderBytes( + com.google.protobuf.ByteString value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000008; + encoder_ = value; + onChanged(); + return this; + } + // @@protoc_insertion_point(builder_scope:hbase.pb.SQLPredicatePushDownFilter) } @@ -1802,13 +1975,14 @@ public final class FilterProtos { "\n\014Filter.proto\022\010hbase.pb\"h\n\'SQLPredicate" + "PushDownCellToColumnMapping\022\025\n\rcolumn_fa" + "mily\030\001 \002(\014\022\021\n\tqualifier\030\002 \002(\014\022\023\n\013column_" + - "name\030\003 \002(\t\"\261\001\n\032SQLPredicatePushDownFilte" + + "name\030\003 \002(\t\"\302\001\n\032SQLPredicatePushDownFilte" + "r\022 \n\030dynamic_logic_expression\030\001 \002(\t\022\036\n\026v" + "alue_from_query_array\030\002 \003(\014\022Q\n\026cell_to_c" + "olumn_mapping\030\003 \003(\01321.hbase.pb.SQLPredic" + - "atePushDownCellToColumnMappingBH\n0org.ap" + - "ache.hadoop.hbase.spark.protobuf.generat" + - "edB\014FilterProtosH\001\210\001\001\240\001\001" + "atePushDownCellToColumnMapping\022\017\n\007encode" + + "r\030\004 \002(\tBH\n0org.apache.hadoop.hbase.spark" + + ".protobuf.generatedB\014FilterProtosH\001\210\001\001\240\001", + "\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -1826,7 +2000,7 @@ public final class FilterProtos { internal_static_hbase_pb_SQLPredicatePushDownFilter_fieldAccessorTable = new com.google.protobuf.GeneratedMessage.FieldAccessorTable( internal_static_hbase_pb_SQLPredicatePushDownFilter_descriptor, - new java.lang.String[] { "DynamicLogicExpression", "ValueFromQueryArray", "CellToColumnMapping", }); + new java.lang.String[] { "DynamicLogicExpression", "ValueFromQueryArray", "CellToColumnMapping", "Encoder", }); return null; } }; diff --git hbase-spark/src/main/protobuf/Filter.proto hbase-spark/src/main/protobuf/Filter.proto index e076ce8..47b95ff 100644 --- hbase-spark/src/main/protobuf/Filter.proto +++ hbase-spark/src/main/protobuf/Filter.proto @@ -35,4 +35,5 @@ message SQLPredicatePushDownFilter { required string dynamic_logic_expression = 1; repeated bytes value_from_query_array = 2; repeated SQLPredicatePushDownCellToColumnMapping cell_to_column_mapping = 3; + required string encoder = 4; } \ No newline at end of file diff --git hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala index 1697036..8b15cc7 100644 --- hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala +++ hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DefaultSource.scala @@ -23,9 +23,7 @@ import java.util.concurrent.ConcurrentLinkedQueue import org.apache.hadoop.hbase.client._ import org.apache.hadoop.hbase.io.ImmutableBytesWritable import org.apache.hadoop.hbase.mapred.TableOutputFormat -import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf -import org.apache.hadoop.hbase.spark.datasources.HBaseTableScanRDD -import org.apache.hadoop.hbase.spark.datasources.SerializableConfiguration +import org.apache.hadoop.hbase.spark.datasources._ import org.apache.hadoop.hbase.types._ import org.apache.hadoop.hbase.util.{Bytes, PositionedByteRange, SimplePositionedMutableByteRange} import org.apache.hadoop.hbase._ @@ -92,6 +90,9 @@ case class HBaseRelation ( val minTimestamp = parameters.get(HBaseSparkConf.MIN_TIMESTAMP).map(_.toLong) val maxTimestamp = parameters.get(HBaseSparkConf.MAX_TIMESTAMP).map(_.toLong) val maxVersions = parameters.get(HBaseSparkConf.MAX_VERSIONS).map(_.toInt) + val encoderClsName = parameters.get(HBaseSparkConf.ENCODER).getOrElse(HBaseSparkConf.defaultEncoder) + + @transient val encoder = JavaBytesEncoder.create(encoderClsName) val catalog = HBaseTableCatalog(parameters) def tableName = catalog.name @@ -335,7 +336,7 @@ case class HBaseRelation ( val pushDownFilterJava = if (usePushDownColumnFilter && pushDownDynamicLogicExpression != null) { Some(new SparkSQLPushDownFilter(pushDownDynamicLogicExpression, - valueArray, requiredQualifierDefinitionList)) + valueArray, requiredQualifierDefinitionList, encoderClsName)) } else { None } @@ -402,11 +403,17 @@ case class HBaseRelation ( (superRowKeyFilter, superDynamicLogicExpression, queryValueArray) } + /** + * Because we don't assume any specific encoder/decoder, and order is inconsistent between java primitive + * type and its byte array, we have to split the predicates on some of the java primitive type into multiple + * predicates, and union these predicates together to make the predicates be performed correctly. + * For example, if we have "COLUMN < 2", we will transform it into + * "0 <= COLUMN < 2 OR Integer.MIN_VALUE <= COLUMN <= -1" + */ def transverseFilterTree(parentRowKeyFilter:RowKeyFilter, valueArray:mutable.MutableList[Array[Byte]], filter:Filter): DynamicLogicExpression = { filter match { - case EqualTo(attr, value) => val field = catalog.getField(attr) if (field != null) { @@ -420,18 +427,38 @@ case class HBaseRelation ( valueArray += byteValue } new EqualLogicExpression(attr, valueArray.length - 1, false) + + /** + * BoundRange will split the predicates into multiple byte array boundaries. + * Each boundaries is mapped into the RowKeyFilter and then is unioned by the reduce + * operation. If the data type is not supported by BoundRange, b will be None, and there is + * no operation happens on the parentRowKeyFilter. + * + * Note that because LessThan is not inclusive, thus the first bound should be exclusive, + * which is controlled by inc. + * + * The other predicates, i.e., GreaterThan/LessThanOrEqual/GreaterThanOrEqual follows + * the similar logic. + */ case LessThan(attr, value) => val field = catalog.getField(attr) if (field != null) { if (field.isRowKey) { - parentRowKeyFilter.mergeIntersect(new RowKeyFilter(null, - new ScanRange(DefaultSourceStaticUtils.getByteValue(field, - value.toString), false, - new Array[Byte](0), true))) + val b = BoundRange(value) + var inc = false + b.map(_.less.map { x => + val r = new RowKeyFilter(null, + new ScanRange(x.upper, inc, x.low, true) + ) + inc = true + r + }).map { x => + x.reduce { (i, j) => + i.mergeUnion(j) + } + }.map(parentRowKeyFilter.mergeIntersect(_)) } - val byteValue = - DefaultSourceStaticUtils.getByteValue(catalog.getField(attr), - value.toString) + val byteValue = encoder.encode(field.dt, value) valueArray += byteValue } new LessThanLogicExpression(attr, valueArray.length - 1) @@ -439,13 +466,20 @@ case class HBaseRelation ( val field = catalog.getField(attr) if (field != null) { if (field.isRowKey) { - parentRowKeyFilter.mergeIntersect(new RowKeyFilter(null, - new ScanRange(null, true, DefaultSourceStaticUtils.getByteValue(field, - value.toString), false))) + val b = BoundRange(value) + var inc = false + b.map(_.greater.map{x => + val r = new RowKeyFilter(null, + new ScanRange(x.upper, true, x.low, inc)) + inc = true + r + }).map { x => + x.reduce { (i, j) => + i.mergeUnion(j) + } + }.map(parentRowKeyFilter.mergeIntersect(_)) } - val byteValue = - DefaultSourceStaticUtils.getByteValue(field, - value.toString) + val byteValue = encoder.encode(field.dt, value) valueArray += byteValue } new GreaterThanLogicExpression(attr, valueArray.length - 1) @@ -453,14 +487,17 @@ case class HBaseRelation ( val field = catalog.getField(attr) if (field != null) { if (field.isRowKey) { - parentRowKeyFilter.mergeIntersect(new RowKeyFilter(null, - new ScanRange(DefaultSourceStaticUtils.getByteValue(field, - value.toString), true, - new Array[Byte](0), true))) + val b = BoundRange(value) + b.map(_.less.map(x => + new RowKeyFilter(null, + new ScanRange(x.upper, true, x.low, true)))) + .map { x => + x.reduce{ (i, j) => + i.mergeUnion(j) + } + }.map(parentRowKeyFilter.mergeIntersect(_)) } - val byteValue = - DefaultSourceStaticUtils.getByteValue(catalog.getField(attr), - value.toString) + val byteValue = encoder.encode(field.dt, value) valueArray += byteValue } new LessThanOrEqualLogicExpression(attr, valueArray.length - 1) @@ -468,15 +505,18 @@ case class HBaseRelation ( val field = catalog.getField(attr) if (field != null) { if (field.isRowKey) { - parentRowKeyFilter.mergeIntersect(new RowKeyFilter(null, - new ScanRange(null, true, DefaultSourceStaticUtils.getByteValue(field, - value.toString), true))) + val b = BoundRange(value) + b.map(_.greater.map(x => + new RowKeyFilter(null, + new ScanRange(x.upper, true, x.low, true)))) + .map { x => + x.reduce { (i, j) => + i.mergeUnion(j) + } + }.map(parentRowKeyFilter.mergeIntersect(_)) } - val byteValue = - DefaultSourceStaticUtils.getByteValue(catalog.getField(attr), - value.toString) + val byteValue = encoder.encode(field.dt, value) valueArray += byteValue - } new GreaterThanOrEqualLogicExpression(attr, valueArray.length - 1) case Or(left, right) => @@ -587,7 +627,7 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, var leftRange:ScanRange = null var rightRange:ScanRange = null - //First identify the Left range + // First identify the Left range // Also lower bound can't be null if (compareRange(lowerBound, other.lowerBound) < 0 || compareRange(upperBound, other.upperBound) < 0) { @@ -598,17 +638,34 @@ class ScanRange(var upperBound:Array[Byte], var isUpperBoundEqualTo:Boolean, rightRange = this } - //Then see if leftRange goes to null or if leftRange.upperBound - // upper is greater or equals to rightRange.lowerBound - if (leftRange.upperBound == null || - Bytes.compareTo(leftRange.upperBound, rightRange.lowerBound) >= 0) { - new ScanRange(leftRange.upperBound, leftRange.isUpperBoundEqualTo, rightRange.lowerBound, rightRange.isLowerBoundEqualTo) + if (hasOverlap(leftRange, rightRange)) { + // Find the upper bound and lower bound + if (compareRange(leftRange.upperBound, rightRange.upperBound) >= 0) { + new ScanRange(rightRange.upperBound, rightRange.isUpperBoundEqualTo, + rightRange.lowerBound, rightRange.isLowerBoundEqualTo) + } else { + new ScanRange(leftRange.upperBound, leftRange.isUpperBoundEqualTo, + rightRange.lowerBound, rightRange.isLowerBoundEqualTo) + } } else { null } } /** + * The leftRange.upperBound has to be larger than the rightRange's lowBound. + * Otherwise, there is no overlap. + * + * @param left: The range with the smaller lowBound + * @param right: The range with the larger lowBound + * @return Whether two ranges has overlap. + */ + + def hasOverlap(left: ScanRange, right: ScanRange): Boolean = { + compareRange(left.upperBound, right.lowerBound) >= 0 + } + + /** * Special compare logic because we can have null values * for left or right bound * @@ -1046,7 +1103,7 @@ class RowKeyFilter (currentPoint:Array[Byte] = null, * * @param other Filter to merge */ - def mergeUnion(other:RowKeyFilter): Unit = { + def mergeUnion(other:RowKeyFilter): RowKeyFilter = { other.points.foreach( p => points += p) other.ranges.foreach( otherR => { @@ -1058,6 +1115,7 @@ class RowKeyFilter (currentPoint:Array[Byte] = null, }} if (!doesOverLap) ranges.+=(otherR) }) + this } /** @@ -1066,7 +1124,7 @@ class RowKeyFilter (currentPoint:Array[Byte] = null, * * @param other Filter to merge */ - def mergeIntersect(other:RowKeyFilter): Unit = { + def mergeIntersect(other:RowKeyFilter): RowKeyFilter = { val survivingPoints = new mutable.MutableList[Array[Byte]]() val didntSurviveFirstPassPoints = new mutable.MutableList[Array[Byte]]() if (points == null || points.length == 0) { @@ -1112,6 +1170,7 @@ class RowKeyFilter (currentPoint:Array[Byte] = null, } points = survivingPoints ranges = survivingRanges + this } override def toString:String = { diff --git hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala index fa61860..1a1d478 100644 --- hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala +++ hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpression.scala @@ -19,8 +19,11 @@ package org.apache.hadoop.hbase.spark import java.util +import org.apache.hadoop.hbase.spark.datasources.{BytesEncoder, JavaBytesEncoder} +import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder import org.apache.hadoop.hbase.util.Bytes - +import org.apache.spark.sql.datasources.hbase.{Field, Utils} +import org.apache.spark.sql.types._ /** * Dynamic logic for SQL push down logic there is an instance for most * common operations and a pass through for other operations not covered here @@ -38,7 +41,31 @@ trait DynamicLogicExpression { appendToExpression(strBuilder) strBuilder.toString() } + def filterOps: JavaBytesEncoder = JavaBytesEncoder.Unknown + def appendToExpression(strBuilder:StringBuilder) + + var encoder: BytesEncoder = _ + + def setEncoder(enc: BytesEncoder): DynamicLogicExpression = { + encoder = enc + this + } +} + +trait CompareTrait { + self: DynamicLogicExpression => + def columnName: String + def valueFromQueryIndex: Int + def execute(columnToCurrentRowValueMap: + util.HashMap[String, ByteArrayComparable], + valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { + val currentRowValue = columnToCurrentRowValueMap.get(columnName) + val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) + currentRowValue != null && + encoder.filter(currentRowValue.bytes, currentRowValue.offset, currentRowValue.length, + valueFromQuery, 0, valueFromQuery.length, filterOps) + } } class AndLogicExpression (val leftExpression:DynamicLogicExpression, @@ -113,59 +140,28 @@ class IsNullLogicExpression (val columnName:String, } } -class GreaterThanLogicExpression (val columnName:String, - val valueFromQueryIndex:Int) - extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { - val currentRowValue = columnToCurrentRowValueMap.get(columnName) - val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) - - currentRowValue != null && - Bytes.compareTo(currentRowValue.bytes, - currentRowValue.offset, currentRowValue.length, valueFromQuery, - 0, valueFromQuery.length) > 0 - } +class GreaterThanLogicExpression (override val columnName:String, + override val valueFromQueryIndex:Int) + extends DynamicLogicExpression with CompareTrait{ + override val filterOps = JavaBytesEncoder.Greater override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " > " + valueFromQueryIndex) } } -class GreaterThanOrEqualLogicExpression (val columnName:String, - val valueFromQueryIndex:Int) - extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { - val currentRowValue = columnToCurrentRowValueMap.get(columnName) - val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) - - currentRowValue != null && - Bytes.compareTo(currentRowValue.bytes, - currentRowValue.offset, currentRowValue.length, valueFromQuery, - 0, valueFromQuery.length) >= 0 - } +class GreaterThanOrEqualLogicExpression (override val columnName:String, + override val valueFromQueryIndex:Int) + extends DynamicLogicExpression with CompareTrait{ + override val filterOps = JavaBytesEncoder.GreaterEqual override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " >= " + valueFromQueryIndex) } } -class LessThanLogicExpression (val columnName:String, - val valueFromQueryIndex:Int) - extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { - val currentRowValue = columnToCurrentRowValueMap.get(columnName) - val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) - - currentRowValue != null && - Bytes.compareTo(currentRowValue.bytes, - currentRowValue.offset, currentRowValue.length, valueFromQuery, - 0, valueFromQuery.length) < 0 - } - +class LessThanLogicExpression (override val columnName:String, + override val valueFromQueryIndex:Int) + extends DynamicLogicExpression with CompareTrait { + override val filterOps = JavaBytesEncoder.Less override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " < " + valueFromQueryIndex) } @@ -173,19 +169,8 @@ class LessThanLogicExpression (val columnName:String, class LessThanOrEqualLogicExpression (val columnName:String, val valueFromQueryIndex:Int) - extends DynamicLogicExpression{ - override def execute(columnToCurrentRowValueMap: - util.HashMap[String, ByteArrayComparable], - valueFromQueryValueArray:Array[Array[Byte]]): Boolean = { - val currentRowValue = columnToCurrentRowValueMap.get(columnName) - val valueFromQuery = valueFromQueryValueArray(valueFromQueryIndex) - - currentRowValue != null && - Bytes.compareTo(currentRowValue.bytes, - currentRowValue.offset, currentRowValue.length, valueFromQuery, - 0, valueFromQuery.length) <= 0 - } - + extends DynamicLogicExpression with CompareTrait{ + override val filterOps = JavaBytesEncoder.LessEqual override def appendToExpression(strBuilder: StringBuilder): Unit = { strBuilder.append(columnName + " <= " + valueFromQueryIndex) } @@ -197,58 +182,66 @@ class PassThroughLogicExpression() extends DynamicLogicExpression { valueFromQueryValueArray: Array[Array[Byte]]): Boolean = true override def appendToExpression(strBuilder: StringBuilder): Unit = { - strBuilder.append("Pass") + // Fix the offset bug by add dummy to avoid crash the region server. + // because in the DynamicLogicExpressionBuilder.build function, the command is always retrieved from offset + 1 as below + // val command = expressionArray(offSet + 1) + // we have to padding it so that `Pass` is on the right offset. + strBuilder.append("dummy Pass -1") } } object DynamicLogicExpressionBuilder { - def build(expressionString:String): DynamicLogicExpression = { + def build(expressionString: String, encoder: BytesEncoder): DynamicLogicExpression = { - val expressionAndOffset = build(expressionString.split(' '), 0) + val expressionAndOffset = build(expressionString.split(' '), 0, encoder) expressionAndOffset._1 } private def build(expressionArray:Array[String], - offSet:Int): (DynamicLogicExpression, Int) = { - if (expressionArray(offSet).equals("(")) { - val left = build(expressionArray, offSet + 1) - val right = build(expressionArray, left._2 + 1) - if (expressionArray(left._2).equals("AND")) { - (new AndLogicExpression(left._1, right._1), right._2 + 1) - } else if (expressionArray(left._2).equals("OR")) { - (new OrLogicExpression(left._1, right._1), right._2 + 1) - } else { - throw new Throwable("Unknown gate:" + expressionArray(left._2)) - } - } else { - val command = expressionArray(offSet + 1) - if (command.equals("<")) { - (new LessThanLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) - } else if (command.equals("<=")) { - (new LessThanOrEqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) - } else if (command.equals(">")) { - (new GreaterThanLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) - } else if (command.equals(">=")) { - (new GreaterThanOrEqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt), offSet + 3) - } else if (command.equals("==")) { - (new EqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt, false), offSet + 3) - } else if (command.equals("!=")) { - (new EqualLogicExpression(expressionArray(offSet), - expressionArray(offSet + 2).toInt, true), offSet + 3) - } else if (command.equals("isNull")) { - (new IsNullLogicExpression(expressionArray(offSet), false), offSet + 2) - } else if (command.equals("isNotNull")) { - (new IsNullLogicExpression(expressionArray(offSet), true), offSet + 2) - } else if (command.equals("Pass")) { - (new PassThroughLogicExpression, offSet + 2) + offSet:Int, encoder: BytesEncoder): (DynamicLogicExpression, Int) = { + val expr = { + if (expressionArray(offSet).equals("(")) { + val left = build(expressionArray, offSet + 1, encoder) + val right = build(expressionArray, left._2 + 1, encoder) + if (expressionArray(left._2).equals("AND")) { + (new AndLogicExpression(left._1, right._1), right._2 + 1) + } else if (expressionArray(left._2).equals("OR")) { + (new OrLogicExpression(left._1, right._1), right._2 + 1) + } else { + throw new Throwable("Unknown gate:" + expressionArray(left._2)) + } } else { - throw new Throwable("Unknown logic command:" + command) + val command = expressionArray(offSet + 1) + if (command.equals("<")) { + (new LessThanLogicExpression(expressionArray(offSet), + expressionArray(offSet + 2).toInt), offSet + 3) + } else if (command.equals("<=")) { + (new LessThanOrEqualLogicExpression(expressionArray(offSet), + expressionArray(offSet + 2).toInt), offSet + 3) + } else if (command.equals(">")) { + (new GreaterThanLogicExpression(expressionArray(offSet), + expressionArray(offSet + 2).toInt), offSet + 3) + } else if (command.equals(">=")) { + (new GreaterThanOrEqualLogicExpression(expressionArray(offSet), + expressionArray(offSet + 2).toInt), offSet + 3) + } else if (command.equals("==")) { + (new EqualLogicExpression(expressionArray(offSet), + expressionArray(offSet + 2).toInt, false), offSet + 3) + } else if (command.equals("!=")) { + (new EqualLogicExpression(expressionArray(offSet), + expressionArray(offSet + 2).toInt, true), offSet + 3) + } else if (command.equals("isNull")) { + (new IsNullLogicExpression(expressionArray(offSet), false), offSet + 2) + } else if (command.equals("isNotNull")) { + (new IsNullLogicExpression(expressionArray(offSet), true), offSet + 2) + } else if (command.equals("Pass")) { + (new PassThroughLogicExpression, offSet + 3) + } else { + throw new Throwable("Unknown logic command:" + command) + } } } + expr._1.setEncoder(encoder) + expr } } \ No newline at end of file diff --git hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/BoundRange.scala hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/BoundRange.scala new file mode 100644 index 0000000..b6b8e03 --- /dev/null +++ hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/BoundRange.scala @@ -0,0 +1,147 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.spark.datasources + +import org.apache.hadoop.hbase.util.Bytes +import org.apache.hadoop.hbase.spark.hbase._ +import org.apache.spark.Logging +import org.apache.spark.unsafe.types.UTF8String + +/** + * The ranges for the data type whose size is known. Whether the bound is inclusive + * or exclusive is undefind, and upper to the caller to decide. + * + * @param low: the lower bound of the range. + * @param upper: the upper bound of the range. + */ +case class BoundRange(low: Array[Byte],upper: Array[Byte]) + +/** + * The class identifies the ranges for a java primitive type. The caller needs + * to decide the bound is either inclusive or exclusive on its own. + * information + * + * @param less: the set of ranges for LessThan/LessOrEqualThan + * @param greater: the set of ranges for GreaterThan/GreaterThanOrEqualTo + * @param value: the byte array of the original value + */ +case class BoundRanges(less: Array[BoundRange], greater: Array[BoundRange], value: Array[Byte]) + +/** + * Evaluate the java primitive type and return the BoundRanges. For one value, it may have + * multiple output ranges because of the inconsistency of order between java primitive type + * and its byte array order. + * + * For short, integer, and long, the order of number is consistent with byte array order + * if two number has the same sign bit. But the negative number is larger than positive + * number in byte array. + * + * For double and float, the order of positive number is consistent with its byte array order. + * But the order of negative number is the reverse order of byte array. Please refer to IEEE-754 + * and https://en.wikipedia.org/wiki/Single-precision_floating-point_format + */ +object BoundRange extends Logging{ + def apply(in: Any): Option[BoundRanges] = in match { + case a: Integer => + val b = Bytes.toBytes(a) + if (a >= 0) { + logDebug(s"range is 0 to $a and ${Integer.MIN_VALUE} to -1") + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(0: Int), b), + BoundRange(Bytes.toBytes(Integer.MIN_VALUE), Bytes.toBytes(-1: Int))), + Array(BoundRange(b, Bytes.toBytes(Integer.MAX_VALUE))), b)) + } else { + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(Integer.MIN_VALUE), b)), + Array(BoundRange(b, Bytes.toBytes(-1: Integer)), + BoundRange(Bytes.toBytes(0: Int), Bytes.toBytes(Integer.MAX_VALUE))), b)) + } + case a: Long => + val b = Bytes.toBytes(a) + if (a >= 0) { + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(0: Long), b), + BoundRange(Bytes.toBytes(Long.MinValue), Bytes.toBytes(-1: Long))), + Array(BoundRange(b, Bytes.toBytes(Long.MaxValue))), b)) + } else { + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(Long.MinValue), b)), + Array(BoundRange(b, Bytes.toBytes(-1: Long)), + BoundRange(Bytes.toBytes(0: Long), Bytes.toBytes(Long.MaxValue))), b)) + } + case a: Short => + val b = Bytes.toBytes(a) + if (a >= 0) { + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(0: Short), b), + BoundRange(Bytes.toBytes(Short.MinValue), Bytes.toBytes(-1: Short))), + Array(BoundRange(b, Bytes.toBytes(Short.MaxValue))), b)) + } else { + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(Short.MinValue), b)), + Array(BoundRange(b, Bytes.toBytes(-1: Short)), + BoundRange(Bytes.toBytes(0: Short), Bytes.toBytes(Short.MaxValue))), b)) + } + case a: Double => + val b = Bytes.toBytes(a) + if (a >= 0.0f) { + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(0.0d), b), + BoundRange(Bytes.toBytes(-0.0d), Bytes.toBytes(Double.MinValue))), + Array(BoundRange(b, Bytes.toBytes(Double.MaxValue))), b)) + } else { + Some(BoundRanges( + Array(BoundRange(b, Bytes.toBytes(Double.MinValue))), + Array(BoundRange(Bytes.toBytes(-0.0d), b), + BoundRange(Bytes.toBytes(0.0d), Bytes.toBytes(Double.MaxValue))), b)) + } + case a: Float => + val b = Bytes.toBytes(a) + if (a >= 0.0f) { + Some(BoundRanges( + Array(BoundRange(Bytes.toBytes(0.0f), b), + BoundRange(Bytes.toBytes(-0.0f), Bytes.toBytes(Float.MinValue))), + Array(BoundRange(b, Bytes.toBytes(Float.MaxValue))), b)) + } else { + Some(BoundRanges( + Array(BoundRange(b, Bytes.toBytes(Float.MinValue))), + Array(BoundRange(Bytes.toBytes(-0.0f), b), + BoundRange(Bytes.toBytes(0.0f), Bytes.toBytes(Float.MaxValue))), b)) + } + case a: Array[Byte] => + Some(BoundRanges( + Array(BoundRange(bytesMin, a)), + Array(BoundRange(a, bytesMax)), a)) + case a: Byte => + val b = Array(a) + Some(BoundRanges( + Array(BoundRange(bytesMin, b)), + Array(BoundRange(b, bytesMax)), b)) + case a: String => + val b = Bytes.toBytes(a) + Some(BoundRanges( + Array(BoundRange(bytesMin, b)), + Array(BoundRange(b, bytesMax)), b)) + case a: UTF8String => + val b = a.getBytes + Some(BoundRanges( + Array(BoundRange(bytesMin, b)), + Array(BoundRange(b, bytesMax)), b)) + case _ => None + } +} diff --git hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala index be2af30..d9ee494 100644 --- hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala +++ hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/HBaseSparkConf.scala @@ -41,4 +41,6 @@ object HBaseSparkConf{ val MIN_TIMESTAMP = "hbase.spark.query.minTimestamp" val MAX_TIMESTAMP = "hbase.spark.query.maxTimestamp" val MAX_VERSIONS = "hbase.spark.query.maxVersions" + val ENCODER = "hbase.spark.query.encoder" + val defaultEncoder = classOf[NaiveEncoder].getCanonicalName } diff --git hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/JavaBytesEncoder.scala hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/JavaBytesEncoder.scala new file mode 100644 index 0000000..90b2d91 --- /dev/null +++ hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/JavaBytesEncoder.scala @@ -0,0 +1,199 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.spark.datasources + +import org.apache.hadoop.hbase.spark.datasources.JavaBytesEncoder.JavaBytesEncoder +import org.apache.hadoop.hbase.util.Bytes +import org.apache.spark.sql.types._ + +/** + * The trait to support plugin architecture for different encoder/decoder. + * encoder is used for serialize the data type to byte arrway and the filter is + * used to filter out the unnecessary records. + */ +trait BytesEncoder { + def encode(dt: DataType, value: Any): Array[Byte] + + /** + * The function performing real filtering operations. The format of filterBytes depends on the + * implementation of the encoder/decoder. + * + * @param input: the current input byte array that needs to be filtered out + * @param offset1: the starting offset of the input byte array. + * @param length1: the length of the input byte array. + * @param filterBytes: the byte array provided by query condition. + * @param offset2: the starting offset in the filterBytes. + * @param length2: the length of the bytes in the filterBytes + * @param ops: The operation of the filter operator. + * @return true: the record satisfies the predicates + * false: the record does not satisfies the predicates. + */ + def filter(input: Array[Byte], offset1: Int, length1: Int, + filterBytes: Array[Byte], offset2: Int, length2: Int, + ops: JavaBytesEncoder): Boolean +} + +/** + * This is the naive non-order preserving encoder/decoder. + * Due to the inconsistency of the order between java primitive types + * and their bytearray. The data type has to be passed in so that the filter + * can work correctly, which is done by wrapping the type into the first byte + * of the serialized array. + */ +class NaiveEncoder extends BytesEncoder { + var code = 0 + def nextCode: Byte = { + code += 1 + (code - 1).asInstanceOf[Byte] + } + val BooleanEnc = nextCode + val ShortEnc = nextCode + val IntEnc = nextCode + val LongEnc = nextCode + val FloatEnc = nextCode + val DoubleEnc = nextCode + val StringEnc = nextCode + val BinaryEnc = nextCode + val TimestampEnc = nextCode + val UnknownEnc = nextCode + + def compare(c: Int, ops: JavaBytesEncoder): Boolean = { + ops match { + case JavaBytesEncoder.Greater => c > 0 + case JavaBytesEncoder.GreaterEqual => c >= 0 + case JavaBytesEncoder.Less => c < 0 + case JavaBytesEncoder.LessEqual => c <= 0 + } + } + + /** + * encode the data type into byte array. Note that it is a naive implementation with the + * data type byte appending to the head of the serialized byte array. + * + * @param dt: The data type of the input + * @param value: the value of the input + * @return the byte array with the first byte indicating the data type. + */ + override def encode(dt: DataType, + value: Any): Array[Byte] = { + dt match { + case BooleanType => + val result = new Array[Byte](Bytes.SIZEOF_BOOLEAN + 1) + result(0) = BooleanEnc + value.asInstanceOf[Boolean] match { + case true => result(1) = -1: Byte + case false => result(1) = 0: Byte + } + result + case ShortType => + val result = new Array[Byte](Bytes.SIZEOF_SHORT + 1) + result(0) = ShortEnc + Bytes.putShort(result, 1, value.asInstanceOf[Short]) + result + case IntegerType => + val result = new Array[Byte](Bytes.SIZEOF_INT + 1) + result(0) = IntEnc + Bytes.putInt(result, 1, value.asInstanceOf[Int]) + result + case LongType|TimestampType => + val result = new Array[Byte](Bytes.SIZEOF_LONG + 1) + result(0) = LongEnc + Bytes.putLong(result, 1, value.asInstanceOf[Long]) + result + case FloatType => + val result = new Array[Byte](Bytes.SIZEOF_FLOAT + 1) + result(0) = FloatEnc + Bytes.putFloat(result, 1, value.asInstanceOf[Float]) + result + case DoubleType => + val result = new Array[Byte](Bytes.SIZEOF_DOUBLE + 1) + result(0) = DoubleEnc + Bytes.putDouble(result, 1, value.asInstanceOf[Double]) + result + case BinaryType => + val v = value.asInstanceOf[Array[Bytes]] + val result = new Array[Byte](v.length + 1) + result(0) = BinaryEnc + System.arraycopy(v, 0, result, 1, v.length) + result + case StringType => + val bytes = Bytes.toBytes(value.asInstanceOf[String]) + val result = new Array[Byte](bytes.length + 1) + result(0) = StringEnc + System.arraycopy(bytes, 0, result, 1, bytes.length) + result + case _ => + val bytes = Bytes.toBytes(value.toString) + val result = new Array[Byte](bytes.length + 1) + result(0) = UnknownEnc + System.arraycopy(bytes, 0, result, 1, bytes.length) + result + } + } + + override def filter(input: Array[Byte], offset1: Int, length1: Int, + filterBytes: Array[Byte], offset2: Int, length2: Int, + ops: JavaBytesEncoder): Boolean = { + filterBytes(offset2) match { + case ShortEnc => + val in = Bytes.toShort(input, offset1) + val value = Bytes.toShort(filterBytes, offset2 + 1) + compare(in.compareTo(value), ops) + case IntEnc => + val in = Bytes.toInt(input, offset1) + val value = Bytes.toInt(filterBytes, offset2 + 1) + compare(in.compareTo(value), ops) + case LongEnc | TimestampEnc => + val in = Bytes.toInt(input, offset1) + val value = Bytes.toInt(filterBytes, offset2 + 1) + compare(in.compareTo(value), ops) + case FloatEnc => + val in = Bytes.toFloat(input, offset1) + val value = Bytes.toFloat(filterBytes, offset2 + 1) + compare(in.compareTo(value), ops) + case DoubleEnc => + val in = Bytes.toDouble(input, offset1) + val value = Bytes.toDouble(filterBytes, offset2 + 1) + compare(in.compareTo(value), ops) + case _ => + // for String, Byte, Binary, Boolean and other types + // we can use the order of byte array directly. + compare( + Bytes.compareTo(input, offset1, length1, filterBytes, offset2 + 1, length2 - 1), ops) + } + } +} + +object JavaBytesEncoder extends Enumeration { + type JavaBytesEncoder = Value + val Greater, GreaterEqual, Less, LessEqual, Equal, Unknown = Value + + /** + * create the encoder/decoder + * + * @param clsName: the class name of the encoder/decoder class + * @return the instance of the encoder plugin. + */ + def create(clsName: String): BytesEncoder = { + try { + Class.forName(clsName).newInstance.asInstanceOf[BytesEncoder] + } catch { + case _: Throwable => new NaiveEncoder() + } + } +} \ No newline at end of file diff --git hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala index 4ff0413..ce7b55a 100644 --- hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala +++ hbase-spark/src/main/scala/org/apache/hadoop/hbase/spark/datasources/package.scala @@ -23,6 +23,8 @@ import scala.math.Ordering package object hbase { type HBaseType = Array[Byte] + def bytesMin = new Array[Byte](0) + def bytesMax = null val ByteMax = -1.asInstanceOf[Byte] val ByteMin = 0.asInstanceOf[Byte] val ord: Ordering[HBaseType] = new Ordering[HBaseType] { diff --git hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala index 831c7de..c2d611f 100644 --- hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala +++ hbase-spark/src/main/scala/org/apache/spark/sql/datasources/hbase/HBaseTableCatalog.scala @@ -156,7 +156,7 @@ case class HBaseTableCatalog( def get(key: String) = params.get(key) // Setup the start and length for each dimension of row key at runtime. - def dynSetupRowKey(rowKey: HBaseType) { + def dynSetupRowKey(rowKey: Array[Byte]) { logDebug(s"length: ${rowKey.length}") if(row.varLength) { var start = 0 diff --git hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala index 4312b38..0f8baed 100644 --- hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala +++ hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DefaultSourceSuite.scala @@ -329,14 +329,7 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { equals("( KEY_FIELD < 0 OR KEY_FIELD > 1 )")) assert(executionRules.rowKeyFilter.points.size == 0) - assert(executionRules.rowKeyFilter.ranges.size == 1) - - val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes(""))) - assert(scanRange1.upperBound == null) - assert(scanRange1.isLowerBoundEqualTo) - assert(scanRange1.isUpperBoundEqualTo) - + assert(executionRules.rowKeyFilter.ranges.size == 2) assert(results.length == 5) } @@ -358,18 +351,14 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { assert(executionRules.rowKeyFilter.points.size == 0) - assert(executionRules.rowKeyFilter.ranges.size == 2) + assert(executionRules.rowKeyFilter.ranges.size == 3) val scanRange1 = executionRules.rowKeyFilter.ranges.get(0).get - assert(Bytes.equals(scanRange1.lowerBound,Bytes.toBytes(""))) assert(Bytes.equals(scanRange1.upperBound, Bytes.toBytes(2))) assert(scanRange1.isLowerBoundEqualTo) assert(!scanRange1.isUpperBoundEqualTo) val scanRange2 = executionRules.rowKeyFilter.ranges.get(1).get - assert(Bytes.equals(scanRange2.lowerBound, Bytes.toBytes(4))) - assert(scanRange2.upperBound == null) - assert(!scanRange2.isLowerBoundEqualTo) assert(scanRange2.isUpperBoundEqualTo) assert(results.length == 2) diff --git hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala index 3140ebd..ff4201c 100644 --- hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala +++ hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/DynamicLogicExpressionSuite.scala @@ -19,13 +19,17 @@ package org.apache.hadoop.hbase.spark import java.util +import org.apache.hadoop.hbase.spark.datasources.{HBaseSparkConf, JavaBytesEncoder} import org.apache.hadoop.hbase.util.Bytes import org.apache.spark.Logging +import org.apache.spark.sql.types._ import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} class DynamicLogicExpressionSuite extends FunSuite with BeforeAndAfterEach with BeforeAndAfterAll with Logging { + val encoder = JavaBytesEncoder.create(HBaseSparkConf.defaultEncoder) + test("Basic And Test") { val leftLogic = new LessThanLogicExpression("Col1", 0) val rightLogic = new GreaterThanLogicExpression("Col1", 1) @@ -35,33 +39,33 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10))) val valueFromQueryValueArray = new Array[Array[Byte]](2) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10) assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) val expressionString = andLogic.toExpressionString assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )")) - val builtExpression = DynamicLogicExpressionBuilder.build(expressionString) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10) assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) } @@ -75,41 +79,41 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(10))) val valueFromQueryValueArray = new Array[Array[Byte]](2) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10) assert(OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) - valueFromQueryValueArray(1) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10) assert(!OrLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) val expressionString = OrLogic.toExpressionString assert(expressionString.equals("( Col1 < 0 OR Col1 > 1 )")) - val builtExpression = DynamicLogicExpressionBuilder.build(expressionString) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) - valueFromQueryValueArray(1) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 5) assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(15) - valueFromQueryValueArray(1) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 15) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10) assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) - valueFromQueryValueArray(1) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) + valueFromQueryValueArray(1) = encoder.encode(IntegerType, 10) assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) } @@ -127,40 +131,40 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { val valueFromQueryValueArray = new Array[Array[Byte]](1) //great than - valueFromQueryValueArray(0) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(20) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20) assert(!greaterLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) //great than and equal - valueFromQueryValueArray(0) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5) assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) assert(greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(20) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20) assert(!greaterAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) //less than - valueFromQueryValueArray(0) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(5) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 5) assert(!lessLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) //less than and equal - valueFromQueryValueArray(0) = Bytes.toBytes(20) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20) assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(20) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 20) assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) - valueFromQueryValueArray(0) = Bytes.toBytes(10) + valueFromQueryValueArray(0) = encoder.encode(IntegerType, 10) assert(lessAndEqualLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) //equal too @@ -183,8 +187,137 @@ BeforeAndAfterEach with BeforeAndAfterAll with Logging { valueFromQueryValueArray(0) = Bytes.toBytes(5) assert(passThrough.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + } + + + test("Double Type") { + val leftLogic = new LessThanLogicExpression("Col1", 0) + val rightLogic = new GreaterThanLogicExpression("Col1", 1) + val andLogic = new AndLogicExpression(leftLogic, rightLogic) + + val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]() + + columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(-4.0d))) + val valueFromQueryValueArray = new Array[Array[Byte]](2) + valueFromQueryValueArray(0) = encoder.encode(DoubleType, 15.0d) + valueFromQueryValueArray(1) = encoder.encode(DoubleType, -5.0d) + assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(DoubleType, 10.0d) + valueFromQueryValueArray(1) = encoder.encode(DoubleType, -1.0d) + assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(DoubleType, -10.0d) + valueFromQueryValueArray(1) = encoder.encode(DoubleType, -20.0d) + assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + val expressionString = andLogic.toExpressionString + // Note that here 0 and 1 is index, instead of value. + assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )")) + + val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder) + valueFromQueryValueArray(0) = encoder.encode(DoubleType, 15.0d) + valueFromQueryValueArray(1) = encoder.encode(DoubleType, -5.0d) + assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(DoubleType, 10.0d) + valueFromQueryValueArray(1) = encoder.encode(DoubleType, -1.0d) + assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(DoubleType, -10.0d) + valueFromQueryValueArray(1) = encoder.encode(DoubleType, -20.0d) + assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + } + + test("Float Type") { + val leftLogic = new LessThanLogicExpression("Col1", 0) + val rightLogic = new GreaterThanLogicExpression("Col1", 1) + val andLogic = new AndLogicExpression(leftLogic, rightLogic) + + val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]() + columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(-4.0f))) + val valueFromQueryValueArray = new Array[Array[Byte]](2) + valueFromQueryValueArray(0) = encoder.encode(FloatType, 15.0f) + valueFromQueryValueArray(1) = encoder.encode(FloatType, -5.0f) + assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(FloatType, 10.0f) + valueFromQueryValueArray(1) = encoder.encode(FloatType, -1.0f) + assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(FloatType, -10.0f) + valueFromQueryValueArray(1) = encoder.encode(FloatType, -20.0f) + assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + val expressionString = andLogic.toExpressionString + // Note that here 0 and 1 is index, instead of value. + assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )")) + + val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder) + valueFromQueryValueArray(0) = encoder.encode(FloatType, 15.0f) + valueFromQueryValueArray(1) = encoder.encode(FloatType, -5.0f) + assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(FloatType, 10.0f) + valueFromQueryValueArray(1) = encoder.encode(FloatType, -1.0f) + assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(FloatType, -10.0f) + valueFromQueryValueArray(1) = encoder.encode(FloatType, -20.0f) + assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + } + + test("String Type") { + val leftLogic = new LessThanLogicExpression("Col1", 0) + val rightLogic = new GreaterThanLogicExpression("Col1", 1) + val andLogic = new AndLogicExpression(leftLogic, rightLogic) + + val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]() + + columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes("row005"))) + val valueFromQueryValueArray = new Array[Array[Byte]](2) + valueFromQueryValueArray(0) = encoder.encode(StringType, "row015") + valueFromQueryValueArray(1) = encoder.encode(StringType, "row000") + assert(andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(StringType, "row004") + valueFromQueryValueArray(1) = encoder.encode(StringType, "row000") + assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(StringType, "row020") + valueFromQueryValueArray(1) = encoder.encode(StringType, "row010") + assert(!andLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + val expressionString = andLogic.toExpressionString + // Note that here 0 and 1 is index, instead of value. + assert(expressionString.equals("( Col1 < 0 AND Col1 > 1 )")) + + val builtExpression = DynamicLogicExpressionBuilder.build(expressionString, encoder) + valueFromQueryValueArray(0) = encoder.encode(StringType, "row015") + valueFromQueryValueArray(1) = encoder.encode(StringType, "row000") + assert(builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(StringType, "row004") + valueFromQueryValueArray(1) = encoder.encode(StringType, "row000") + assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + + valueFromQueryValueArray(0) = encoder.encode(StringType, "row020") + valueFromQueryValueArray(1) = encoder.encode(StringType, "row010") + assert(!builtExpression.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) } + test("Boolean Type") { + val leftLogic = new LessThanLogicExpression("Col1", 0) + val rightLogic = new GreaterThanLogicExpression("Col1", 1) + + val columnToCurrentRowValueMap = new util.HashMap[String, ByteArrayComparable]() + columnToCurrentRowValueMap.put("Col1", new ByteArrayComparable(Bytes.toBytes(false))) + val valueFromQueryValueArray = new Array[Array[Byte]](2) + valueFromQueryValueArray(0) = encoder.encode(BooleanType, true) + valueFromQueryValueArray(1) = encoder.encode(BooleanType, false) + assert(leftLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + assert(!rightLogic.execute(columnToCurrentRowValueMap, valueFromQueryValueArray)) + } } diff --git hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala new file mode 100644 index 0000000..bd32ff9 --- /dev/null +++ hbase-spark/src/test/scala/org/apache/hadoop/hbase/spark/PartitionFilterSuite.scala @@ -0,0 +1,498 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with + * this work for additional information regarding copyright ownership. + * The ASF licenses this file to You under the Apache License, Version 2.0 + * (the "License"); you may not use this file except in compliance with + * the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.spark + +import org.apache.hadoop.hbase.spark.datasources.HBaseSparkConf +import org.apache.hadoop.hbase.{TableName, HBaseTestingUtility} +import org.apache.spark.sql.datasources.hbase.HBaseTableCatalog +import org.apache.spark.sql.{DataFrame, SQLContext} +import org.apache.spark.{SparkConf, SparkContext, Logging} +import org.scalatest.{BeforeAndAfterAll, BeforeAndAfterEach, FunSuite} + +case class FilterRangeRecord( + col0: Integer, + col1: Boolean, + col2: Double, + col3: Float, + col4: Int, + col5: Long, + col6: Short, + col7: String, + col8: Byte) + +object FilterRangeRecord { + def apply(i: Int): FilterRangeRecord = { + FilterRangeRecord(if (i % 2 == 0) i else -i, + i % 2 == 0, + if (i % 2 == 0) i.toDouble else -i.toDouble, + i.toFloat, + if (i % 2 == 0) i else -i, + i.toLong, + i.toShort, + s"String$i extra", + i.toByte) + } +} + +class PartitionFilterSuite extends FunSuite with + BeforeAndAfterEach with BeforeAndAfterAll with Logging { + @transient var sc: SparkContext = null + var TEST_UTIL: HBaseTestingUtility = new HBaseTestingUtility + + var sqlContext: SQLContext = null + var df: DataFrame = null + var srcDf: DataFrame = null + + def withCatalog(cat: String): DataFrame = { + sqlContext + .read + .options(Map(HBaseTableCatalog.tableCatalog -> cat)) + .format("org.apache.hadoop.hbase.spark") + .load() + } + + override def beforeAll() { + + TEST_UTIL.startMiniCluster + val sparkConf = new SparkConf + sparkConf.set(HBaseSparkConf.BLOCK_CACHE_ENABLE, "true") + sparkConf.set(HBaseSparkConf.BATCH_NUM, "100") + sparkConf.set(HBaseSparkConf.CACHE_SIZE, "100") + + sc = new SparkContext("local", "test", sparkConf) + new HBaseContext(sc, TEST_UTIL.getConfiguration) + sqlContext = new SQLContext(sc) + } + + override def afterAll() { + logInfo("shutting down minicluster") + TEST_UTIL.shutdownMiniCluster() + + sc.stop() + } + + override def beforeEach(): Unit = { + DefaultSourceStaticUtils.lastFiveExecutionRules.clear() + } + + val catalog = s"""{ + |"table":{"namespace":"default", "name":"rangeTable"}, + |"rowkey":"key", + |"columns":{ + |"col0":{"cf":"rowkey", "col":"key", "type":"int"}, + |"col1":{"cf":"cf1", "col":"col1", "type":"boolean"}, + |"col2":{"cf":"cf2", "col":"col2", "type":"double"}, + |"col3":{"cf":"cf3", "col":"col3", "type":"float"}, + |"col4":{"cf":"cf4", "col":"col4", "type":"int"}, + |"col5":{"cf":"cf5", "col":"col5", "type":"bigint"}, + |"col6":{"cf":"cf6", "col":"col6", "type":"smallint"}, + |"col7":{"cf":"cf7", "col":"col7", "type":"string"}, + |"col8":{"cf":"cf8", "col":"col8", "type":"tinyint"} + |} + |}""".stripMargin + + test("populate rangeTable") { + val sql = sqlContext + import sql.implicits._ + + val data = (0 until 32).map { i => + FilterRangeRecord(i) + } + + srcDf = sc.parallelize(data).toDF + srcDf.write.options( + Map(HBaseTableCatalog.tableCatalog -> catalog, HBaseTableCatalog.newTable -> "5")) + .format("org.apache.hadoop.hbase.spark") + .save() + } + test("rangeTable full query") { + val df = withCatalog(catalog) + df.show + assert(df.count() === 32) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| -31| + *| -29| + *| -27| + *| -25| + *| -23| + *| -21| + *| -19| + *| -17| + *| -15| + *| -13| + *| -11| + *| -9| + *| -7| + *| -5| + *| -3| + *| -1| + *+----+ + */ + test("rangeTable rowkey less than 0") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" < 0).select($"col0") + s.show + assert(s.count() === 16) + val srcS = srcDf.filter($"col0" < 0).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col4| + *+----+ + *| -31| + *| -29| + *| -27| + *| -25| + *| -23| + *| -21| + *| -19| + *| -17| + *| -15| + *| -13| + *| -11| + *| -9| + *| -7| + *| -5| + *| -3| + *| -1| + *+----+ + */ + test("rangeTable int col less than 0") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col4" < 0).select($"col4") + s.show + assert(s.count() === 16) + val srcS = srcDf.filter($"col4" < 0).select($"col4") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+-----+ + *| col2| + *+-----+ + *| 0.0| + *| 2.0| + *|-31.0| + *|-29.0| + *|-27.0| + *|-25.0| + *|-23.0| + *|-21.0| + *|-19.0| + *|-17.0| + *|-15.0| + *|-13.0| + *|-11.0| + *| -9.0| + *| -7.0| + *| -5.0| + *| -3.0| + *| -1.0| + *+-----+ + */ + test("rangeTable double col less than 0") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col2" < 3.0).select($"col2") + s.show + assert(s.count() === 18) + val srcS = srcDf.filter($"col2" < 3.0).select($"col2") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + * expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| -31| + *| -29| + *| -27| + *| -25| + *| -23| + *| -21| + *| -19| + *| -17| + *| -15| + *| -13| + *| -11| + *+----+ + * + */ + test("rangeTable lessequal than -10") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" <= -10).select($"col0") + s.show + assert(s.count() === 11) + val srcS = srcDf.filter($"col0" <= -10).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| -31| + *| -29| + *| -27| + *| -25| + *| -23| + *| -21| + *| -19| + *| -17| + *| -15| + *| -13| + *| -11| + *| -9| + *+----+ + */ + test("rangeTable lessequal than -9") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" <= -9).select($"col0") + s.show + assert(s.count() === 12) + val srcS = srcDf.filter($"col0" <= -9).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| 0| + *| 2| + *| 4| + *| 6| + *| 8| + *| 10| + *| 12| + *| 14| + *| 16| + *| 18| + *| 20| + *| 22| + *| 24| + *| 26| + *| 28| + *| 30| + *| -9| + *| -7| + *| -5| + *| -3| + *+----+ + */ + test("rangeTable greaterequal than -9") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" >= -9).select($"col0") + s.show + assert(s.count() === 21) + val srcS = srcDf.filter($"col0" >= -9).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| 0| + *| 2| + *| 4| + *| 6| + *| 8| + *| 10| + *| 12| + *| 14| + *| 16| + *| 18| + *| 20| + *| 22| + *| 24| + *| 26| + *| 28| + *| 30| + *+----+ + */ + test("rangeTable greaterequal than 0") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" >= 0).select($"col0") + s.show + assert(s.count() === 16) + val srcS = srcDf.filter($"col0" >= 0).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| 12| + *| 14| + *| 16| + *| 18| + *| 20| + *| 22| + *| 24| + *| 26| + *| 28| + *| 30| + *+----+ + */ + test("rangeTable greater than 10") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" > 10).select($"col0") + s.show + assert(s.count() === 10) + val srcS = srcDf.filter($"col0" > 10).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| 0| + *| 2| + *| 4| + *| 6| + *| 8| + *| 10| + *| -9| + *| -7| + *| -5| + *| -3| + *| -1| + *+----+ + */ + test("rangeTable and") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" > -10 && $"col0" <= 10).select($"col0") + s.show + assert(s.count() === 11) + val srcS = srcDf.filter($"col0" > -10 && $"col0" <= 10).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| 12| + *| 14| + *| 16| + *| 18| + *| 20| + *| 22| + *| 24| + *| 26| + *| 28| + *| 30| + *| -31| + *| -29| + *| -27| + *| -25| + *| -23| + *| -21| + *| -19| + *| -17| + *| -15| + *| -13| + *+----+ + */ + + test("or") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" <= -10 || $"col0" > 10).select($"col0") + s.show + assert(s.count() === 21) + val srcS = srcDf.filter($"col0" <= -10 || $"col0" > 10).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } + + /** + *expected result: only showing top 20 rows + *+----+ + *|col0| + *+----+ + *| 0| + *| 2| + *| 4| + *| 6| + *| 8| + *| 10| + *| 12| + *| 14| + *| 16| + *| 18| + *| 20| + *| 22| + *| 24| + *| 26| + *| 28| + *| 30| + *| -31| + *| -29| + *| -27| + *| -25| + *+----+ + */ + test("rangeTable all") { + val sql = sqlContext + import sql.implicits._ + val df = withCatalog(catalog) + val s = df.filter($"col0" >= -100).select($"col0") + s.show + assert(s.count() === 32) + val srcS = srcDf.filter($"col0" >= -100).select($"col0") + assert(srcS.collect().toSet === s.collect().toSet) + } +}