diff --git hbase-examples/pom.xml hbase-examples/pom.xml index 28bdbe6..940af46 100644 --- hbase-examples/pom.xml +++ hbase-examples/pom.xml @@ -63,14 +63,26 @@ + org.apache.hbase + hbase-common + test-jar + test + + org.apache.hbase hbase-server + org.apache.hbase + hbase-server + test-jar + test + + org.apache.thrift libthrift - + org.slf4j slf4j-api @@ -79,4 +91,133 @@ slf4j-log4j12 + + + + + + hadoop-1.0 + + + !hadoop.profile + + + + + org.apache.hadoop + hadoop-core + + + org.apache.hadoop + hadoop-test + + + + + + hadoop-2.0 + + + hadoop.profile + 2.0 + + + + + org.apache.hadoop + hadoop-client + + + org.apache.hadoop + hadoop-annotations + + + org.apache.hadoop + hadoop-minicluster + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + generate-test-resources + + build-classpath + + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + + + hadoop-3.0 + + + hadoop.profile + 3.0 + + + + 3.0-SNAPSHOT + + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-annotations + + + org.apache.hadoop + hadoop-minicluster + + + + + + maven-dependency-plugin + + + create-mrapp-generated-classpath + generate-test-resources + + build-classpath + + + + ${project.build.directory}/test-classes/mrapp-generated-classpath + + + + + + + + diff --git hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java new file mode 100644 index 0000000..2bcdd58 --- /dev/null +++ hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java @@ -0,0 +1,290 @@ +/* + * 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.coprocessor.example; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.HConstants.OperationStatusCode; +import org.apache.hadoop.hbase.client.Delete; +import org.apache.hadoop.hbase.client.Mutation; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.CoprocessorException; +import org.apache.hadoop.hbase.coprocessor.CoprocessorService; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder; +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.regionserver.HRegion; +import org.apache.hadoop.hbase.regionserver.OperationStatus; +import org.apache.hadoop.hbase.regionserver.RegionScanner; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.Pair; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; + +/** + * Defines a protocol to delete data in bulk based on a scan. The scan can be range scan or with + * conditions(filters) etc.This can be used to delete rows, column family(s), column qualifier(s) + * or version(s) of columns.When delete type is FAMILY or COLUMN, which all family(s) or column(s) + * getting deleted will be determined by the Scan. Scan need to select all the families/qualifiers + * which need to be deleted.When delete type is VERSION, Which column(s) and version(s) to be + * deleted will be determined by the Scan. Scan need to select all the qualifiers and its versions + * which needs to be deleted.When a timestamp is passed only one version at that timestamp will be + * deleted(even if Scan fetches many versions). When timestamp passed as null, all the versions + * which the Scan selects will get deleted. + * + *
Example:
+ * Scan scan = new Scan();
+ * // set scan properties(rowkey range, filters, timerange etc).
+ * HTable ht = ...;
+ * long noOfDeletedRows = 0L;
+ * Batch.Call<BulkDeleteService, BulkDeleteResponse> callable = 
+ *     new Batch.Call<BulkDeleteService, BulkDeleteResponse>() {
+ *   ServerRpcController controller = new ServerRpcController();
+ *   BlockingRpcCallback<BulkDeleteResponse> rpcCallback = 
+ *     new BlockingRpcCallback<BulkDeleteResponse>();
+ *
+ *   public BulkDeleteResponse call(BulkDeleteService service) throws IOException {
+ *     Builder builder = BulkDeleteRequest.newBuilder();
+ *     builder.setScan(ProtobufUtil.toScan(scan));
+ *     builder.setDeleteType(DeleteType.VERSION);
+ *     builder.setRowBatchSize(rowBatchSize);
+ *     // Set optional timestamp if needed
+ *     builder.setTimestamp(timeStamp);
+ *     service.delete(controller, builder.build(), rpcCallback);
+ *     return rpcCallback.get();
+ *   }
+ * };
+ * Map<byte[], BulkDeleteResponse> result = ht.coprocessorService(BulkDeleteService.class, scan
+ *     .getStartRow(), scan.getStopRow(), callable);
+ * for (BulkDeleteResponse response : result.values()) {
+ *   noOfDeletedRows += response.getRowsDeleted();
+ * }
+ * 
+ */ +public class BulkDeleteEndpoint extends BulkDeleteService implements CoprocessorService, + Coprocessor { + private static final String NO_OF_VERSIONS_TO_DELETE = "noOfVersionsToDelete"; + private static final Log LOG = LogFactory.getLog(BulkDeleteEndpoint.class); + + private RegionCoprocessorEnvironment env; + + @Override + public Service getService() { + return this; + } + + @Override + public void delete(RpcController controller, BulkDeleteRequest request, + RpcCallback done) { + long totalRowsDeleted = 0L; + long totalVersionsDeleted = 0L; + HRegion region = env.getRegion(); + int rowBatchSize = request.getRowBatchSize(); + Long timestamp = null; + if (request.hasTimestamp()) { + timestamp = request.getTimestamp(); + } + DeleteType deleteType = request.getDeleteType(); + boolean hasMore = true; + RegionScanner scanner = null; + try { + Scan scan = ProtobufUtil.toScan(request.getScan()); + if (scan.getFilter() == null && deleteType == DeleteType.ROW) { + // What we need is just the rowkeys. So only 1st KV from any row is enough. + // Only when it is a row delete, we can apply this filter. + // In other types we rely on the scan to know which all columns to be deleted. + scan.setFilter(new FirstKeyOnlyFilter()); + } + // Here by assume that the scan is perfect with the appropriate + // filter and having necessary column(s). + scanner = region.getScanner(scan); + while (hasMore) { + List> deleteRows = new ArrayList>(rowBatchSize); + for (int i = 0; i < rowBatchSize; i++) { + List results = new ArrayList(); + hasMore = scanner.next(results); + if (results.size() > 0) { + deleteRows.add(results); + } + if (!hasMore) { + // There are no more rows. + break; + } + } + if (deleteRows.size() > 0) { + Pair[] deleteWithLockArr = new Pair[deleteRows.size()]; + int i = 0; + for (List deleteRow : deleteRows) { + Delete delete = createDeleteMutation(deleteRow, deleteType, timestamp); + deleteWithLockArr[i++] = new Pair(delete, null); + } + OperationStatus[] opStatus = region.batchMutate(deleteWithLockArr); + for (i = 0; i < opStatus.length; i++) { + if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) { + break; + } + totalRowsDeleted++; + if (deleteType == DeleteType.VERSION) { + byte[] versionsDeleted = deleteWithLockArr[i].getFirst().getAttribute( + NO_OF_VERSIONS_TO_DELETE); + if (versionsDeleted != null) { + totalVersionsDeleted += Bytes.toInt(versionsDeleted); + } + } + } + } + } + } catch (IOException ioe) { + LOG.error(ioe); + // Call ServerRpcController#getFailedOn() to retrieve this IOException at client side. + ResponseConverter.setControllerException(controller, ioe); + } finally { + if (scanner != null) { + try { + scanner.close(); + } catch (IOException ioe) { + LOG.error(ioe); + } + } + } + Builder responseBuilder = BulkDeleteResponse.newBuilder(); + responseBuilder.setRowsDeleted(totalRowsDeleted); + if (deleteType == DeleteType.VERSION) { + responseBuilder.setVersionsDeleted(totalVersionsDeleted); + } + BulkDeleteResponse result = responseBuilder.build(); + done.run(result); + } + + private Delete createDeleteMutation(List deleteRow, DeleteType deleteType, + Long timestamp) { + long ts; + if (timestamp == null) { + ts = HConstants.LATEST_TIMESTAMP; + } else { + ts = timestamp; + } + // We just need the rowkey. Get it from 1st KV. + byte[] row = deleteRow.get(0).getRow(); + Delete delete = new Delete(row, ts, null); + if (deleteType == DeleteType.FAMILY) { + Set families = new TreeSet(Bytes.BYTES_COMPARATOR); + for (KeyValue kv : deleteRow) { + if (families.add(kv.getFamily())) { + delete.deleteFamily(kv.getFamily(), ts); + } + } + } else if (deleteType == DeleteType.COLUMN) { + Set columns = new HashSet(); + for (KeyValue kv : deleteRow) { + Column column = new Column(kv.getFamily(), kv.getQualifier()); + if (columns.add(column)) { + // Making deleteColumns() calls more than once for the same cf:qualifier is not correct + // Every call to deleteColumns() will add a new KV to the familymap which will finally + // get written to the memstore as part of delete(). + delete.deleteColumns(column.family, column.qualifier, ts); + } + } + } else if (deleteType == DeleteType.VERSION) { + // When some timestamp was passed to the delete() call only one version of the column (with + // given timestamp) will be deleted. If no timestamp passed, it will delete N versions. + // How many versions will get deleted depends on the Scan being passed. All the KVs that + // the scan fetched will get deleted. + int noOfVersionsToDelete = 0; + if (timestamp == null) { + for (KeyValue kv : deleteRow) { + delete.deleteColumn(kv.getFamily(), kv.getQualifier(), kv.getTimestamp()); + noOfVersionsToDelete++; + } + } else { + Set columns = new HashSet(); + for (KeyValue kv : deleteRow) { + Column column = new Column(kv.getFamily(), kv.getQualifier()); + // Only one version of particular column getting deleted. + if (columns.add(column)) { + delete.deleteColumn(column.family, column.qualifier, ts); + noOfVersionsToDelete++; + } + } + } + delete.setAttribute(NO_OF_VERSIONS_TO_DELETE, Bytes.toBytes(noOfVersionsToDelete)); + } + return delete; + } + + private static class Column { + private byte[] family; + private byte[] qualifier; + + public Column(byte[] family, byte[] qualifier) { + this.family = family; + this.qualifier = qualifier; + } + + @Override + public boolean equals(Object other) { + if (!(other instanceof Column)) { + return false; + } + Column column = (Column) other; + return Bytes.equals(this.family, column.family) + && Bytes.equals(this.qualifier, column.qualifier); + } + + @Override + public int hashCode() { + int h = 31; + h = h + 13 * Bytes.hashCode(this.family); + h = h + 13 * Bytes.hashCode(this.qualifier); + return h; + } + } + + @Override + public void start(CoprocessorEnvironment env) throws IOException { + if (env instanceof RegionCoprocessorEnvironment) { + this.env = (RegionCoprocessorEnvironment) env; + } else { + throw new CoprocessorException("Must be loaded on a table region!"); + } + } + + @Override + public void stop(CoprocessorEnvironment env) throws IOException { + // nothing to do + } +} diff --git hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java new file mode 100644 index 0000000..465b4e0 --- /dev/null +++ hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java @@ -0,0 +1,164 @@ +/* + * 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.coprocessor.example; + +import com.google.protobuf.RpcCallback; +import com.google.protobuf.RpcController; +import com.google.protobuf.Service; +import org.apache.hadoop.hbase.Coprocessor; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.CoprocessorException; +import org.apache.hadoop.hbase.coprocessor.CoprocessorService; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; +import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; +import org.apache.hadoop.hbase.protobuf.ResponseConverter; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.util.Bytes; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +/** + * Sample coprocessor endpoint exposing a Service interface for counting rows and key values. + * + *

+ * For the protocol buffer definition of the RowCountService, see the source file located under + * hbase-server/src/main/protobuf/Examples.proto. + *

+ */ +public class RowCountEndpoint extends ExampleProtos.RowCountService + implements Coprocessor, CoprocessorService { + private RegionCoprocessorEnvironment env; + + public RowCountEndpoint() { + } + + /** + * Just returns a reference to this object, which implements the RowCounterService interface. + */ + @Override + public Service getService() { + return this; + } + + /** + * Returns a count of the rows in the region where this coprocessor is loaded. + */ + @Override + public void getRowCount(RpcController controller, ExampleProtos.CountRequest request, + RpcCallback done) { + Scan scan = new Scan(); + scan.setFilter(new FirstKeyOnlyFilter()); + ExampleProtos.CountResponse response = null; + InternalScanner scanner = null; + try { + scanner = env.getRegion().getScanner(scan); + List results = new ArrayList(); + boolean hasMore = false; + byte[] lastRow = null; + long count = 0; + do { + hasMore = scanner.next(results); + for (KeyValue kv : results) { + byte[] currentRow = kv.getRow(); + if (lastRow == null || !Bytes.equals(lastRow, currentRow)) { + lastRow = currentRow; + count++; + } + } + results.clear(); + } while (hasMore); + + response = ExampleProtos.CountResponse.newBuilder() + .setCount(count).build(); + } catch (IOException ioe) { + ResponseConverter.setControllerException(controller, ioe); + } finally { + if (scanner != null) { + try { + scanner.close(); + } catch (IOException ignored) {} + } + } + done.run(response); + } + + /** + * Returns a count of all KeyValues in the region where this coprocessor is loaded. + */ + @Override + public void getKeyValueCount(RpcController controller, ExampleProtos.CountRequest request, + RpcCallback done) { + ExampleProtos.CountResponse response = null; + InternalScanner scanner = null; + try { + scanner = env.getRegion().getScanner(new Scan()); + List results = new ArrayList(); + boolean hasMore = false; + long count = 0; + do { + hasMore = scanner.next(results); + for (KeyValue kv : results) { + count++; + } + results.clear(); + } while (hasMore); + + response = ExampleProtos.CountResponse.newBuilder() + .setCount(count).build(); + } catch (IOException ioe) { + ResponseConverter.setControllerException(controller, ioe); + } finally { + if (scanner != null) { + try { + scanner.close(); + } catch (IOException ignored) {} + } + } + done.run(response); + } + + /** + * Stores a reference to the coprocessor environment provided by the + * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} from the region where this + * coprocessor is loaded. Since this is a coprocessor endpoint, it always expects to be loaded + * on a table region, so always expects this to be an instance of + * {@link RegionCoprocessorEnvironment}. + * @param env the environment provided by the coprocessor host + * @throws IOException if the provided environment is not an instance of + * {@code RegionCoprocessorEnvironment} + */ + @Override + public void start(CoprocessorEnvironment env) throws IOException { + if (env instanceof RegionCoprocessorEnvironment) { + this.env = (RegionCoprocessorEnvironment)env; + } else { + throw new CoprocessorException("Must be loaded on a table region!"); + } + } + + @Override + public void stop(CoprocessorEnvironment env) throws IOException { + // nothing to do + } +} diff --git hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java new file mode 100644 index 0000000..2e4eb48 --- /dev/null +++ hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java @@ -0,0 +1,232 @@ +/* + * Copyright The Apache Software Foundation + * + * 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.coprocessor.example; + +import java.io.IOException; +import java.util.Collections; +import java.util.List; +import java.util.NavigableSet; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.hbase.CoprocessorEnvironment; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; +import org.apache.hadoop.hbase.coprocessor.ObserverContext; +import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; +import org.apache.hadoop.hbase.coprocessor.RegionObserver; +import org.apache.hadoop.hbase.regionserver.HStore; +import org.apache.hadoop.hbase.regionserver.HStore.ScanInfo; +import org.apache.hadoop.hbase.regionserver.InternalScanner; +import org.apache.hadoop.hbase.regionserver.KeyValueScanner; +import org.apache.hadoop.hbase.regionserver.ScanType; +import org.apache.hadoop.hbase.regionserver.Store; +import org.apache.hadoop.hbase.regionserver.StoreScanner; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.KeeperException; +import org.apache.zookeeper.WatchedEvent; +import org.apache.zookeeper.Watcher; +import org.apache.zookeeper.ZooKeeper; + +/** + * This is an example showing how a RegionObserver could configured + * via ZooKeeper in order to control a Region compaction, flush, and scan policy. + * + * This also demonstrated the use of shared {@link RegionObserver} state. + * See {@link RegionCoprocessorEnvironment#getSharedData()}. + * + * This would be useful for an incremental backup tool, which would indicate the last + * time of a successful backup via ZK and instruct HBase to not delete data that was + * inserted since (based on wall clock time). + * + * This implements org.apache.zookeeper.Watcher directly instead of using + * {@link ZooKeeperWatcher}, because RegionObservers come and go and currently + * listeners registered with ZooKeeperWatcher cannot be removed. + */ +public class ZooKeeperScanPolicyObserver extends BaseRegionObserver { + public static String node = "/backup/example/lastbackup"; + public static String zkkey = "ZK"; + private static final Log LOG = LogFactory.getLog(ZooKeeperScanPolicyObserver.class); + + /** + * Internal watcher that keep "data" up to date asynchronously. + */ + private static class ZKWatcher implements Watcher { + private byte[] data = null; + private ZooKeeper zk; + private volatile boolean needSetup = true; + private volatile long lastSetupTry = 0; + + public ZKWatcher(ZooKeeper zk) { + this.zk = zk; + // trigger the listening + getData(); + } + + /** + * Get the maintained data. In case of any ZK exceptions this will retry + * establishing the connection (but not more than twice/minute). + * + * getData is on the critical path, so make sure it is fast unless there is + * a problem (network partion, ZK ensemble down, etc) + * Make sure at most one (unlucky) thread retries and other threads don't pile up + * while that threads tries to recreate the connection. + * + * @return the last know version of the data + */ + public byte[] getData() { + // try at most twice/minute + if (needSetup && EnvironmentEdgeManager.currentTimeMillis() > lastSetupTry + 30000) { + synchronized (this) { + // make sure only one thread tries to reconnect + if (needSetup) { + needSetup = false; + } else { + return data; + } + } + // do this without the lock held to avoid threads piling up on this lock, + // as it can take a while + try { + LOG.debug("Connecting to ZK"); + // record this attempt + lastSetupTry = EnvironmentEdgeManager.currentTimeMillis(); + if (zk.exists(node, false) != null) { + data = zk.getData(node, this, null); + LOG.debug("Read synchronously: "+(data == null ? "null" : Bytes.toLong(data))); + } else { + zk.exists(node, this); + } + } catch (Exception x) { + // try again if this fails + needSetup = true; + } + } + return data; + } + + @Override + public void process(WatchedEvent event) { + switch(event.getType()) { + case NodeDataChanged: + case NodeCreated: + try { + // get data and re-watch + data = zk.getData(node, this, null); + LOG.debug("Read asynchronously: "+(data == null ? "null" : Bytes.toLong(data))); + } catch (InterruptedException ix) { + } catch (KeeperException kx) { + needSetup = true; + } + break; + + case NodeDeleted: + try { + // just re-watch + zk.exists(node, this); + data = null; + } catch (InterruptedException ix) { + } catch (KeeperException kx) { + needSetup = true; + } + break; + + default: + // ignore + } + } + } + + @Override + public void start(CoprocessorEnvironment e) throws IOException { + RegionCoprocessorEnvironment re = (RegionCoprocessorEnvironment) e; + if (!re.getSharedData().containsKey(zkkey)) { + // there is a short race here + // in the worst case we create a watcher that will be notified once + re.getSharedData().putIfAbsent( + zkkey, + new ZKWatcher(re.getRegionServerServices().getZooKeeper() + .getRecoverableZooKeeper().getZooKeeper())); + } + } + + @Override + public void stop(CoprocessorEnvironment e) throws IOException { + // nothing to do here + } + + protected ScanInfo getScanInfo(HStore store, RegionCoprocessorEnvironment e) { + byte[] data = ((ZKWatcher)e.getSharedData().get(zkkey)).getData(); + if (data == null) { + return null; + } + ScanInfo oldSI = store.getScanInfo(); + if (oldSI.getTtl() == Long.MAX_VALUE) { + return null; + } + long ttl = Math.max(EnvironmentEdgeManager.currentTimeMillis() - Bytes.toLong(data), oldSI.getTtl()); + return new ScanInfo(store.getFamily(), ttl, + oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); + } + + @Override + public InternalScanner preFlushScannerOpen(final ObserverContext c, + HStore store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { + HStore.ScanInfo scanInfo = getScanInfo(store, c.getEnvironment()); + if (scanInfo == null) { + // take default action + return null; + } + Scan scan = new Scan(); + scan.setMaxVersions(scanInfo.getMaxVersions()); + return new StoreScanner(store, scanInfo, scan, Collections.singletonList(memstoreScanner), + ScanType.MINOR_COMPACT, store.getHRegion().getSmallestReadPoint(), + HConstants.OLDEST_TIMESTAMP); + } + + @Override + public InternalScanner preCompactScannerOpen(final ObserverContext c, + HStore store, List scanners, ScanType scanType, long earliestPutTs, + InternalScanner s) throws IOException { + HStore.ScanInfo scanInfo = getScanInfo(store, c.getEnvironment()); + if (scanInfo == null) { + // take default action + return null; + } + Scan scan = new Scan(); + scan.setMaxVersions(scanInfo.getMaxVersions()); + return new StoreScanner(store, scanInfo, scan, scanners, scanType, store.getHRegion() + .getSmallestReadPoint(), earliestPutTs); + } + + @Override + public KeyValueScanner preStoreScannerOpen(final ObserverContext c, + final HStore store, final Scan scan, final NavigableSet targetCols, + final KeyValueScanner s) throws IOException { + HStore.ScanInfo scanInfo = getScanInfo(store, c.getEnvironment()); + if (scanInfo == null) { + // take default action + return null; + } + return new StoreScanner(store, scanInfo, scan, targetCols); + } +} diff --git hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/BulkDeleteProtos.java hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/BulkDeleteProtos.java new file mode 100644 index 0000000..2d421e1 --- /dev/null +++ hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/BulkDeleteProtos.java @@ -0,0 +1,1512 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: BulkDelete.proto + +package org.apache.hadoop.hbase.coprocessor.example.generated; + +public final class BulkDeleteProtos { + private BulkDeleteProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface BulkDeleteRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required .Scan scan = 1; + boolean hasScan(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan(); + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder(); + + // required .BulkDeleteRequest.DeleteType deleteType = 2; + boolean hasDeleteType(); + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType getDeleteType(); + + // optional uint64 timestamp = 3; + boolean hasTimestamp(); + long getTimestamp(); + + // required uint32 rowBatchSize = 4; + boolean hasRowBatchSize(); + int getRowBatchSize(); + } + public static final class BulkDeleteRequest extends + com.google.protobuf.GeneratedMessage + implements BulkDeleteRequestOrBuilder { + // Use BulkDeleteRequest.newBuilder() to construct. + private BulkDeleteRequest(Builder builder) { + super(builder); + } + private BulkDeleteRequest(boolean noInit) {} + + private static final BulkDeleteRequest defaultInstance; + public static BulkDeleteRequest getDefaultInstance() { + return defaultInstance; + } + + public BulkDeleteRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_fieldAccessorTable; + } + + public enum DeleteType + implements com.google.protobuf.ProtocolMessageEnum { + ROW(0, 0), + FAMILY(1, 1), + COLUMN(2, 2), + VERSION(3, 3), + ; + + public static final int ROW_VALUE = 0; + public static final int FAMILY_VALUE = 1; + public static final int COLUMN_VALUE = 2; + public static final int VERSION_VALUE = 3; + + + public final int getNumber() { return value; } + + public static DeleteType valueOf(int value) { + switch (value) { + case 0: return ROW; + case 1: return FAMILY; + case 2: return COLUMN; + case 3: return VERSION; + default: return null; + } + } + + public static com.google.protobuf.Internal.EnumLiteMap + internalGetValueMap() { + return internalValueMap; + } + private static com.google.protobuf.Internal.EnumLiteMap + internalValueMap = + new com.google.protobuf.Internal.EnumLiteMap() { + public DeleteType findValueByNumber(int number) { + return DeleteType.valueOf(number); + } + }; + + public final com.google.protobuf.Descriptors.EnumValueDescriptor + getValueDescriptor() { + return getDescriptor().getValues().get(index); + } + public final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptorForType() { + return getDescriptor(); + } + public static final com.google.protobuf.Descriptors.EnumDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDescriptor().getEnumTypes().get(0); + } + + private static final DeleteType[] VALUES = { + ROW, FAMILY, COLUMN, VERSION, + }; + + public static DeleteType valueOf( + com.google.protobuf.Descriptors.EnumValueDescriptor desc) { + if (desc.getType() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "EnumValueDescriptor is not for this type."); + } + return VALUES[desc.getIndex()]; + } + + private final int index; + private final int value; + + private DeleteType(int index, int value) { + this.index = index; + this.value = value; + } + + // @@protoc_insertion_point(enum_scope:BulkDeleteRequest.DeleteType) + } + + private int bitField0_; + // required .Scan scan = 1; + public static final int SCAN_FIELD_NUMBER = 1; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan scan_; + public boolean hasScan() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan() { + return scan_; + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder() { + return scan_; + } + + // required .BulkDeleteRequest.DeleteType deleteType = 2; + public static final int DELETETYPE_FIELD_NUMBER = 2; + private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType deleteType_; + public boolean hasDeleteType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType getDeleteType() { + return deleteType_; + } + + // optional uint64 timestamp = 3; + public static final int TIMESTAMP_FIELD_NUMBER = 3; + private long timestamp_; + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTimestamp() { + return timestamp_; + } + + // required uint32 rowBatchSize = 4; + public static final int ROWBATCHSIZE_FIELD_NUMBER = 4; + private int rowBatchSize_; + public boolean hasRowBatchSize() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getRowBatchSize() { + return rowBatchSize_; + } + + private void initFields() { + scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); + deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; + timestamp_ = 0L; + rowBatchSize_ = 0; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasScan()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasDeleteType()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasRowBatchSize()) { + memoizedIsInitialized = 0; + return false; + } + if (!getScan().isInitialized()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeMessage(1, scan_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeEnum(2, deleteType_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + output.writeUInt64(3, timestamp_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + output.writeUInt32(4, rowBatchSize_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeMessageSize(1, scan_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeEnumSize(2, deleteType_.getNumber()); + } + if (((bitField0_ & 0x00000004) == 0x00000004)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(3, timestamp_); + } + if (((bitField0_ & 0x00000008) == 0x00000008)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt32Size(4, rowBatchSize_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest other = (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest) obj; + + boolean result = true; + result = result && (hasScan() == other.hasScan()); + if (hasScan()) { + result = result && getScan() + .equals(other.getScan()); + } + result = result && (hasDeleteType() == other.hasDeleteType()); + if (hasDeleteType()) { + result = result && + (getDeleteType() == other.getDeleteType()); + } + result = result && (hasTimestamp() == other.hasTimestamp()); + if (hasTimestamp()) { + result = result && (getTimestamp() + == other.getTimestamp()); + } + result = result && (hasRowBatchSize() == other.hasRowBatchSize()); + if (hasRowBatchSize()) { + result = result && (getRowBatchSize() + == other.getRowBatchSize()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasScan()) { + hash = (37 * hash) + SCAN_FIELD_NUMBER; + hash = (53 * hash) + getScan().hashCode(); + } + if (hasDeleteType()) { + hash = (37 * hash) + DELETETYPE_FIELD_NUMBER; + hash = (53 * hash) + hashEnum(getDeleteType()); + } + if (hasTimestamp()) { + hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getTimestamp()); + } + if (hasRowBatchSize()) { + hash = (37 * hash) + ROWBATCHSIZE_FIELD_NUMBER; + hash = (53 * hash) + getRowBatchSize(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + getScanFieldBuilder(); + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + if (scanBuilder_ == null) { + scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); + } else { + scanBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; + bitField0_ = (bitField0_ & ~0x00000002); + timestamp_ = 0L; + bitField0_ = (bitField0_ & ~0x00000004); + rowBatchSize_ = 0; + bitField0_ = (bitField0_ & ~0x00000008); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest build() { + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest buildPartial() { + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest result = new org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + if (scanBuilder_ == null) { + result.scan_ = scan_; + } else { + result.scan_ = scanBuilder_.build(); + } + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.deleteType_ = deleteType_; + if (((from_bitField0_ & 0x00000004) == 0x00000004)) { + to_bitField0_ |= 0x00000004; + } + result.timestamp_ = timestamp_; + if (((from_bitField0_ & 0x00000008) == 0x00000008)) { + to_bitField0_ |= 0x00000008; + } + result.rowBatchSize_ = rowBatchSize_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest other) { + if (other == org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance()) return this; + if (other.hasScan()) { + mergeScan(other.getScan()); + } + if (other.hasDeleteType()) { + setDeleteType(other.getDeleteType()); + } + if (other.hasTimestamp()) { + setTimestamp(other.getTimestamp()); + } + if (other.hasRowBatchSize()) { + setRowBatchSize(other.getRowBatchSize()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasScan()) { + + return false; + } + if (!hasDeleteType()) { + + return false; + } + if (!hasRowBatchSize()) { + + return false; + } + if (!getScan().isInitialized()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.newBuilder(); + if (hasScan()) { + subBuilder.mergeFrom(getScan()); + } + input.readMessage(subBuilder, extensionRegistry); + setScan(subBuilder.buildPartial()); + break; + } + case 16: { + int rawValue = input.readEnum(); + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType value = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.valueOf(rawValue); + if (value == null) { + unknownFields.mergeVarintField(2, rawValue); + } else { + bitField0_ |= 0x00000002; + deleteType_ = value; + } + break; + } + case 24: { + bitField0_ |= 0x00000004; + timestamp_ = input.readUInt64(); + break; + } + case 32: { + bitField0_ |= 0x00000008; + rowBatchSize_ = input.readUInt32(); + break; + } + } + } + } + + private int bitField0_; + + // required .Scan scan = 1; + private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder> scanBuilder_; + public boolean hasScan() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan() { + if (scanBuilder_ == null) { + return scan_; + } else { + return scanBuilder_.getMessage(); + } + } + public Builder setScan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan value) { + if (scanBuilder_ == null) { + if (value == null) { + throw new NullPointerException(); + } + scan_ = value; + onChanged(); + } else { + scanBuilder_.setMessage(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder setScan( + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder builderForValue) { + if (scanBuilder_ == null) { + scan_ = builderForValue.build(); + onChanged(); + } else { + scanBuilder_.setMessage(builderForValue.build()); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder mergeScan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan value) { + if (scanBuilder_ == null) { + if (((bitField0_ & 0x00000001) == 0x00000001) && + scan_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance()) { + scan_ = + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.newBuilder(scan_).mergeFrom(value).buildPartial(); + } else { + scan_ = value; + } + onChanged(); + } else { + scanBuilder_.mergeFrom(value); + } + bitField0_ |= 0x00000001; + return this; + } + public Builder clearScan() { + if (scanBuilder_ == null) { + scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); + onChanged(); + } else { + scanBuilder_.clear(); + } + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder getScanBuilder() { + bitField0_ |= 0x00000001; + onChanged(); + return getScanFieldBuilder().getBuilder(); + } + public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder() { + if (scanBuilder_ != null) { + return scanBuilder_.getMessageOrBuilder(); + } else { + return scan_; + } + } + private com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder> + getScanFieldBuilder() { + if (scanBuilder_ == null) { + scanBuilder_ = new com.google.protobuf.SingleFieldBuilder< + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder>( + scan_, + getParentForChildren(), + isClean()); + scan_ = null; + } + return scanBuilder_; + } + + // required .BulkDeleteRequest.DeleteType deleteType = 2; + private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; + public boolean hasDeleteType() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType getDeleteType() { + return deleteType_; + } + public Builder setDeleteType(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + deleteType_ = value; + onChanged(); + return this; + } + public Builder clearDeleteType() { + bitField0_ = (bitField0_ & ~0x00000002); + deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; + onChanged(); + return this; + } + + // optional uint64 timestamp = 3; + private long timestamp_ ; + public boolean hasTimestamp() { + return ((bitField0_ & 0x00000004) == 0x00000004); + } + public long getTimestamp() { + return timestamp_; + } + public Builder setTimestamp(long value) { + bitField0_ |= 0x00000004; + timestamp_ = value; + onChanged(); + return this; + } + public Builder clearTimestamp() { + bitField0_ = (bitField0_ & ~0x00000004); + timestamp_ = 0L; + onChanged(); + return this; + } + + // required uint32 rowBatchSize = 4; + private int rowBatchSize_ ; + public boolean hasRowBatchSize() { + return ((bitField0_ & 0x00000008) == 0x00000008); + } + public int getRowBatchSize() { + return rowBatchSize_; + } + public Builder setRowBatchSize(int value) { + bitField0_ |= 0x00000008; + rowBatchSize_ = value; + onChanged(); + return this; + } + public Builder clearRowBatchSize() { + bitField0_ = (bitField0_ & ~0x00000008); + rowBatchSize_ = 0; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:BulkDeleteRequest) + } + + static { + defaultInstance = new BulkDeleteRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:BulkDeleteRequest) + } + + public interface BulkDeleteResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required uint64 rowsDeleted = 1; + boolean hasRowsDeleted(); + long getRowsDeleted(); + + // optional uint64 versionsDeleted = 2; + boolean hasVersionsDeleted(); + long getVersionsDeleted(); + } + public static final class BulkDeleteResponse extends + com.google.protobuf.GeneratedMessage + implements BulkDeleteResponseOrBuilder { + // Use BulkDeleteResponse.newBuilder() to construct. + private BulkDeleteResponse(Builder builder) { + super(builder); + } + private BulkDeleteResponse(boolean noInit) {} + + private static final BulkDeleteResponse defaultInstance; + public static BulkDeleteResponse getDefaultInstance() { + return defaultInstance; + } + + public BulkDeleteResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_fieldAccessorTable; + } + + private int bitField0_; + // required uint64 rowsDeleted = 1; + public static final int ROWSDELETED_FIELD_NUMBER = 1; + private long rowsDeleted_; + public boolean hasRowsDeleted() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRowsDeleted() { + return rowsDeleted_; + } + + // optional uint64 versionsDeleted = 2; + public static final int VERSIONSDELETED_FIELD_NUMBER = 2; + private long versionsDeleted_; + public boolean hasVersionsDeleted() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getVersionsDeleted() { + return versionsDeleted_; + } + + private void initFields() { + rowsDeleted_ = 0L; + versionsDeleted_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasRowsDeleted()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeUInt64(1, rowsDeleted_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeUInt64(2, versionsDeleted_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(1, rowsDeleted_); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeUInt64Size(2, versionsDeleted_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse other = (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse) obj; + + boolean result = true; + result = result && (hasRowsDeleted() == other.hasRowsDeleted()); + if (hasRowsDeleted()) { + result = result && (getRowsDeleted() + == other.getRowsDeleted()); + } + result = result && (hasVersionsDeleted() == other.hasVersionsDeleted()); + if (hasVersionsDeleted()) { + result = result && (getVersionsDeleted() + == other.getVersionsDeleted()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasRowsDeleted()) { + hash = (37 * hash) + ROWSDELETED_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getRowsDeleted()); + } + if (hasVersionsDeleted()) { + hash = (37 * hash) + VERSIONSDELETED_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getVersionsDeleted()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + rowsDeleted_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + versionsDeleted_ = 0L; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse build() { + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse buildPartial() { + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse result = new org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.rowsDeleted_ = rowsDeleted_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.versionsDeleted_ = versionsDeleted_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse other) { + if (other == org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance()) return this; + if (other.hasRowsDeleted()) { + setRowsDeleted(other.getRowsDeleted()); + } + if (other.hasVersionsDeleted()) { + setVersionsDeleted(other.getVersionsDeleted()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasRowsDeleted()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + rowsDeleted_ = input.readUInt64(); + break; + } + case 16: { + bitField0_ |= 0x00000002; + versionsDeleted_ = input.readUInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required uint64 rowsDeleted = 1; + private long rowsDeleted_ ; + public boolean hasRowsDeleted() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getRowsDeleted() { + return rowsDeleted_; + } + public Builder setRowsDeleted(long value) { + bitField0_ |= 0x00000001; + rowsDeleted_ = value; + onChanged(); + return this; + } + public Builder clearRowsDeleted() { + bitField0_ = (bitField0_ & ~0x00000001); + rowsDeleted_ = 0L; + onChanged(); + return this; + } + + // optional uint64 versionsDeleted = 2; + private long versionsDeleted_ ; + public boolean hasVersionsDeleted() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public long getVersionsDeleted() { + return versionsDeleted_; + } + public Builder setVersionsDeleted(long value) { + bitField0_ |= 0x00000002; + versionsDeleted_ = value; + onChanged(); + return this; + } + public Builder clearVersionsDeleted() { + bitField0_ = (bitField0_ & ~0x00000002); + versionsDeleted_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:BulkDeleteResponse) + } + + static { + defaultInstance = new BulkDeleteResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:BulkDeleteResponse) + } + + public static abstract class BulkDeleteService + implements com.google.protobuf.Service { + protected BulkDeleteService() {} + + public interface Interface { + public abstract void delete( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, + com.google.protobuf.RpcCallback done); + + } + + public static com.google.protobuf.Service newReflectiveService( + final Interface impl) { + return new BulkDeleteService() { + @java.lang.Override + public void delete( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, + com.google.protobuf.RpcCallback done) { + impl.delete(controller, request, done); + } + + }; + } + + public static com.google.protobuf.BlockingService + newReflectiveBlockingService(final BlockingInterface impl) { + return new com.google.protobuf.BlockingService() { + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final com.google.protobuf.Message callBlockingMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request) + throws com.google.protobuf.ServiceException { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callBlockingMethod() given method descriptor for " + + "wrong service type."); + } + switch(method.getIndex()) { + case 0: + return impl.delete(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)request); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + }; + } + + public abstract void delete( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, + com.google.protobuf.RpcCallback done); + + public static final + com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.getDescriptor().getServices().get(0); + } + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final void callMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request, + com.google.protobuf.RpcCallback< + com.google.protobuf.Message> done) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callMethod() given method descriptor for wrong " + + "service type."); + } + switch(method.getIndex()) { + case 0: + this.delete(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public static Stub newStub( + com.google.protobuf.RpcChannel channel) { + return new Stub(channel); + } + + public static final class Stub extends org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService implements Interface { + private Stub(com.google.protobuf.RpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.RpcChannel channel; + + public com.google.protobuf.RpcChannel getChannel() { + return channel; + } + + public void delete( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance())); + } + } + + public static BlockingInterface newBlockingStub( + com.google.protobuf.BlockingRpcChannel channel) { + return new BlockingStub(channel); + } + + public interface BlockingInterface { + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse delete( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request) + throws com.google.protobuf.ServiceException; + } + + private static final class BlockingStub implements BlockingInterface { + private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.BlockingRpcChannel channel; + + public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse delete( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance()); + } + + } + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_BulkDeleteRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_BulkDeleteRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_BulkDeleteResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_BulkDeleteResponse_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\020BulkDelete.proto\032\014Client.proto\"\300\001\n\021Bul" + + "kDeleteRequest\022\023\n\004scan\030\001 \002(\0132\005.Scan\0221\n\nd" + + "eleteType\030\002 \002(\0162\035.BulkDeleteRequest.Dele" + + "teType\022\021\n\ttimestamp\030\003 \001(\004\022\024\n\014rowBatchSiz" + + "e\030\004 \002(\r\":\n\nDeleteType\022\007\n\003ROW\020\000\022\n\n\006FAMILY" + + "\020\001\022\n\n\006COLUMN\020\002\022\013\n\007VERSION\020\003\"B\n\022BulkDelet" + + "eResponse\022\023\n\013rowsDeleted\030\001 \002(\004\022\027\n\017versio" + + "nsDeleted\030\002 \001(\0042F\n\021BulkDeleteService\0221\n\006" + + "delete\022\022.BulkDeleteRequest\032\023.BulkDeleteR" + + "esponseBQ\n5org.apache.hadoop.hbase.copro", + "cessor.example.generatedB\020BulkDeleteProt" + + "osH\001\210\001\001\240\001\001" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_BulkDeleteRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_BulkDeleteRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_BulkDeleteRequest_descriptor, + new java.lang.String[] { "Scan", "DeleteType", "Timestamp", "RowBatchSize", }, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.class, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.Builder.class); + internal_static_BulkDeleteResponse_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_BulkDeleteResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_BulkDeleteResponse_descriptor, + new java.lang.String[] { "RowsDeleted", "VersionsDeleted", }, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder.class); + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + org.apache.hadoop.hbase.protobuf.generated.ClientProtos.getDescriptor(), + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java new file mode 100644 index 0000000..123bf3b --- /dev/null +++ hbase-examples/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java @@ -0,0 +1,1034 @@ +// Generated by the protocol buffer compiler. DO NOT EDIT! +// source: Examples.proto + +package org.apache.hadoop.hbase.coprocessor.example.generated; + +public final class ExampleProtos { + private ExampleProtos() {} + public static void registerAllExtensions( + com.google.protobuf.ExtensionRegistry registry) { + } + public interface CountRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + } + public static final class CountRequest extends + com.google.protobuf.GeneratedMessage + implements CountRequestOrBuilder { + // Use CountRequest.newBuilder() to construct. + private CountRequest(Builder builder) { + super(builder); + } + private CountRequest(boolean noInit) {} + + private static final CountRequest defaultInstance; + public static CountRequest getDefaultInstance() { + return defaultInstance; + } + + public CountRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_fieldAccessorTable; + } + + private void initFields() { + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest other = (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest) obj; + + boolean result = true; + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest build() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest buildPartial() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = new org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest(this); + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest other) { + if (other == org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance()) return this; + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + } + } + } + + + // @@protoc_insertion_point(builder_scope:CountRequest) + } + + static { + defaultInstance = new CountRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CountRequest) + } + + public interface CountResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required int64 count = 1 [default = 0]; + boolean hasCount(); + long getCount(); + } + public static final class CountResponse extends + com.google.protobuf.GeneratedMessage + implements CountResponseOrBuilder { + // Use CountResponse.newBuilder() to construct. + private CountResponse(Builder builder) { + super(builder); + } + private CountResponse(boolean noInit) {} + + private static final CountResponse defaultInstance; + public static CountResponse getDefaultInstance() { + return defaultInstance; + } + + public CountResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_fieldAccessorTable; + } + + private int bitField0_; + // required int64 count = 1 [default = 0]; + public static final int COUNT_FIELD_NUMBER = 1; + private long count_; + public boolean hasCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getCount() { + return count_; + } + + private void initFields() { + count_ = 0L; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasCount()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeInt64(1, count_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeInt64Size(1, count_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse other = (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) obj; + + boolean result = true; + result = result && (hasCount() == other.hasCount()); + if (hasCount()) { + result = result && (getCount() + == other.getCount()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasCount()) { + hash = (37 * hash) + COUNT_FIELD_NUMBER; + hash = (53 * hash) + hashLong(getCount()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + count_ = 0L; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse build() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse buildPartial() { + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = new org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.count_ = count_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) { + return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse other) { + if (other == org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()) return this; + if (other.hasCount()) { + setCount(other.getCount()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasCount()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + count_ = input.readInt64(); + break; + } + } + } + } + + private int bitField0_; + + // required int64 count = 1 [default = 0]; + private long count_ ; + public boolean hasCount() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public long getCount() { + return count_; + } + public Builder setCount(long value) { + bitField0_ |= 0x00000001; + count_ = value; + onChanged(); + return this; + } + public Builder clearCount() { + bitField0_ = (bitField0_ & ~0x00000001); + count_ = 0L; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:CountResponse) + } + + static { + defaultInstance = new CountResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:CountResponse) + } + + public static abstract class RowCountService + implements com.google.protobuf.Service { + protected RowCountService() {} + + public interface Interface { + public abstract void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + } + + public static com.google.protobuf.Service newReflectiveService( + final Interface impl) { + return new RowCountService() { + @java.lang.Override + public void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + impl.getRowCount(controller, request, done); + } + + @java.lang.Override + public void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + impl.getKeyValueCount(controller, request, done); + } + + }; + } + + public static com.google.protobuf.BlockingService + newReflectiveBlockingService(final BlockingInterface impl) { + return new com.google.protobuf.BlockingService() { + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final com.google.protobuf.Message callBlockingMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request) + throws com.google.protobuf.ServiceException { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callBlockingMethod() given method descriptor for " + + "wrong service type."); + } + switch(method.getIndex()) { + case 0: + return impl.getRowCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request); + case 1: + return impl.getKeyValueCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + }; + } + + public abstract void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done); + + public static final + com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptor() { + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.getDescriptor().getServices().get(0); + } + public final com.google.protobuf.Descriptors.ServiceDescriptor + getDescriptorForType() { + return getDescriptor(); + } + + public final void callMethod( + com.google.protobuf.Descriptors.MethodDescriptor method, + com.google.protobuf.RpcController controller, + com.google.protobuf.Message request, + com.google.protobuf.RpcCallback< + com.google.protobuf.Message> done) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.callMethod() given method descriptor for wrong " + + "service type."); + } + switch(method.getIndex()) { + case 0: + this.getRowCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 1: + this.getKeyValueCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getRequestPrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getRequestPrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public final com.google.protobuf.Message + getResponsePrototype( + com.google.protobuf.Descriptors.MethodDescriptor method) { + if (method.getService() != getDescriptor()) { + throw new java.lang.IllegalArgumentException( + "Service.getResponsePrototype() given method " + + "descriptor for wrong service type."); + } + switch(method.getIndex()) { + case 0: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + case 1: + return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); + default: + throw new java.lang.AssertionError("Can't get here."); + } + } + + public static Stub newStub( + com.google.protobuf.RpcChannel channel) { + return new Stub(channel); + } + + public static final class Stub extends org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.RowCountService implements Interface { + private Stub(com.google.protobuf.RpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.RpcChannel channel; + + public com.google.protobuf.RpcChannel getChannel() { + return channel; + } + + public void getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance())); + } + + public void getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance())); + } + } + + public static BlockingInterface newBlockingStub( + com.google.protobuf.BlockingRpcChannel channel) { + return new BlockingStub(channel); + } + + public interface BlockingInterface { + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException; + } + + private static final class BlockingStub implements BlockingInterface { + private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { + this.channel = channel; + } + + private final com.google.protobuf.BlockingRpcChannel channel; + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getRowCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(0), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()); + } + + + public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getKeyValueCount( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(1), + controller, + request, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()); + } + + } + } + + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CountRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CountRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_CountResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_CountResponse_fieldAccessorTable; + + public static com.google.protobuf.Descriptors.FileDescriptor + getDescriptor() { + return descriptor; + } + private static com.google.protobuf.Descriptors.FileDescriptor + descriptor; + static { + java.lang.String[] descriptorData = { + "\n\016Examples.proto\"\016\n\014CountRequest\"!\n\rCoun" + + "tResponse\022\020\n\005count\030\001 \002(\003:\00102r\n\017RowCountS" + + "ervice\022,\n\013getRowCount\022\r.CountRequest\032\016.C" + + "ountResponse\0221\n\020getKeyValueCount\022\r.Count" + + "Request\032\016.CountResponseBN\n5org.apache.ha" + + "doop.hbase.coprocessor.example.generated" + + "B\rExampleProtosH\001\210\001\001\240\001\001" + }; + com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = + new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { + public com.google.protobuf.ExtensionRegistry assignDescriptors( + com.google.protobuf.Descriptors.FileDescriptor root) { + descriptor = root; + internal_static_CountRequest_descriptor = + getDescriptor().getMessageTypes().get(0); + internal_static_CountRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CountRequest_descriptor, + new java.lang.String[] { }, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.Builder.class); + internal_static_CountResponse_descriptor = + getDescriptor().getMessageTypes().get(1); + internal_static_CountResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_CountResponse_descriptor, + new java.lang.String[] { "Count", }, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, + org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.Builder.class); + return null; + } + }; + com.google.protobuf.Descriptors.FileDescriptor + .internalBuildGeneratedFileFrom(descriptorData, + new com.google.protobuf.Descriptors.FileDescriptor[] { + }, assigner); + } + + // @@protoc_insertion_point(outer_class_scope) +} diff --git hbase-examples/src/main/protobuf/BulkDelete.proto hbase-examples/src/main/protobuf/BulkDelete.proto new file mode 100644 index 0000000..a0e56dd --- /dev/null +++ hbase-examples/src/main/protobuf/BulkDelete.proto @@ -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. + */ + +option java_package = "org.apache.hadoop.hbase.coprocessor.example.generated"; +option java_outer_classname = "BulkDeleteProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +import "Client.proto"; + +message BulkDeleteRequest { + required Scan scan = 1; + required DeleteType deleteType = 2; + optional uint64 timestamp = 3; + required uint32 rowBatchSize = 4; + + enum DeleteType { + ROW = 0; + FAMILY = 1; + COLUMN = 2; + VERSION = 3; + } +} + +message BulkDeleteResponse { + required uint64 rowsDeleted = 1; + optional uint64 versionsDeleted = 2; +} + +service BulkDeleteService { + rpc delete(BulkDeleteRequest) + returns (BulkDeleteResponse); +} \ No newline at end of file diff --git hbase-examples/src/main/protobuf/Examples.proto hbase-examples/src/main/protobuf/Examples.proto new file mode 100644 index 0000000..24fbad6 --- /dev/null +++ hbase-examples/src/main/protobuf/Examples.proto @@ -0,0 +1,37 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.hbase.coprocessor.example.generated"; +option java_outer_classname = "ExampleProtos"; +option java_generic_services = true; +option java_generate_equals_and_hash = true; +option optimize_for = SPEED; + +message CountRequest { +} + +message CountResponse { + required int64 count = 1 [default = 0]; +} + +service RowCountService { + rpc getRowCount(CountRequest) + returns (CountResponse); + rpc getKeyValueCount(CountRequest) + returns (CountResponse); +} \ No newline at end of file diff --git hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java new file mode 100644 index 0000000..e8faf30 --- /dev/null +++ hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java @@ -0,0 +1,434 @@ +/* + * 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.coprocessor.example; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.Map; + +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HConstants; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.KeyValue; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.client.Scan; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.Builder; +import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType; +import org.apache.hadoop.hbase.filter.FilterList; +import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; +import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; +import org.apache.hadoop.hbase.filter.FilterList.Operator; +import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; +import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.protobuf.ProtobufUtil; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(MediumTests.class) +public class TestBulkDeleteProtocol { + private static final byte[] FAMILY1 = Bytes.toBytes("cf1"); + private static final byte[] FAMILY2 = Bytes.toBytes("cf2"); + private static final byte[] QUALIFIER1 = Bytes.toBytes("c1"); + private static final byte[] QUALIFIER2 = Bytes.toBytes("c2"); + private static final byte[] QUALIFIER3 = Bytes.toBytes("c3"); + private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + + @BeforeClass + public static void setupBeforeClass() throws Exception { + TEST_UTIL.getConfiguration().set(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY, + BulkDeleteEndpoint.class.getName()); + TEST_UTIL.startMiniCluster(2); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testBulkDeleteEndpoint() throws Throwable { + byte[] tableName = Bytes.toBytes("testBulkDeleteEndpoint"); + HTable ht = createTable(tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + byte[] rowkey = Bytes.toBytes(j); + puts.add(createPut(rowkey, "v1")); + } + ht.put(puts); + // Deleting all the rows. + long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, new Scan(), 5, DeleteType.ROW, null); + assertEquals(100, noOfRowsDeleted); + + int rows = 0; + for (Result result : ht.getScanner(new Scan())) { + rows++; + } + assertEquals(0, rows); + } + + @Test + public void testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion() + throws Throwable { + byte[] tableName = Bytes + .toBytes("testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion"); + HTable ht = createTable(tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + byte[] rowkey = Bytes.toBytes(j); + puts.add(createPut(rowkey, "v1")); + } + ht.put(puts); + // Deleting all the rows. + long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, new Scan(), 10, DeleteType.ROW, null); + assertEquals(100, noOfRowsDeleted); + + int rows = 0; + for (Result result : ht.getScanner(new Scan())) { + rows++; + } + assertEquals(0, rows); + } + + private long invokeBulkDeleteProtocol(byte[] tableName, final Scan scan, final int rowBatchSize, + final DeleteType deleteType, final Long timeStamp) throws Throwable { + HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + long noOfDeletedRows = 0L; + Batch.Call callable = + new Batch.Call() { + ServerRpcController controller = new ServerRpcController(); + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback(); + + public BulkDeleteResponse call(BulkDeleteService service) throws IOException { + Builder builder = BulkDeleteRequest.newBuilder(); + builder.setScan(ProtobufUtil.toScan(scan)); + builder.setDeleteType(deleteType); + builder.setRowBatchSize(rowBatchSize); + if (timeStamp != null) { + builder.setTimestamp(timeStamp); + } + service.delete(controller, builder.build(), rpcCallback); + return rpcCallback.get(); + } + }; + Map result = ht.coprocessorService(BulkDeleteService.class, scan + .getStartRow(), scan.getStopRow(), callable); + for (BulkDeleteResponse response : result.values()) { + noOfDeletedRows += response.getRowsDeleted(); + } + return noOfDeletedRows; + } + + @Test + public void testBulkDeleteWithConditionBasedDelete() throws Throwable { + byte[] tableName = Bytes.toBytes("testBulkDeleteWithConditionBasedDelete"); + HTable ht = createTable(tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + byte[] rowkey = Bytes.toBytes(j); + String value = (j % 10 == 0) ? "v1" : "v2"; + puts.add(createPut(rowkey, value)); + } + ht.put(puts); + Scan scan = new Scan(); + FilterList fl = new FilterList(Operator.MUST_PASS_ALL); + SingleColumnValueFilter scvf = new SingleColumnValueFilter(FAMILY1, QUALIFIER3, + CompareOp.EQUAL, Bytes.toBytes("v1")); + // fl.addFilter(new FirstKeyOnlyFilter()); + fl.addFilter(scvf); + scan.setFilter(fl); + // Deleting all the rows where cf1:c1=v1 + long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.ROW, null); + assertEquals(10, noOfRowsDeleted); + + int rows = 0; + for (Result result : ht.getScanner(new Scan())) { + rows++; + } + assertEquals(90, rows); + } + + @Test + public void testBulkDeleteColumn() throws Throwable { + byte[] tableName = Bytes.toBytes("testBulkDeleteColumn"); + HTable ht = createTable(tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + byte[] rowkey = Bytes.toBytes(j); + String value = (j % 10 == 0) ? "v1" : "v2"; + puts.add(createPut(rowkey, value)); + } + ht.put(puts); + Scan scan = new Scan(); + scan.addColumn(FAMILY1, QUALIFIER2); + // Delete the column cf1:col2 + long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.COLUMN, null); + assertEquals(100, noOfRowsDeleted); + + int rows = 0; + for (Result result : ht.getScanner(new Scan())) { + assertEquals(2, result.getFamilyMap(FAMILY1).size()); + assertTrue(result.getColumn(FAMILY1, QUALIFIER2).isEmpty()); + assertEquals(1, result.getColumn(FAMILY1, QUALIFIER1).size()); + assertEquals(1, result.getColumn(FAMILY1, QUALIFIER3).size()); + rows++; + } + assertEquals(100, rows); + } + + @Test + public void testBulkDeleteFamily() throws Throwable { + byte[] tableName = Bytes.toBytes("testBulkDeleteFamily"); + HTableDescriptor htd = new HTableDescriptor(tableName); + htd.addFamily(new HColumnDescriptor(FAMILY1)); + htd.addFamily(new HColumnDescriptor(FAMILY2)); + TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); + HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + Put put = new Put(Bytes.toBytes(j)); + put.add(FAMILY1, QUALIFIER1, "v1".getBytes()); + put.add(FAMILY2, QUALIFIER2, "v2".getBytes()); + puts.add(put); + } + ht.put(puts); + Scan scan = new Scan(); + scan.addFamily(FAMILY1); + // Delete the column family cf1 + long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.FAMILY, null); + assertEquals(100, noOfRowsDeleted); + int rows = 0; + for (Result result : ht.getScanner(new Scan())) { + assertTrue(result.getFamilyMap(FAMILY1).isEmpty()); + assertEquals(1, result.getColumn(FAMILY2, QUALIFIER2).size()); + rows++; + } + assertEquals(100, rows); + } + + @Test + public void testBulkDeleteColumnVersion() throws Throwable { + byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersion"); + HTable ht = createTable(tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + Put put = new Put(Bytes.toBytes(j)); + byte[] value = "v1".getBytes(); + put.add(FAMILY1, QUALIFIER1, 1234L, value); + put.add(FAMILY1, QUALIFIER2, 1234L, value); + put.add(FAMILY1, QUALIFIER3, 1234L, value); + // Latest version values + value = "v2".getBytes(); + put.add(FAMILY1, QUALIFIER1, value); + put.add(FAMILY1, QUALIFIER2, value); + put.add(FAMILY1, QUALIFIER3, value); + put.add(FAMILY1, null, value); + puts.add(put); + } + ht.put(puts); + Scan scan = new Scan(); + scan.addFamily(FAMILY1); + // Delete the latest version values of all the columns in family cf1. + long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.VERSION, + HConstants.LATEST_TIMESTAMP); + assertEquals(100, noOfRowsDeleted); + int rows = 0; + scan = new Scan(); + scan.setMaxVersions(); + for (Result result : ht.getScanner(scan)) { + assertEquals(3, result.getFamilyMap(FAMILY1).size()); + List column = result.getColumn(FAMILY1, QUALIFIER1); + assertEquals(1, column.size()); + assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue())); + + column = result.getColumn(FAMILY1, QUALIFIER2); + assertEquals(1, column.size()); + assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue())); + + column = result.getColumn(FAMILY1, QUALIFIER3); + assertEquals(1, column.size()); + assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue())); + rows++; + } + assertEquals(100, rows); + } + + @Test + public void testBulkDeleteColumnVersionBasedOnTS() throws Throwable { + byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersionBasedOnTS"); + HTable ht = createTable(tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + Put put = new Put(Bytes.toBytes(j)); + // TS = 1000L + byte[] value = "v1".getBytes(); + put.add(FAMILY1, QUALIFIER1, 1000L, value); + put.add(FAMILY1, QUALIFIER2, 1000L, value); + put.add(FAMILY1, QUALIFIER3, 1000L, value); + // TS = 1234L + value = "v2".getBytes(); + put.add(FAMILY1, QUALIFIER1, 1234L, value); + put.add(FAMILY1, QUALIFIER2, 1234L, value); + put.add(FAMILY1, QUALIFIER3, 1234L, value); + // Latest version values + value = "v3".getBytes(); + put.add(FAMILY1, QUALIFIER1, value); + put.add(FAMILY1, QUALIFIER2, value); + put.add(FAMILY1, QUALIFIER3, value); + puts.add(put); + } + ht.put(puts); + Scan scan = new Scan(); + scan.addColumn(FAMILY1, QUALIFIER3); + // Delete the column cf1:c3's one version at TS=1234 + long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.VERSION, 1234L); + assertEquals(100, noOfRowsDeleted); + int rows = 0; + scan = new Scan(); + scan.setMaxVersions(); + for (Result result : ht.getScanner(scan)) { + assertEquals(3, result.getFamilyMap(FAMILY1).size()); + assertEquals(3, result.getColumn(FAMILY1, QUALIFIER1).size()); + assertEquals(3, result.getColumn(FAMILY1, QUALIFIER2).size()); + List column = result.getColumn(FAMILY1, QUALIFIER3); + assertEquals(2, column.size()); + assertTrue(Bytes.equals("v3".getBytes(), column.get(0).getValue())); + assertTrue(Bytes.equals("v1".getBytes(), column.get(1).getValue())); + rows++; + } + assertEquals(100, rows); + } + + @Test + public void testBulkDeleteWithNumberOfVersions() throws Throwable { + byte[] tableName = Bytes.toBytes("testBulkDeleteWithNumberOfVersions"); + HTable ht = createTable(tableName); + List puts = new ArrayList(100); + for (int j = 0; j < 100; j++) { + Put put = new Put(Bytes.toBytes(j)); + // TS = 1000L + byte[] value = "v1".getBytes(); + put.add(FAMILY1, QUALIFIER1, 1000L, value); + put.add(FAMILY1, QUALIFIER2, 1000L, value); + put.add(FAMILY1, QUALIFIER3, 1000L, value); + // TS = 1234L + value = "v2".getBytes(); + put.add(FAMILY1, QUALIFIER1, 1234L, value); + put.add(FAMILY1, QUALIFIER2, 1234L, value); + put.add(FAMILY1, QUALIFIER3, 1234L, value); + // TS = 2000L + value = "v3".getBytes(); + put.add(FAMILY1, QUALIFIER1, 2000L, value); + put.add(FAMILY1, QUALIFIER2, 2000L, value); + put.add(FAMILY1, QUALIFIER3, 2000L, value); + // Latest version values + value = "v4".getBytes(); + put.add(FAMILY1, QUALIFIER1, value); + put.add(FAMILY1, QUALIFIER2, value); + put.add(FAMILY1, QUALIFIER3, value); + puts.add(put); + } + ht.put(puts); + + // Delete all the versions of columns cf1:c1 and cf1:c2 falling with the time range + // [1000,2000) + final Scan scan = new Scan(); + scan.addColumn(FAMILY1, QUALIFIER1); + scan.addColumn(FAMILY1, QUALIFIER2); + scan.setTimeRange(1000L, 2000L); + scan.setMaxVersions(); + + long noOfDeletedRows = 0L; + long noOfVersionsDeleted = 0L; + Batch.Call callable = + new Batch.Call() { + ServerRpcController controller = new ServerRpcController(); + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback(); + + public BulkDeleteResponse call(BulkDeleteService service) throws IOException { + Builder builder = BulkDeleteRequest.newBuilder(); + builder.setScan(ProtobufUtil.toScan(scan)); + builder.setDeleteType(DeleteType.VERSION); + builder.setRowBatchSize(500); + service.delete(controller, builder.build(), rpcCallback); + return rpcCallback.get(); + } + }; + Map result = ht.coprocessorService(BulkDeleteService.class, scan + .getStartRow(), scan.getStopRow(), callable); + for (BulkDeleteResponse response : result.values()) { + noOfDeletedRows += response.getRowsDeleted(); + noOfVersionsDeleted += response.getVersionsDeleted(); + } + assertEquals(100, noOfDeletedRows); + assertEquals(400, noOfVersionsDeleted); + + int rows = 0; + Scan scan1 = new Scan(); + scan1.setMaxVersions(); + for (Result res : ht.getScanner(scan1)) { + assertEquals(3, res.getFamilyMap(FAMILY1).size()); + List column = res.getColumn(FAMILY1, QUALIFIER1); + assertEquals(2, column.size()); + assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue())); + assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue())); + column = res.getColumn(FAMILY1, QUALIFIER2); + assertEquals(2, column.size()); + assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue())); + assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue())); + assertEquals(4, res.getColumn(FAMILY1, QUALIFIER3).size()); + rows++; + } + assertEquals(100, rows); + } + + private HTable createTable(byte[] tableName) throws IOException { + HTableDescriptor htd = new HTableDescriptor(tableName); + HColumnDescriptor hcd = new HColumnDescriptor(FAMILY1); + hcd.setMaxVersions(10);// Just setting 10 as I am not testing with more than 10 versions here + htd.addFamily(hcd); + TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); + HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); + return ht; + } + + private Put createPut(byte[] rowkey, String value) throws IOException { + Put put = new Put(rowkey); + put.add(FAMILY1, QUALIFIER1, value.getBytes()); + put.add(FAMILY1, QUALIFIER2, value.getBytes()); + put.add(FAMILY1, QUALIFIER3, value.getBytes()); + return put; + } +} \ No newline at end of file diff --git hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java new file mode 100644 index 0000000..e582997 --- /dev/null +++ hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java @@ -0,0 +1,108 @@ +/* + * 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.coprocessor.example; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.coprocessor.Batch; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; +import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; +import org.apache.hadoop.hbase.ipc.ServerRpcController; +import org.apache.hadoop.hbase.util.Bytes; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +import static junit.framework.Assert.*; + +/** + * Test case demonstrating client interactions with the {@link RowCountEndpoint} + * sample coprocessor Service implementation. + */ +@Category(MediumTests.class) +public class TestRowCountEndpoint { + private static final byte[] TEST_TABLE = Bytes.toBytes("testrowcounter"); + private static final byte[] TEST_FAMILY = Bytes.toBytes("f"); + private static final byte[] TEST_COLUMN = Bytes.toBytes("col"); + + private static HBaseTestingUtility TEST_UTIL = null; + private static Configuration CONF = null; + + @BeforeClass + public static void setupBeforeClass() throws Exception { + TEST_UTIL = new HBaseTestingUtility(); + CONF = TEST_UTIL.getConfiguration(); + CONF.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, + RowCountEndpoint.class.getName()); + + TEST_UTIL.startMiniCluster(); + TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testEndpoint() throws Throwable { + HTable table = new HTable(CONF, TEST_TABLE); + + // insert some test rows + for (int i=0; i<5; i++) { + byte[] iBytes = Bytes.toBytes(i); + Put p = new Put(iBytes); + p.add(TEST_FAMILY, TEST_COLUMN, iBytes); + table.put(p); + } + + final ExampleProtos.CountRequest request = ExampleProtos.CountRequest.getDefaultInstance(); + Map results = table.coprocessorService(ExampleProtos.RowCountService.class, + null, null, + new Batch.Call() { + public Long call(ExampleProtos.RowCountService counter) throws IOException { + ServerRpcController controller = new ServerRpcController(); + BlockingRpcCallback rpcCallback = + new BlockingRpcCallback(); + counter.getRowCount(controller, request, rpcCallback); + ExampleProtos.CountResponse response = rpcCallback.get(); + if (controller.failedOnException()) { + throw controller.getFailedOn(); + } + return (response != null && response.hasCount()) ? response.getCount() : 0; + } + }); + // should be one region with results + assertEquals(1, results.size()); + Iterator iter = results.values().iterator(); + Long val = iter.next(); + assertNotNull(val); + assertEquals(5l, val.longValue()); + } + +} diff --git hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java new file mode 100644 index 0000000..83ac40c --- /dev/null +++ hbase-examples/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java @@ -0,0 +1,131 @@ +/* + * Copyright The Apache Software Foundation + * + * 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.coprocessor.example; + +import static org.junit.Assert.assertEquals; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.HBaseTestingUtility; +import org.apache.hadoop.hbase.HColumnDescriptor; +import org.apache.hadoop.hbase.HTableDescriptor; +import org.apache.hadoop.hbase.MediumTests; +import org.apache.hadoop.hbase.client.Get; +import org.apache.hadoop.hbase.client.HConnectionManager; +import org.apache.hadoop.hbase.client.HTable; +import org.apache.hadoop.hbase.client.Put; +import org.apache.hadoop.hbase.client.Result; +import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.zookeeper.ZooKeeper; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +@Category(MediumTests.class) +public class TestZooKeeperScanPolicyObserver { + private static final Log LOG = LogFactory.getLog(TestZooKeeperScanPolicyObserver.class); + private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + private static final byte[] F = Bytes.toBytes("fam"); + private static final byte[] Q = Bytes.toBytes("qual"); + private static final byte[] R = Bytes.toBytes("row"); + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + // Test we can first start the ZK cluster by itself + Configuration conf = TEST_UTIL.getConfiguration(); + conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, + ZooKeeperScanPolicyObserver.class.getName()); + TEST_UTIL.startMiniZKCluster(); + TEST_UTIL.startMiniCluster(); + } + + @AfterClass + public static void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testScanPolicyObserver() throws Exception { + byte[] tableName = Bytes.toBytes("testScanPolicyObserver"); + HTableDescriptor desc = new HTableDescriptor(tableName); + HColumnDescriptor hcd = new HColumnDescriptor(F) + .setMaxVersions(10) + .setTimeToLive(1); + desc.addFamily(hcd); + TEST_UTIL.getHBaseAdmin().createTable(desc); + HTable t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); + long now = EnvironmentEdgeManager.currentTimeMillis(); + + ZooKeeperWatcher zkw = HConnectionManager.getConnection(TEST_UTIL.getConfiguration()) + .getZooKeeperWatcher(); + ZooKeeper zk = zkw.getRecoverableZooKeeper().getZooKeeper(); + ZKUtil.createWithParents(zkw, ZooKeeperScanPolicyObserver.node); + // let's say test last backup was 1h ago + // using plain ZK here, because RecoverableZooKeeper add extra encoding to the data + zk.setData(ZooKeeperScanPolicyObserver.node, Bytes.toBytes(now - 3600*1000), -1); + + LOG.debug("Set time: "+Bytes.toLong(Bytes.toBytes(now - 3600*1000))); + + // sleep for 1s to give the ZK change a chance to reach the watcher in the observer. + // TODO: Better to wait for the data to be propagated + Thread.sleep(1000); + + long ts = now - 2000; + Put p = new Put(R); + p.add(F, Q, ts, Q); + t.put(p); + p = new Put(R); + p.add(F, Q, ts+1, Q); + t.put(p); + + // these two should be expired but for the override + // (their ts was 2s in the past) + Get g = new Get(R); + g.setMaxVersions(10); + Result r = t.get(g); + // still there? + assertEquals(2, r.size()); + + TEST_UTIL.flush(tableName); + TEST_UTIL.compact(tableName, true); + + g = new Get(R); + g.setMaxVersions(10); + r = t.get(g); + // still there? + assertEquals(2, r.size()); + zk.setData(ZooKeeperScanPolicyObserver.node, Bytes.toBytes(now), -1); + LOG.debug("Set time: "+now); + + TEST_UTIL.compact(tableName, true); + + g = new Get(R); + g.setMaxVersions(10); + r = t.get(g); + // should be gone now + assertEquals(0, r.size()); + t.close(); + } +} diff --git hbase-protocol/src/main/protobuf/BulkDelete.proto hbase-protocol/src/main/protobuf/BulkDelete.proto deleted file mode 100644 index a0e56dd..0000000 --- hbase-protocol/src/main/protobuf/BulkDelete.proto +++ /dev/null @@ -1,49 +0,0 @@ -/** - * 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. - */ - -option java_package = "org.apache.hadoop.hbase.coprocessor.example.generated"; -option java_outer_classname = "BulkDeleteProtos"; -option java_generic_services = true; -option java_generate_equals_and_hash = true; -option optimize_for = SPEED; - -import "Client.proto"; - -message BulkDeleteRequest { - required Scan scan = 1; - required DeleteType deleteType = 2; - optional uint64 timestamp = 3; - required uint32 rowBatchSize = 4; - - enum DeleteType { - ROW = 0; - FAMILY = 1; - COLUMN = 2; - VERSION = 3; - } -} - -message BulkDeleteResponse { - required uint64 rowsDeleted = 1; - optional uint64 versionsDeleted = 2; -} - -service BulkDeleteService { - rpc delete(BulkDeleteRequest) - returns (BulkDeleteResponse); -} \ No newline at end of file diff --git hbase-protocol/src/main/protobuf/Examples.proto hbase-protocol/src/main/protobuf/Examples.proto deleted file mode 100644 index 24fbad6..0000000 --- hbase-protocol/src/main/protobuf/Examples.proto +++ /dev/null @@ -1,37 +0,0 @@ -/** - * 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. - */ - -option java_package = "org.apache.hadoop.hbase.coprocessor.example.generated"; -option java_outer_classname = "ExampleProtos"; -option java_generic_services = true; -option java_generate_equals_and_hash = true; -option optimize_for = SPEED; - -message CountRequest { -} - -message CountResponse { - required int64 count = 1 [default = 0]; -} - -service RowCountService { - rpc getRowCount(CountRequest) - returns (CountResponse); - rpc getKeyValueCount(CountRequest) - returns (CountResponse); -} \ No newline at end of file diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java deleted file mode 100644 index 2bcdd58..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/BulkDeleteEndpoint.java +++ /dev/null @@ -1,290 +0,0 @@ -/* - * 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.coprocessor.example; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.TreeSet; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.Coprocessor; -import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.HConstants.OperationStatusCode; -import org.apache.hadoop.hbase.client.Delete; -import org.apache.hadoop.hbase.client.Mutation; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.coprocessor.CoprocessorException; -import org.apache.hadoop.hbase.coprocessor.CoprocessorService; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder; -import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.regionserver.HRegion; -import org.apache.hadoop.hbase.regionserver.OperationStatus; -import org.apache.hadoop.hbase.regionserver.RegionScanner; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.Pair; - -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; - -/** - * Defines a protocol to delete data in bulk based on a scan. The scan can be range scan or with - * conditions(filters) etc.This can be used to delete rows, column family(s), column qualifier(s) - * or version(s) of columns.When delete type is FAMILY or COLUMN, which all family(s) or column(s) - * getting deleted will be determined by the Scan. Scan need to select all the families/qualifiers - * which need to be deleted.When delete type is VERSION, Which column(s) and version(s) to be - * deleted will be determined by the Scan. Scan need to select all the qualifiers and its versions - * which needs to be deleted.When a timestamp is passed only one version at that timestamp will be - * deleted(even if Scan fetches many versions). When timestamp passed as null, all the versions - * which the Scan selects will get deleted. - * - *
Example:
- * Scan scan = new Scan();
- * // set scan properties(rowkey range, filters, timerange etc).
- * HTable ht = ...;
- * long noOfDeletedRows = 0L;
- * Batch.Call<BulkDeleteService, BulkDeleteResponse> callable = 
- *     new Batch.Call<BulkDeleteService, BulkDeleteResponse>() {
- *   ServerRpcController controller = new ServerRpcController();
- *   BlockingRpcCallback<BulkDeleteResponse> rpcCallback = 
- *     new BlockingRpcCallback<BulkDeleteResponse>();
- *
- *   public BulkDeleteResponse call(BulkDeleteService service) throws IOException {
- *     Builder builder = BulkDeleteRequest.newBuilder();
- *     builder.setScan(ProtobufUtil.toScan(scan));
- *     builder.setDeleteType(DeleteType.VERSION);
- *     builder.setRowBatchSize(rowBatchSize);
- *     // Set optional timestamp if needed
- *     builder.setTimestamp(timeStamp);
- *     service.delete(controller, builder.build(), rpcCallback);
- *     return rpcCallback.get();
- *   }
- * };
- * Map<byte[], BulkDeleteResponse> result = ht.coprocessorService(BulkDeleteService.class, scan
- *     .getStartRow(), scan.getStopRow(), callable);
- * for (BulkDeleteResponse response : result.values()) {
- *   noOfDeletedRows += response.getRowsDeleted();
- * }
- * 
- */ -public class BulkDeleteEndpoint extends BulkDeleteService implements CoprocessorService, - Coprocessor { - private static final String NO_OF_VERSIONS_TO_DELETE = "noOfVersionsToDelete"; - private static final Log LOG = LogFactory.getLog(BulkDeleteEndpoint.class); - - private RegionCoprocessorEnvironment env; - - @Override - public Service getService() { - return this; - } - - @Override - public void delete(RpcController controller, BulkDeleteRequest request, - RpcCallback done) { - long totalRowsDeleted = 0L; - long totalVersionsDeleted = 0L; - HRegion region = env.getRegion(); - int rowBatchSize = request.getRowBatchSize(); - Long timestamp = null; - if (request.hasTimestamp()) { - timestamp = request.getTimestamp(); - } - DeleteType deleteType = request.getDeleteType(); - boolean hasMore = true; - RegionScanner scanner = null; - try { - Scan scan = ProtobufUtil.toScan(request.getScan()); - if (scan.getFilter() == null && deleteType == DeleteType.ROW) { - // What we need is just the rowkeys. So only 1st KV from any row is enough. - // Only when it is a row delete, we can apply this filter. - // In other types we rely on the scan to know which all columns to be deleted. - scan.setFilter(new FirstKeyOnlyFilter()); - } - // Here by assume that the scan is perfect with the appropriate - // filter and having necessary column(s). - scanner = region.getScanner(scan); - while (hasMore) { - List> deleteRows = new ArrayList>(rowBatchSize); - for (int i = 0; i < rowBatchSize; i++) { - List results = new ArrayList(); - hasMore = scanner.next(results); - if (results.size() > 0) { - deleteRows.add(results); - } - if (!hasMore) { - // There are no more rows. - break; - } - } - if (deleteRows.size() > 0) { - Pair[] deleteWithLockArr = new Pair[deleteRows.size()]; - int i = 0; - for (List deleteRow : deleteRows) { - Delete delete = createDeleteMutation(deleteRow, deleteType, timestamp); - deleteWithLockArr[i++] = new Pair(delete, null); - } - OperationStatus[] opStatus = region.batchMutate(deleteWithLockArr); - for (i = 0; i < opStatus.length; i++) { - if (opStatus[i].getOperationStatusCode() != OperationStatusCode.SUCCESS) { - break; - } - totalRowsDeleted++; - if (deleteType == DeleteType.VERSION) { - byte[] versionsDeleted = deleteWithLockArr[i].getFirst().getAttribute( - NO_OF_VERSIONS_TO_DELETE); - if (versionsDeleted != null) { - totalVersionsDeleted += Bytes.toInt(versionsDeleted); - } - } - } - } - } - } catch (IOException ioe) { - LOG.error(ioe); - // Call ServerRpcController#getFailedOn() to retrieve this IOException at client side. - ResponseConverter.setControllerException(controller, ioe); - } finally { - if (scanner != null) { - try { - scanner.close(); - } catch (IOException ioe) { - LOG.error(ioe); - } - } - } - Builder responseBuilder = BulkDeleteResponse.newBuilder(); - responseBuilder.setRowsDeleted(totalRowsDeleted); - if (deleteType == DeleteType.VERSION) { - responseBuilder.setVersionsDeleted(totalVersionsDeleted); - } - BulkDeleteResponse result = responseBuilder.build(); - done.run(result); - } - - private Delete createDeleteMutation(List deleteRow, DeleteType deleteType, - Long timestamp) { - long ts; - if (timestamp == null) { - ts = HConstants.LATEST_TIMESTAMP; - } else { - ts = timestamp; - } - // We just need the rowkey. Get it from 1st KV. - byte[] row = deleteRow.get(0).getRow(); - Delete delete = new Delete(row, ts, null); - if (deleteType == DeleteType.FAMILY) { - Set families = new TreeSet(Bytes.BYTES_COMPARATOR); - for (KeyValue kv : deleteRow) { - if (families.add(kv.getFamily())) { - delete.deleteFamily(kv.getFamily(), ts); - } - } - } else if (deleteType == DeleteType.COLUMN) { - Set columns = new HashSet(); - for (KeyValue kv : deleteRow) { - Column column = new Column(kv.getFamily(), kv.getQualifier()); - if (columns.add(column)) { - // Making deleteColumns() calls more than once for the same cf:qualifier is not correct - // Every call to deleteColumns() will add a new KV to the familymap which will finally - // get written to the memstore as part of delete(). - delete.deleteColumns(column.family, column.qualifier, ts); - } - } - } else if (deleteType == DeleteType.VERSION) { - // When some timestamp was passed to the delete() call only one version of the column (with - // given timestamp) will be deleted. If no timestamp passed, it will delete N versions. - // How many versions will get deleted depends on the Scan being passed. All the KVs that - // the scan fetched will get deleted. - int noOfVersionsToDelete = 0; - if (timestamp == null) { - for (KeyValue kv : deleteRow) { - delete.deleteColumn(kv.getFamily(), kv.getQualifier(), kv.getTimestamp()); - noOfVersionsToDelete++; - } - } else { - Set columns = new HashSet(); - for (KeyValue kv : deleteRow) { - Column column = new Column(kv.getFamily(), kv.getQualifier()); - // Only one version of particular column getting deleted. - if (columns.add(column)) { - delete.deleteColumn(column.family, column.qualifier, ts); - noOfVersionsToDelete++; - } - } - } - delete.setAttribute(NO_OF_VERSIONS_TO_DELETE, Bytes.toBytes(noOfVersionsToDelete)); - } - return delete; - } - - private static class Column { - private byte[] family; - private byte[] qualifier; - - public Column(byte[] family, byte[] qualifier) { - this.family = family; - this.qualifier = qualifier; - } - - @Override - public boolean equals(Object other) { - if (!(other instanceof Column)) { - return false; - } - Column column = (Column) other; - return Bytes.equals(this.family, column.family) - && Bytes.equals(this.qualifier, column.qualifier); - } - - @Override - public int hashCode() { - int h = 31; - h = h + 13 * Bytes.hashCode(this.family); - h = h + 13 * Bytes.hashCode(this.qualifier); - return h; - } - } - - @Override - public void start(CoprocessorEnvironment env) throws IOException { - if (env instanceof RegionCoprocessorEnvironment) { - this.env = (RegionCoprocessorEnvironment) env; - } else { - throw new CoprocessorException("Must be loaded on a table region!"); - } - } - - @Override - public void stop(CoprocessorEnvironment env) throws IOException { - // nothing to do - } -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java deleted file mode 100644 index 465b4e0..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/RowCountEndpoint.java +++ /dev/null @@ -1,164 +0,0 @@ -/* - * 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.coprocessor.example; - -import com.google.protobuf.RpcCallback; -import com.google.protobuf.RpcController; -import com.google.protobuf.Service; -import org.apache.hadoop.hbase.Coprocessor; -import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.coprocessor.CoprocessorException; -import org.apache.hadoop.hbase.coprocessor.CoprocessorService; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; -import org.apache.hadoop.hbase.filter.FirstKeyOnlyFilter; -import org.apache.hadoop.hbase.protobuf.ResponseConverter; -import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.util.Bytes; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; - -/** - * Sample coprocessor endpoint exposing a Service interface for counting rows and key values. - * - *

- * For the protocol buffer definition of the RowCountService, see the source file located under - * hbase-server/src/main/protobuf/Examples.proto. - *

- */ -public class RowCountEndpoint extends ExampleProtos.RowCountService - implements Coprocessor, CoprocessorService { - private RegionCoprocessorEnvironment env; - - public RowCountEndpoint() { - } - - /** - * Just returns a reference to this object, which implements the RowCounterService interface. - */ - @Override - public Service getService() { - return this; - } - - /** - * Returns a count of the rows in the region where this coprocessor is loaded. - */ - @Override - public void getRowCount(RpcController controller, ExampleProtos.CountRequest request, - RpcCallback done) { - Scan scan = new Scan(); - scan.setFilter(new FirstKeyOnlyFilter()); - ExampleProtos.CountResponse response = null; - InternalScanner scanner = null; - try { - scanner = env.getRegion().getScanner(scan); - List results = new ArrayList(); - boolean hasMore = false; - byte[] lastRow = null; - long count = 0; - do { - hasMore = scanner.next(results); - for (KeyValue kv : results) { - byte[] currentRow = kv.getRow(); - if (lastRow == null || !Bytes.equals(lastRow, currentRow)) { - lastRow = currentRow; - count++; - } - } - results.clear(); - } while (hasMore); - - response = ExampleProtos.CountResponse.newBuilder() - .setCount(count).build(); - } catch (IOException ioe) { - ResponseConverter.setControllerException(controller, ioe); - } finally { - if (scanner != null) { - try { - scanner.close(); - } catch (IOException ignored) {} - } - } - done.run(response); - } - - /** - * Returns a count of all KeyValues in the region where this coprocessor is loaded. - */ - @Override - public void getKeyValueCount(RpcController controller, ExampleProtos.CountRequest request, - RpcCallback done) { - ExampleProtos.CountResponse response = null; - InternalScanner scanner = null; - try { - scanner = env.getRegion().getScanner(new Scan()); - List results = new ArrayList(); - boolean hasMore = false; - long count = 0; - do { - hasMore = scanner.next(results); - for (KeyValue kv : results) { - count++; - } - results.clear(); - } while (hasMore); - - response = ExampleProtos.CountResponse.newBuilder() - .setCount(count).build(); - } catch (IOException ioe) { - ResponseConverter.setControllerException(controller, ioe); - } finally { - if (scanner != null) { - try { - scanner.close(); - } catch (IOException ignored) {} - } - } - done.run(response); - } - - /** - * Stores a reference to the coprocessor environment provided by the - * {@link org.apache.hadoop.hbase.regionserver.RegionCoprocessorHost} from the region where this - * coprocessor is loaded. Since this is a coprocessor endpoint, it always expects to be loaded - * on a table region, so always expects this to be an instance of - * {@link RegionCoprocessorEnvironment}. - * @param env the environment provided by the coprocessor host - * @throws IOException if the provided environment is not an instance of - * {@code RegionCoprocessorEnvironment} - */ - @Override - public void start(CoprocessorEnvironment env) throws IOException { - if (env instanceof RegionCoprocessorEnvironment) { - this.env = (RegionCoprocessorEnvironment)env; - } else { - throw new CoprocessorException("Must be loaded on a table region!"); - } - } - - @Override - public void stop(CoprocessorEnvironment env) throws IOException { - // nothing to do - } -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java deleted file mode 100644 index 2e4eb48..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/ZooKeeperScanPolicyObserver.java +++ /dev/null @@ -1,232 +0,0 @@ -/* - * Copyright The Apache Software Foundation - * - * 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.coprocessor.example; - -import java.io.IOException; -import java.util.Collections; -import java.util.List; -import java.util.NavigableSet; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.hbase.CoprocessorEnvironment; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.coprocessor.BaseRegionObserver; -import org.apache.hadoop.hbase.coprocessor.ObserverContext; -import org.apache.hadoop.hbase.coprocessor.RegionCoprocessorEnvironment; -import org.apache.hadoop.hbase.coprocessor.RegionObserver; -import org.apache.hadoop.hbase.regionserver.HStore; -import org.apache.hadoop.hbase.regionserver.HStore.ScanInfo; -import org.apache.hadoop.hbase.regionserver.InternalScanner; -import org.apache.hadoop.hbase.regionserver.KeyValueScanner; -import org.apache.hadoop.hbase.regionserver.ScanType; -import org.apache.hadoop.hbase.regionserver.Store; -import org.apache.hadoop.hbase.regionserver.StoreScanner; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.zookeeper.KeeperException; -import org.apache.zookeeper.WatchedEvent; -import org.apache.zookeeper.Watcher; -import org.apache.zookeeper.ZooKeeper; - -/** - * This is an example showing how a RegionObserver could configured - * via ZooKeeper in order to control a Region compaction, flush, and scan policy. - * - * This also demonstrated the use of shared {@link RegionObserver} state. - * See {@link RegionCoprocessorEnvironment#getSharedData()}. - * - * This would be useful for an incremental backup tool, which would indicate the last - * time of a successful backup via ZK and instruct HBase to not delete data that was - * inserted since (based on wall clock time). - * - * This implements org.apache.zookeeper.Watcher directly instead of using - * {@link ZooKeeperWatcher}, because RegionObservers come and go and currently - * listeners registered with ZooKeeperWatcher cannot be removed. - */ -public class ZooKeeperScanPolicyObserver extends BaseRegionObserver { - public static String node = "/backup/example/lastbackup"; - public static String zkkey = "ZK"; - private static final Log LOG = LogFactory.getLog(ZooKeeperScanPolicyObserver.class); - - /** - * Internal watcher that keep "data" up to date asynchronously. - */ - private static class ZKWatcher implements Watcher { - private byte[] data = null; - private ZooKeeper zk; - private volatile boolean needSetup = true; - private volatile long lastSetupTry = 0; - - public ZKWatcher(ZooKeeper zk) { - this.zk = zk; - // trigger the listening - getData(); - } - - /** - * Get the maintained data. In case of any ZK exceptions this will retry - * establishing the connection (but not more than twice/minute). - * - * getData is on the critical path, so make sure it is fast unless there is - * a problem (network partion, ZK ensemble down, etc) - * Make sure at most one (unlucky) thread retries and other threads don't pile up - * while that threads tries to recreate the connection. - * - * @return the last know version of the data - */ - public byte[] getData() { - // try at most twice/minute - if (needSetup && EnvironmentEdgeManager.currentTimeMillis() > lastSetupTry + 30000) { - synchronized (this) { - // make sure only one thread tries to reconnect - if (needSetup) { - needSetup = false; - } else { - return data; - } - } - // do this without the lock held to avoid threads piling up on this lock, - // as it can take a while - try { - LOG.debug("Connecting to ZK"); - // record this attempt - lastSetupTry = EnvironmentEdgeManager.currentTimeMillis(); - if (zk.exists(node, false) != null) { - data = zk.getData(node, this, null); - LOG.debug("Read synchronously: "+(data == null ? "null" : Bytes.toLong(data))); - } else { - zk.exists(node, this); - } - } catch (Exception x) { - // try again if this fails - needSetup = true; - } - } - return data; - } - - @Override - public void process(WatchedEvent event) { - switch(event.getType()) { - case NodeDataChanged: - case NodeCreated: - try { - // get data and re-watch - data = zk.getData(node, this, null); - LOG.debug("Read asynchronously: "+(data == null ? "null" : Bytes.toLong(data))); - } catch (InterruptedException ix) { - } catch (KeeperException kx) { - needSetup = true; - } - break; - - case NodeDeleted: - try { - // just re-watch - zk.exists(node, this); - data = null; - } catch (InterruptedException ix) { - } catch (KeeperException kx) { - needSetup = true; - } - break; - - default: - // ignore - } - } - } - - @Override - public void start(CoprocessorEnvironment e) throws IOException { - RegionCoprocessorEnvironment re = (RegionCoprocessorEnvironment) e; - if (!re.getSharedData().containsKey(zkkey)) { - // there is a short race here - // in the worst case we create a watcher that will be notified once - re.getSharedData().putIfAbsent( - zkkey, - new ZKWatcher(re.getRegionServerServices().getZooKeeper() - .getRecoverableZooKeeper().getZooKeeper())); - } - } - - @Override - public void stop(CoprocessorEnvironment e) throws IOException { - // nothing to do here - } - - protected ScanInfo getScanInfo(HStore store, RegionCoprocessorEnvironment e) { - byte[] data = ((ZKWatcher)e.getSharedData().get(zkkey)).getData(); - if (data == null) { - return null; - } - ScanInfo oldSI = store.getScanInfo(); - if (oldSI.getTtl() == Long.MAX_VALUE) { - return null; - } - long ttl = Math.max(EnvironmentEdgeManager.currentTimeMillis() - Bytes.toLong(data), oldSI.getTtl()); - return new ScanInfo(store.getFamily(), ttl, - oldSI.getTimeToPurgeDeletes(), oldSI.getComparator()); - } - - @Override - public InternalScanner preFlushScannerOpen(final ObserverContext c, - HStore store, KeyValueScanner memstoreScanner, InternalScanner s) throws IOException { - HStore.ScanInfo scanInfo = getScanInfo(store, c.getEnvironment()); - if (scanInfo == null) { - // take default action - return null; - } - Scan scan = new Scan(); - scan.setMaxVersions(scanInfo.getMaxVersions()); - return new StoreScanner(store, scanInfo, scan, Collections.singletonList(memstoreScanner), - ScanType.MINOR_COMPACT, store.getHRegion().getSmallestReadPoint(), - HConstants.OLDEST_TIMESTAMP); - } - - @Override - public InternalScanner preCompactScannerOpen(final ObserverContext c, - HStore store, List scanners, ScanType scanType, long earliestPutTs, - InternalScanner s) throws IOException { - HStore.ScanInfo scanInfo = getScanInfo(store, c.getEnvironment()); - if (scanInfo == null) { - // take default action - return null; - } - Scan scan = new Scan(); - scan.setMaxVersions(scanInfo.getMaxVersions()); - return new StoreScanner(store, scanInfo, scan, scanners, scanType, store.getHRegion() - .getSmallestReadPoint(), earliestPutTs); - } - - @Override - public KeyValueScanner preStoreScannerOpen(final ObserverContext c, - final HStore store, final Scan scan, final NavigableSet targetCols, - final KeyValueScanner s) throws IOException { - HStore.ScanInfo scanInfo = getScanInfo(store, c.getEnvironment()); - if (scanInfo == null) { - // take default action - return null; - } - return new StoreScanner(store, scanInfo, scan, targetCols); - } -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/BulkDeleteProtos.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/BulkDeleteProtos.java deleted file mode 100644 index 2d421e1..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/BulkDeleteProtos.java +++ /dev/null @@ -1,1512 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: BulkDelete.proto - -package org.apache.hadoop.hbase.coprocessor.example.generated; - -public final class BulkDeleteProtos { - private BulkDeleteProtos() {} - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - } - public interface BulkDeleteRequestOrBuilder - extends com.google.protobuf.MessageOrBuilder { - - // required .Scan scan = 1; - boolean hasScan(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan(); - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder(); - - // required .BulkDeleteRequest.DeleteType deleteType = 2; - boolean hasDeleteType(); - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType getDeleteType(); - - // optional uint64 timestamp = 3; - boolean hasTimestamp(); - long getTimestamp(); - - // required uint32 rowBatchSize = 4; - boolean hasRowBatchSize(); - int getRowBatchSize(); - } - public static final class BulkDeleteRequest extends - com.google.protobuf.GeneratedMessage - implements BulkDeleteRequestOrBuilder { - // Use BulkDeleteRequest.newBuilder() to construct. - private BulkDeleteRequest(Builder builder) { - super(builder); - } - private BulkDeleteRequest(boolean noInit) {} - - private static final BulkDeleteRequest defaultInstance; - public static BulkDeleteRequest getDefaultInstance() { - return defaultInstance; - } - - public BulkDeleteRequest getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_fieldAccessorTable; - } - - public enum DeleteType - implements com.google.protobuf.ProtocolMessageEnum { - ROW(0, 0), - FAMILY(1, 1), - COLUMN(2, 2), - VERSION(3, 3), - ; - - public static final int ROW_VALUE = 0; - public static final int FAMILY_VALUE = 1; - public static final int COLUMN_VALUE = 2; - public static final int VERSION_VALUE = 3; - - - public final int getNumber() { return value; } - - public static DeleteType valueOf(int value) { - switch (value) { - case 0: return ROW; - case 1: return FAMILY; - case 2: return COLUMN; - case 3: return VERSION; - default: return null; - } - } - - public static com.google.protobuf.Internal.EnumLiteMap - internalGetValueMap() { - return internalValueMap; - } - private static com.google.protobuf.Internal.EnumLiteMap - internalValueMap = - new com.google.protobuf.Internal.EnumLiteMap() { - public DeleteType findValueByNumber(int number) { - return DeleteType.valueOf(number); - } - }; - - public final com.google.protobuf.Descriptors.EnumValueDescriptor - getValueDescriptor() { - return getDescriptor().getValues().get(index); - } - public final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptorForType() { - return getDescriptor(); - } - public static final com.google.protobuf.Descriptors.EnumDescriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDescriptor().getEnumTypes().get(0); - } - - private static final DeleteType[] VALUES = { - ROW, FAMILY, COLUMN, VERSION, - }; - - public static DeleteType valueOf( - com.google.protobuf.Descriptors.EnumValueDescriptor desc) { - if (desc.getType() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "EnumValueDescriptor is not for this type."); - } - return VALUES[desc.getIndex()]; - } - - private final int index; - private final int value; - - private DeleteType(int index, int value) { - this.index = index; - this.value = value; - } - - // @@protoc_insertion_point(enum_scope:BulkDeleteRequest.DeleteType) - } - - private int bitField0_; - // required .Scan scan = 1; - public static final int SCAN_FIELD_NUMBER = 1; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan scan_; - public boolean hasScan() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan() { - return scan_; - } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder() { - return scan_; - } - - // required .BulkDeleteRequest.DeleteType deleteType = 2; - public static final int DELETETYPE_FIELD_NUMBER = 2; - private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType deleteType_; - public boolean hasDeleteType() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType getDeleteType() { - return deleteType_; - } - - // optional uint64 timestamp = 3; - public static final int TIMESTAMP_FIELD_NUMBER = 3; - private long timestamp_; - public boolean hasTimestamp() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTimestamp() { - return timestamp_; - } - - // required uint32 rowBatchSize = 4; - public static final int ROWBATCHSIZE_FIELD_NUMBER = 4; - private int rowBatchSize_; - public boolean hasRowBatchSize() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public int getRowBatchSize() { - return rowBatchSize_; - } - - private void initFields() { - scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); - deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; - timestamp_ = 0L; - rowBatchSize_ = 0; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasScan()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasDeleteType()) { - memoizedIsInitialized = 0; - return false; - } - if (!hasRowBatchSize()) { - memoizedIsInitialized = 0; - return false; - } - if (!getScan().isInitialized()) { - memoizedIsInitialized = 0; - return false; - } - memoizedIsInitialized = 1; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeMessage(1, scan_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeEnum(2, deleteType_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - output.writeUInt64(3, timestamp_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - output.writeUInt32(4, rowBatchSize_); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream - .computeMessageSize(1, scan_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeEnumSize(2, deleteType_.getNumber()); - } - if (((bitField0_ & 0x00000004) == 0x00000004)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(3, timestamp_); - } - if (((bitField0_ & 0x00000008) == 0x00000008)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt32Size(4, rowBatchSize_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - private static final long serialVersionUID = 0L; - @java.lang.Override - protected java.lang.Object writeReplace() - throws java.io.ObjectStreamException { - return super.writeReplace(); - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)) { - return super.equals(obj); - } - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest other = (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest) obj; - - boolean result = true; - result = result && (hasScan() == other.hasScan()); - if (hasScan()) { - result = result && getScan() - .equals(other.getScan()); - } - result = result && (hasDeleteType() == other.hasDeleteType()); - if (hasDeleteType()) { - result = result && - (getDeleteType() == other.getDeleteType()); - } - result = result && (hasTimestamp() == other.hasTimestamp()); - if (hasTimestamp()) { - result = result && (getTimestamp() - == other.getTimestamp()); - } - result = result && (hasRowBatchSize() == other.hasRowBatchSize()); - if (hasRowBatchSize()) { - result = result && (getRowBatchSize() - == other.getRowBatchSize()); - } - result = result && - getUnknownFields().equals(other.getUnknownFields()); - return result; - } - - @java.lang.Override - public int hashCode() { - int hash = 41; - hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasScan()) { - hash = (37 * hash) + SCAN_FIELD_NUMBER; - hash = (53 * hash) + getScan().hashCode(); - } - if (hasDeleteType()) { - hash = (37 * hash) + DELETETYPE_FIELD_NUMBER; - hash = (53 * hash) + hashEnum(getDeleteType()); - } - if (hasTimestamp()) { - hash = (37 * hash) + TIMESTAMP_FIELD_NUMBER; - hash = (53 * hash) + hashLong(getTimestamp()); - } - if (hasRowBatchSize()) { - hash = (37 * hash) + ROWBATCHSIZE_FIELD_NUMBER; - hash = (53 * hash) + getRowBatchSize(); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - return hash; - } - - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteRequest_fieldAccessorTable; - } - - // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder(BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - getScanFieldBuilder(); - } - } - private static Builder create() { - return new Builder(); - } - - public Builder clear() { - super.clear(); - if (scanBuilder_ == null) { - scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); - } else { - scanBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000001); - deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; - bitField0_ = (bitField0_ & ~0x00000002); - timestamp_ = 0L; - bitField0_ = (bitField0_ & ~0x00000004); - rowBatchSize_ = 0; - bitField0_ = (bitField0_ & ~0x00000008); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDescriptor(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest getDefaultInstanceForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest build() { - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest buildPartial() { - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest result = new org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - if (scanBuilder_ == null) { - result.scan_ = scan_; - } else { - result.scan_ = scanBuilder_.build(); - } - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.deleteType_ = deleteType_; - if (((from_bitField0_ & 0x00000004) == 0x00000004)) { - to_bitField0_ |= 0x00000004; - } - result.timestamp_ = timestamp_; - if (((from_bitField0_ & 0x00000008) == 0x00000008)) { - to_bitField0_ |= 0x00000008; - } - result.rowBatchSize_ = rowBatchSize_; - result.bitField0_ = to_bitField0_; - onBuilt(); - return result; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest) { - return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest other) { - if (other == org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance()) return this; - if (other.hasScan()) { - mergeScan(other.getScan()); - } - if (other.hasDeleteType()) { - setDeleteType(other.getDeleteType()); - } - if (other.hasTimestamp()) { - setTimestamp(other.getTimestamp()); - } - if (other.hasRowBatchSize()) { - setRowBatchSize(other.getRowBatchSize()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - if (!hasScan()) { - - return false; - } - if (!hasDeleteType()) { - - return false; - } - if (!hasRowBatchSize()) { - - return false; - } - if (!getScan().isInitialized()) { - - return false; - } - return true; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 10: { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder subBuilder = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.newBuilder(); - if (hasScan()) { - subBuilder.mergeFrom(getScan()); - } - input.readMessage(subBuilder, extensionRegistry); - setScan(subBuilder.buildPartial()); - break; - } - case 16: { - int rawValue = input.readEnum(); - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType value = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.valueOf(rawValue); - if (value == null) { - unknownFields.mergeVarintField(2, rawValue); - } else { - bitField0_ |= 0x00000002; - deleteType_ = value; - } - break; - } - case 24: { - bitField0_ |= 0x00000004; - timestamp_ = input.readUInt64(); - break; - } - case 32: { - bitField0_ |= 0x00000008; - rowBatchSize_ = input.readUInt32(); - break; - } - } - } - } - - private int bitField0_; - - // required .Scan scan = 1; - private org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder> scanBuilder_; - public boolean hasScan() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan getScan() { - if (scanBuilder_ == null) { - return scan_; - } else { - return scanBuilder_.getMessage(); - } - } - public Builder setScan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan value) { - if (scanBuilder_ == null) { - if (value == null) { - throw new NullPointerException(); - } - scan_ = value; - onChanged(); - } else { - scanBuilder_.setMessage(value); - } - bitField0_ |= 0x00000001; - return this; - } - public Builder setScan( - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder builderForValue) { - if (scanBuilder_ == null) { - scan_ = builderForValue.build(); - onChanged(); - } else { - scanBuilder_.setMessage(builderForValue.build()); - } - bitField0_ |= 0x00000001; - return this; - } - public Builder mergeScan(org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan value) { - if (scanBuilder_ == null) { - if (((bitField0_ & 0x00000001) == 0x00000001) && - scan_ != org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance()) { - scan_ = - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.newBuilder(scan_).mergeFrom(value).buildPartial(); - } else { - scan_ = value; - } - onChanged(); - } else { - scanBuilder_.mergeFrom(value); - } - bitField0_ |= 0x00000001; - return this; - } - public Builder clearScan() { - if (scanBuilder_ == null) { - scan_ = org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.getDefaultInstance(); - onChanged(); - } else { - scanBuilder_.clear(); - } - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder getScanBuilder() { - bitField0_ |= 0x00000001; - onChanged(); - return getScanFieldBuilder().getBuilder(); - } - public org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder getScanOrBuilder() { - if (scanBuilder_ != null) { - return scanBuilder_.getMessageOrBuilder(); - } else { - return scan_; - } - } - private com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder> - getScanFieldBuilder() { - if (scanBuilder_ == null) { - scanBuilder_ = new com.google.protobuf.SingleFieldBuilder< - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.Scan.Builder, org.apache.hadoop.hbase.protobuf.generated.ClientProtos.ScanOrBuilder>( - scan_, - getParentForChildren(), - isClean()); - scan_ = null; - } - return scanBuilder_; - } - - // required .BulkDeleteRequest.DeleteType deleteType = 2; - private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; - public boolean hasDeleteType() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType getDeleteType() { - return deleteType_; - } - public Builder setDeleteType(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType value) { - if (value == null) { - throw new NullPointerException(); - } - bitField0_ |= 0x00000002; - deleteType_ = value; - onChanged(); - return this; - } - public Builder clearDeleteType() { - bitField0_ = (bitField0_ & ~0x00000002); - deleteType_ = org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType.ROW; - onChanged(); - return this; - } - - // optional uint64 timestamp = 3; - private long timestamp_ ; - public boolean hasTimestamp() { - return ((bitField0_ & 0x00000004) == 0x00000004); - } - public long getTimestamp() { - return timestamp_; - } - public Builder setTimestamp(long value) { - bitField0_ |= 0x00000004; - timestamp_ = value; - onChanged(); - return this; - } - public Builder clearTimestamp() { - bitField0_ = (bitField0_ & ~0x00000004); - timestamp_ = 0L; - onChanged(); - return this; - } - - // required uint32 rowBatchSize = 4; - private int rowBatchSize_ ; - public boolean hasRowBatchSize() { - return ((bitField0_ & 0x00000008) == 0x00000008); - } - public int getRowBatchSize() { - return rowBatchSize_; - } - public Builder setRowBatchSize(int value) { - bitField0_ |= 0x00000008; - rowBatchSize_ = value; - onChanged(); - return this; - } - public Builder clearRowBatchSize() { - bitField0_ = (bitField0_ & ~0x00000008); - rowBatchSize_ = 0; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:BulkDeleteRequest) - } - - static { - defaultInstance = new BulkDeleteRequest(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:BulkDeleteRequest) - } - - public interface BulkDeleteResponseOrBuilder - extends com.google.protobuf.MessageOrBuilder { - - // required uint64 rowsDeleted = 1; - boolean hasRowsDeleted(); - long getRowsDeleted(); - - // optional uint64 versionsDeleted = 2; - boolean hasVersionsDeleted(); - long getVersionsDeleted(); - } - public static final class BulkDeleteResponse extends - com.google.protobuf.GeneratedMessage - implements BulkDeleteResponseOrBuilder { - // Use BulkDeleteResponse.newBuilder() to construct. - private BulkDeleteResponse(Builder builder) { - super(builder); - } - private BulkDeleteResponse(boolean noInit) {} - - private static final BulkDeleteResponse defaultInstance; - public static BulkDeleteResponse getDefaultInstance() { - return defaultInstance; - } - - public BulkDeleteResponse getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_fieldAccessorTable; - } - - private int bitField0_; - // required uint64 rowsDeleted = 1; - public static final int ROWSDELETED_FIELD_NUMBER = 1; - private long rowsDeleted_; - public boolean hasRowsDeleted() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRowsDeleted() { - return rowsDeleted_; - } - - // optional uint64 versionsDeleted = 2; - public static final int VERSIONSDELETED_FIELD_NUMBER = 2; - private long versionsDeleted_; - public boolean hasVersionsDeleted() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getVersionsDeleted() { - return versionsDeleted_; - } - - private void initFields() { - rowsDeleted_ = 0L; - versionsDeleted_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasRowsDeleted()) { - memoizedIsInitialized = 0; - return false; - } - memoizedIsInitialized = 1; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeUInt64(1, rowsDeleted_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - output.writeUInt64(2, versionsDeleted_); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(1, rowsDeleted_); - } - if (((bitField0_ & 0x00000002) == 0x00000002)) { - size += com.google.protobuf.CodedOutputStream - .computeUInt64Size(2, versionsDeleted_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - private static final long serialVersionUID = 0L; - @java.lang.Override - protected java.lang.Object writeReplace() - throws java.io.ObjectStreamException { - return super.writeReplace(); - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse)) { - return super.equals(obj); - } - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse other = (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse) obj; - - boolean result = true; - result = result && (hasRowsDeleted() == other.hasRowsDeleted()); - if (hasRowsDeleted()) { - result = result && (getRowsDeleted() - == other.getRowsDeleted()); - } - result = result && (hasVersionsDeleted() == other.hasVersionsDeleted()); - if (hasVersionsDeleted()) { - result = result && (getVersionsDeleted() - == other.getVersionsDeleted()); - } - result = result && - getUnknownFields().equals(other.getUnknownFields()); - return result; - } - - @java.lang.Override - public int hashCode() { - int hash = 41; - hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasRowsDeleted()) { - hash = (37 * hash) + ROWSDELETED_FIELD_NUMBER; - hash = (53 * hash) + hashLong(getRowsDeleted()); - } - if (hasVersionsDeleted()) { - hash = (37 * hash) + VERSIONSDELETED_FIELD_NUMBER; - hash = (53 * hash) + hashLong(getVersionsDeleted()); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - return hash; - } - - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.internal_static_BulkDeleteResponse_fieldAccessorTable; - } - - // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder(BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - } - } - private static Builder create() { - return new Builder(); - } - - public Builder clear() { - super.clear(); - rowsDeleted_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - versionsDeleted_ = 0L; - bitField0_ = (bitField0_ & ~0x00000002); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDescriptor(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse getDefaultInstanceForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse build() { - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse buildPartial() { - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse result = new org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.rowsDeleted_ = rowsDeleted_; - if (((from_bitField0_ & 0x00000002) == 0x00000002)) { - to_bitField0_ |= 0x00000002; - } - result.versionsDeleted_ = versionsDeleted_; - result.bitField0_ = to_bitField0_; - onBuilt(); - return result; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse) { - return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse other) { - if (other == org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance()) return this; - if (other.hasRowsDeleted()) { - setRowsDeleted(other.getRowsDeleted()); - } - if (other.hasVersionsDeleted()) { - setVersionsDeleted(other.getVersionsDeleted()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - if (!hasRowsDeleted()) { - - return false; - } - return true; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 8: { - bitField0_ |= 0x00000001; - rowsDeleted_ = input.readUInt64(); - break; - } - case 16: { - bitField0_ |= 0x00000002; - versionsDeleted_ = input.readUInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required uint64 rowsDeleted = 1; - private long rowsDeleted_ ; - public boolean hasRowsDeleted() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getRowsDeleted() { - return rowsDeleted_; - } - public Builder setRowsDeleted(long value) { - bitField0_ |= 0x00000001; - rowsDeleted_ = value; - onChanged(); - return this; - } - public Builder clearRowsDeleted() { - bitField0_ = (bitField0_ & ~0x00000001); - rowsDeleted_ = 0L; - onChanged(); - return this; - } - - // optional uint64 versionsDeleted = 2; - private long versionsDeleted_ ; - public boolean hasVersionsDeleted() { - return ((bitField0_ & 0x00000002) == 0x00000002); - } - public long getVersionsDeleted() { - return versionsDeleted_; - } - public Builder setVersionsDeleted(long value) { - bitField0_ |= 0x00000002; - versionsDeleted_ = value; - onChanged(); - return this; - } - public Builder clearVersionsDeleted() { - bitField0_ = (bitField0_ & ~0x00000002); - versionsDeleted_ = 0L; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:BulkDeleteResponse) - } - - static { - defaultInstance = new BulkDeleteResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:BulkDeleteResponse) - } - - public static abstract class BulkDeleteService - implements com.google.protobuf.Service { - protected BulkDeleteService() {} - - public interface Interface { - public abstract void delete( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, - com.google.protobuf.RpcCallback done); - - } - - public static com.google.protobuf.Service newReflectiveService( - final Interface impl) { - return new BulkDeleteService() { - @java.lang.Override - public void delete( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, - com.google.protobuf.RpcCallback done) { - impl.delete(controller, request, done); - } - - }; - } - - public static com.google.protobuf.BlockingService - newReflectiveBlockingService(final BlockingInterface impl) { - return new com.google.protobuf.BlockingService() { - public final com.google.protobuf.Descriptors.ServiceDescriptor - getDescriptorForType() { - return getDescriptor(); - } - - public final com.google.protobuf.Message callBlockingMethod( - com.google.protobuf.Descriptors.MethodDescriptor method, - com.google.protobuf.RpcController controller, - com.google.protobuf.Message request) - throws com.google.protobuf.ServiceException { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.callBlockingMethod() given method descriptor for " + - "wrong service type."); - } - switch(method.getIndex()) { - case 0: - return impl.delete(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)request); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getRequestPrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getRequestPrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getResponsePrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getResponsePrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - }; - } - - public abstract void delete( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, - com.google.protobuf.RpcCallback done); - - public static final - com.google.protobuf.Descriptors.ServiceDescriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.getDescriptor().getServices().get(0); - } - public final com.google.protobuf.Descriptors.ServiceDescriptor - getDescriptorForType() { - return getDescriptor(); - } - - public final void callMethod( - com.google.protobuf.Descriptors.MethodDescriptor method, - com.google.protobuf.RpcController controller, - com.google.protobuf.Message request, - com.google.protobuf.RpcCallback< - com.google.protobuf.Message> done) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.callMethod() given method descriptor for wrong " + - "service type."); - } - switch(method.getIndex()) { - case 0: - this.delete(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest)request, - com.google.protobuf.RpcUtil.specializeCallback( - done)); - return; - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getRequestPrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getRequestPrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getResponsePrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getResponsePrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public static Stub newStub( - com.google.protobuf.RpcChannel channel) { - return new Stub(channel); - } - - public static final class Stub extends org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService implements Interface { - private Stub(com.google.protobuf.RpcChannel channel) { - this.channel = channel; - } - - private final com.google.protobuf.RpcChannel channel; - - public com.google.protobuf.RpcChannel getChannel() { - return channel; - } - - public void delete( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request, - com.google.protobuf.RpcCallback done) { - channel.callMethod( - getDescriptor().getMethods().get(0), - controller, - request, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance(), - com.google.protobuf.RpcUtil.generalizeCallback( - done, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.class, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance())); - } - } - - public static BlockingInterface newBlockingStub( - com.google.protobuf.BlockingRpcChannel channel) { - return new BlockingStub(channel); - } - - public interface BlockingInterface { - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse delete( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request) - throws com.google.protobuf.ServiceException; - } - - private static final class BlockingStub implements BlockingInterface { - private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { - this.channel = channel; - } - - private final com.google.protobuf.BlockingRpcChannel channel; - - public org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse delete( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest request) - throws com.google.protobuf.ServiceException { - return (org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(0), - controller, - request, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.getDefaultInstance()); - } - - } - } - - private static com.google.protobuf.Descriptors.Descriptor - internal_static_BulkDeleteRequest_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_BulkDeleteRequest_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_BulkDeleteResponse_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_BulkDeleteResponse_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n\020BulkDelete.proto\032\014Client.proto\"\300\001\n\021Bul" + - "kDeleteRequest\022\023\n\004scan\030\001 \002(\0132\005.Scan\0221\n\nd" + - "eleteType\030\002 \002(\0162\035.BulkDeleteRequest.Dele" + - "teType\022\021\n\ttimestamp\030\003 \001(\004\022\024\n\014rowBatchSiz" + - "e\030\004 \002(\r\":\n\nDeleteType\022\007\n\003ROW\020\000\022\n\n\006FAMILY" + - "\020\001\022\n\n\006COLUMN\020\002\022\013\n\007VERSION\020\003\"B\n\022BulkDelet" + - "eResponse\022\023\n\013rowsDeleted\030\001 \002(\004\022\027\n\017versio" + - "nsDeleted\030\002 \001(\0042F\n\021BulkDeleteService\0221\n\006" + - "delete\022\022.BulkDeleteRequest\032\023.BulkDeleteR" + - "esponseBQ\n5org.apache.hadoop.hbase.copro", - "cessor.example.generatedB\020BulkDeleteProt" + - "osH\001\210\001\001\240\001\001" - }; - com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = - new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { - public com.google.protobuf.ExtensionRegistry assignDescriptors( - com.google.protobuf.Descriptors.FileDescriptor root) { - descriptor = root; - internal_static_BulkDeleteRequest_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_BulkDeleteRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_BulkDeleteRequest_descriptor, - new java.lang.String[] { "Scan", "DeleteType", "Timestamp", "RowBatchSize", }, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.class, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.Builder.class); - internal_static_BulkDeleteResponse_descriptor = - getDescriptor().getMessageTypes().get(1); - internal_static_BulkDeleteResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_BulkDeleteResponse_descriptor, - new java.lang.String[] { "RowsDeleted", "VersionsDeleted", }, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.class, - org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse.Builder.class); - return null; - } - }; - com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - org.apache.hadoop.hbase.protobuf.generated.ClientProtos.getDescriptor(), - }, assigner); - } - - // @@protoc_insertion_point(outer_class_scope) -} diff --git hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java deleted file mode 100644 index 123bf3b..0000000 --- hbase-server/src/main/java/org/apache/hadoop/hbase/coprocessor/example/generated/ExampleProtos.java +++ /dev/null @@ -1,1034 +0,0 @@ -// Generated by the protocol buffer compiler. DO NOT EDIT! -// source: Examples.proto - -package org.apache.hadoop.hbase.coprocessor.example.generated; - -public final class ExampleProtos { - private ExampleProtos() {} - public static void registerAllExtensions( - com.google.protobuf.ExtensionRegistry registry) { - } - public interface CountRequestOrBuilder - extends com.google.protobuf.MessageOrBuilder { - } - public static final class CountRequest extends - com.google.protobuf.GeneratedMessage - implements CountRequestOrBuilder { - // Use CountRequest.newBuilder() to construct. - private CountRequest(Builder builder) { - super(builder); - } - private CountRequest(boolean noInit) {} - - private static final CountRequest defaultInstance; - public static CountRequest getDefaultInstance() { - return defaultInstance; - } - - public CountRequest getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_fieldAccessorTable; - } - - private void initFields() { - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - memoizedIsInitialized = 1; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - private static final long serialVersionUID = 0L; - @java.lang.Override - protected java.lang.Object writeReplace() - throws java.io.ObjectStreamException { - return super.writeReplace(); - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)) { - return super.equals(obj); - } - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest other = (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest) obj; - - boolean result = true; - result = result && - getUnknownFields().equals(other.getUnknownFields()); - return result; - } - - @java.lang.Override - public int hashCode() { - int hash = 41; - hash = (19 * hash) + getDescriptorForType().hashCode(); - hash = (29 * hash) + getUnknownFields().hashCode(); - return hash; - } - - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequestOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountRequest_fieldAccessorTable; - } - - // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder(BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - } - } - private static Builder create() { - return new Builder(); - } - - public Builder clear() { - super.clear(); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDescriptor(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest getDefaultInstanceForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest build() { - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest buildPartial() { - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest result = new org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest(this); - onBuilt(); - return result; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest) { - return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest other) { - if (other == org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance()) return this; - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - return true; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - } - } - } - - - // @@protoc_insertion_point(builder_scope:CountRequest) - } - - static { - defaultInstance = new CountRequest(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:CountRequest) - } - - public interface CountResponseOrBuilder - extends com.google.protobuf.MessageOrBuilder { - - // required int64 count = 1 [default = 0]; - boolean hasCount(); - long getCount(); - } - public static final class CountResponse extends - com.google.protobuf.GeneratedMessage - implements CountResponseOrBuilder { - // Use CountResponse.newBuilder() to construct. - private CountResponse(Builder builder) { - super(builder); - } - private CountResponse(boolean noInit) {} - - private static final CountResponse defaultInstance; - public static CountResponse getDefaultInstance() { - return defaultInstance; - } - - public CountResponse getDefaultInstanceForType() { - return defaultInstance; - } - - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_fieldAccessorTable; - } - - private int bitField0_; - // required int64 count = 1 [default = 0]; - public static final int COUNT_FIELD_NUMBER = 1; - private long count_; - public boolean hasCount() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getCount() { - return count_; - } - - private void initFields() { - count_ = 0L; - } - private byte memoizedIsInitialized = -1; - public final boolean isInitialized() { - byte isInitialized = memoizedIsInitialized; - if (isInitialized != -1) return isInitialized == 1; - - if (!hasCount()) { - memoizedIsInitialized = 0; - return false; - } - memoizedIsInitialized = 1; - return true; - } - - public void writeTo(com.google.protobuf.CodedOutputStream output) - throws java.io.IOException { - getSerializedSize(); - if (((bitField0_ & 0x00000001) == 0x00000001)) { - output.writeInt64(1, count_); - } - getUnknownFields().writeTo(output); - } - - private int memoizedSerializedSize = -1; - public int getSerializedSize() { - int size = memoizedSerializedSize; - if (size != -1) return size; - - size = 0; - if (((bitField0_ & 0x00000001) == 0x00000001)) { - size += com.google.protobuf.CodedOutputStream - .computeInt64Size(1, count_); - } - size += getUnknownFields().getSerializedSize(); - memoizedSerializedSize = size; - return size; - } - - private static final long serialVersionUID = 0L; - @java.lang.Override - protected java.lang.Object writeReplace() - throws java.io.ObjectStreamException { - return super.writeReplace(); - } - - @java.lang.Override - public boolean equals(final java.lang.Object obj) { - if (obj == this) { - return true; - } - if (!(obj instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse)) { - return super.equals(obj); - } - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse other = (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) obj; - - boolean result = true; - result = result && (hasCount() == other.hasCount()); - if (hasCount()) { - result = result && (getCount() - == other.getCount()); - } - result = result && - getUnknownFields().equals(other.getUnknownFields()); - return result; - } - - @java.lang.Override - public int hashCode() { - int hash = 41; - hash = (19 * hash) + getDescriptorForType().hashCode(); - if (hasCount()) { - hash = (37 * hash) + COUNT_FIELD_NUMBER; - hash = (53 * hash) + hashLong(getCount()); - } - hash = (29 * hash) + getUnknownFields().hashCode(); - return hash; - } - - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( - com.google.protobuf.ByteString data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( - com.google.protobuf.ByteString data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom(byte[] data) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( - byte[] data, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws com.google.protobuf.InvalidProtocolBufferException { - return newBuilder().mergeFrom(data, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom(java.io.InputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseDelimitedFrom(java.io.InputStream input) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseDelimitedFrom( - java.io.InputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - Builder builder = newBuilder(); - if (builder.mergeDelimitedFrom(input, extensionRegistry)) { - return builder.buildParsed(); - } else { - return null; - } - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( - com.google.protobuf.CodedInputStream input) - throws java.io.IOException { - return newBuilder().mergeFrom(input).buildParsed(); - } - public static org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse parseFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - return newBuilder().mergeFrom(input, extensionRegistry) - .buildParsed(); - } - - public static Builder newBuilder() { return Builder.create(); } - public Builder newBuilderForType() { return newBuilder(); } - public static Builder newBuilder(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse prototype) { - return newBuilder().mergeFrom(prototype); - } - public Builder toBuilder() { return newBuilder(this); } - - @java.lang.Override - protected Builder newBuilderForType( - com.google.protobuf.GeneratedMessage.BuilderParent parent) { - Builder builder = new Builder(parent); - return builder; - } - public static final class Builder extends - com.google.protobuf.GeneratedMessage.Builder - implements org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponseOrBuilder { - public static final com.google.protobuf.Descriptors.Descriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_descriptor; - } - - protected com.google.protobuf.GeneratedMessage.FieldAccessorTable - internalGetFieldAccessorTable() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.internal_static_CountResponse_fieldAccessorTable; - } - - // Construct using org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.newBuilder() - private Builder() { - maybeForceBuilderInitialization(); - } - - private Builder(BuilderParent parent) { - super(parent); - maybeForceBuilderInitialization(); - } - private void maybeForceBuilderInitialization() { - if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { - } - } - private static Builder create() { - return new Builder(); - } - - public Builder clear() { - super.clear(); - count_ = 0L; - bitField0_ = (bitField0_ & ~0x00000001); - return this; - } - - public Builder clone() { - return create().mergeFrom(buildPartial()); - } - - public com.google.protobuf.Descriptors.Descriptor - getDescriptorForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDescriptor(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getDefaultInstanceForType() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse build() { - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException(result); - } - return result; - } - - private org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse buildParsed() - throws com.google.protobuf.InvalidProtocolBufferException { - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = buildPartial(); - if (!result.isInitialized()) { - throw newUninitializedMessageException( - result).asInvalidProtocolBufferException(); - } - return result; - } - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse buildPartial() { - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse result = new org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse(this); - int from_bitField0_ = bitField0_; - int to_bitField0_ = 0; - if (((from_bitField0_ & 0x00000001) == 0x00000001)) { - to_bitField0_ |= 0x00000001; - } - result.count_ = count_; - result.bitField0_ = to_bitField0_; - onBuilt(); - return result; - } - - public Builder mergeFrom(com.google.protobuf.Message other) { - if (other instanceof org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) { - return mergeFrom((org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse)other); - } else { - super.mergeFrom(other); - return this; - } - } - - public Builder mergeFrom(org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse other) { - if (other == org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()) return this; - if (other.hasCount()) { - setCount(other.getCount()); - } - this.mergeUnknownFields(other.getUnknownFields()); - return this; - } - - public final boolean isInitialized() { - if (!hasCount()) { - - return false; - } - return true; - } - - public Builder mergeFrom( - com.google.protobuf.CodedInputStream input, - com.google.protobuf.ExtensionRegistryLite extensionRegistry) - throws java.io.IOException { - com.google.protobuf.UnknownFieldSet.Builder unknownFields = - com.google.protobuf.UnknownFieldSet.newBuilder( - this.getUnknownFields()); - while (true) { - int tag = input.readTag(); - switch (tag) { - case 0: - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - default: { - if (!parseUnknownField(input, unknownFields, - extensionRegistry, tag)) { - this.setUnknownFields(unknownFields.build()); - onChanged(); - return this; - } - break; - } - case 8: { - bitField0_ |= 0x00000001; - count_ = input.readInt64(); - break; - } - } - } - } - - private int bitField0_; - - // required int64 count = 1 [default = 0]; - private long count_ ; - public boolean hasCount() { - return ((bitField0_ & 0x00000001) == 0x00000001); - } - public long getCount() { - return count_; - } - public Builder setCount(long value) { - bitField0_ |= 0x00000001; - count_ = value; - onChanged(); - return this; - } - public Builder clearCount() { - bitField0_ = (bitField0_ & ~0x00000001); - count_ = 0L; - onChanged(); - return this; - } - - // @@protoc_insertion_point(builder_scope:CountResponse) - } - - static { - defaultInstance = new CountResponse(true); - defaultInstance.initFields(); - } - - // @@protoc_insertion_point(class_scope:CountResponse) - } - - public static abstract class RowCountService - implements com.google.protobuf.Service { - protected RowCountService() {} - - public interface Interface { - public abstract void getRowCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done); - - public abstract void getKeyValueCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done); - - } - - public static com.google.protobuf.Service newReflectiveService( - final Interface impl) { - return new RowCountService() { - @java.lang.Override - public void getRowCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done) { - impl.getRowCount(controller, request, done); - } - - @java.lang.Override - public void getKeyValueCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done) { - impl.getKeyValueCount(controller, request, done); - } - - }; - } - - public static com.google.protobuf.BlockingService - newReflectiveBlockingService(final BlockingInterface impl) { - return new com.google.protobuf.BlockingService() { - public final com.google.protobuf.Descriptors.ServiceDescriptor - getDescriptorForType() { - return getDescriptor(); - } - - public final com.google.protobuf.Message callBlockingMethod( - com.google.protobuf.Descriptors.MethodDescriptor method, - com.google.protobuf.RpcController controller, - com.google.protobuf.Message request) - throws com.google.protobuf.ServiceException { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.callBlockingMethod() given method descriptor for " + - "wrong service type."); - } - switch(method.getIndex()) { - case 0: - return impl.getRowCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request); - case 1: - return impl.getKeyValueCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getRequestPrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getRequestPrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); - case 1: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getResponsePrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getResponsePrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); - case 1: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - }; - } - - public abstract void getRowCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done); - - public abstract void getKeyValueCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done); - - public static final - com.google.protobuf.Descriptors.ServiceDescriptor - getDescriptor() { - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.getDescriptor().getServices().get(0); - } - public final com.google.protobuf.Descriptors.ServiceDescriptor - getDescriptorForType() { - return getDescriptor(); - } - - public final void callMethod( - com.google.protobuf.Descriptors.MethodDescriptor method, - com.google.protobuf.RpcController controller, - com.google.protobuf.Message request, - com.google.protobuf.RpcCallback< - com.google.protobuf.Message> done) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.callMethod() given method descriptor for wrong " + - "service type."); - } - switch(method.getIndex()) { - case 0: - this.getRowCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request, - com.google.protobuf.RpcUtil.specializeCallback( - done)); - return; - case 1: - this.getKeyValueCount(controller, (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest)request, - com.google.protobuf.RpcUtil.specializeCallback( - done)); - return; - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getRequestPrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getRequestPrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); - case 1: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public final com.google.protobuf.Message - getResponsePrototype( - com.google.protobuf.Descriptors.MethodDescriptor method) { - if (method.getService() != getDescriptor()) { - throw new java.lang.IllegalArgumentException( - "Service.getResponsePrototype() given method " + - "descriptor for wrong service type."); - } - switch(method.getIndex()) { - case 0: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); - case 1: - return org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(); - default: - throw new java.lang.AssertionError("Can't get here."); - } - } - - public static Stub newStub( - com.google.protobuf.RpcChannel channel) { - return new Stub(channel); - } - - public static final class Stub extends org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.RowCountService implements Interface { - private Stub(com.google.protobuf.RpcChannel channel) { - this.channel = channel; - } - - private final com.google.protobuf.RpcChannel channel; - - public com.google.protobuf.RpcChannel getChannel() { - return channel; - } - - public void getRowCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done) { - channel.callMethod( - getDescriptor().getMethods().get(0), - controller, - request, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(), - com.google.protobuf.RpcUtil.generalizeCallback( - done, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance())); - } - - public void getKeyValueCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request, - com.google.protobuf.RpcCallback done) { - channel.callMethod( - getDescriptor().getMethods().get(1), - controller, - request, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance(), - com.google.protobuf.RpcUtil.generalizeCallback( - done, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance())); - } - } - - public static BlockingInterface newBlockingStub( - com.google.protobuf.BlockingRpcChannel channel) { - return new BlockingStub(channel); - } - - public interface BlockingInterface { - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getRowCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) - throws com.google.protobuf.ServiceException; - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getKeyValueCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) - throws com.google.protobuf.ServiceException; - } - - private static final class BlockingStub implements BlockingInterface { - private BlockingStub(com.google.protobuf.BlockingRpcChannel channel) { - this.channel = channel; - } - - private final com.google.protobuf.BlockingRpcChannel channel; - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getRowCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) - throws com.google.protobuf.ServiceException { - return (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(0), - controller, - request, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()); - } - - - public org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse getKeyValueCount( - com.google.protobuf.RpcController controller, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest request) - throws com.google.protobuf.ServiceException { - return (org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse) channel.callBlockingMethod( - getDescriptor().getMethods().get(1), - controller, - request, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.getDefaultInstance()); - } - - } - } - - private static com.google.protobuf.Descriptors.Descriptor - internal_static_CountRequest_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_CountRequest_fieldAccessorTable; - private static com.google.protobuf.Descriptors.Descriptor - internal_static_CountResponse_descriptor; - private static - com.google.protobuf.GeneratedMessage.FieldAccessorTable - internal_static_CountResponse_fieldAccessorTable; - - public static com.google.protobuf.Descriptors.FileDescriptor - getDescriptor() { - return descriptor; - } - private static com.google.protobuf.Descriptors.FileDescriptor - descriptor; - static { - java.lang.String[] descriptorData = { - "\n\016Examples.proto\"\016\n\014CountRequest\"!\n\rCoun" + - "tResponse\022\020\n\005count\030\001 \002(\003:\00102r\n\017RowCountS" + - "ervice\022,\n\013getRowCount\022\r.CountRequest\032\016.C" + - "ountResponse\0221\n\020getKeyValueCount\022\r.Count" + - "Request\032\016.CountResponseBN\n5org.apache.ha" + - "doop.hbase.coprocessor.example.generated" + - "B\rExampleProtosH\001\210\001\001\240\001\001" - }; - com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = - new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { - public com.google.protobuf.ExtensionRegistry assignDescriptors( - com.google.protobuf.Descriptors.FileDescriptor root) { - descriptor = root; - internal_static_CountRequest_descriptor = - getDescriptor().getMessageTypes().get(0); - internal_static_CountRequest_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_CountRequest_descriptor, - new java.lang.String[] { }, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.class, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountRequest.Builder.class); - internal_static_CountResponse_descriptor = - getDescriptor().getMessageTypes().get(1); - internal_static_CountResponse_fieldAccessorTable = new - com.google.protobuf.GeneratedMessage.FieldAccessorTable( - internal_static_CountResponse_descriptor, - new java.lang.String[] { "Count", }, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.class, - org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos.CountResponse.Builder.class); - return null; - } - }; - com.google.protobuf.Descriptors.FileDescriptor - .internalBuildGeneratedFileFrom(descriptorData, - new com.google.protobuf.Descriptors.FileDescriptor[] { - }, assigner); - } - - // @@protoc_insertion_point(outer_class_scope) -} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java deleted file mode 100644 index e8faf30..0000000 --- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestBulkDeleteProtocol.java +++ /dev/null @@ -1,434 +0,0 @@ -/* - * 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.coprocessor.example; - -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; - -import java.io.IOException; -import java.util.ArrayList; -import java.util.List; -import java.util.Map; - -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HConstants; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.KeyValue; -import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.client.Scan; -import org.apache.hadoop.hbase.client.coprocessor.Batch; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteResponse; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteService; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.Builder; -import org.apache.hadoop.hbase.coprocessor.example.generated.BulkDeleteProtos.BulkDeleteRequest.DeleteType; -import org.apache.hadoop.hbase.filter.FilterList; -import org.apache.hadoop.hbase.filter.SingleColumnValueFilter; -import org.apache.hadoop.hbase.filter.CompareFilter.CompareOp; -import org.apache.hadoop.hbase.filter.FilterList.Operator; -import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; -import org.apache.hadoop.hbase.ipc.ServerRpcController; -import org.apache.hadoop.hbase.protobuf.ProtobufUtil; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(MediumTests.class) -public class TestBulkDeleteProtocol { - private static final byte[] FAMILY1 = Bytes.toBytes("cf1"); - private static final byte[] FAMILY2 = Bytes.toBytes("cf2"); - private static final byte[] QUALIFIER1 = Bytes.toBytes("c1"); - private static final byte[] QUALIFIER2 = Bytes.toBytes("c2"); - private static final byte[] QUALIFIER3 = Bytes.toBytes("c3"); - private static final HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - - @BeforeClass - public static void setupBeforeClass() throws Exception { - TEST_UTIL.getConfiguration().set(CoprocessorHost.USER_REGION_COPROCESSOR_CONF_KEY, - BulkDeleteEndpoint.class.getName()); - TEST_UTIL.startMiniCluster(2); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Test - public void testBulkDeleteEndpoint() throws Throwable { - byte[] tableName = Bytes.toBytes("testBulkDeleteEndpoint"); - HTable ht = createTable(tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - byte[] rowkey = Bytes.toBytes(j); - puts.add(createPut(rowkey, "v1")); - } - ht.put(puts); - // Deleting all the rows. - long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, new Scan(), 5, DeleteType.ROW, null); - assertEquals(100, noOfRowsDeleted); - - int rows = 0; - for (Result result : ht.getScanner(new Scan())) { - rows++; - } - assertEquals(0, rows); - } - - @Test - public void testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion() - throws Throwable { - byte[] tableName = Bytes - .toBytes("testBulkDeleteEndpointWhenRowBatchSizeLessThanRowsToDeleteFromARegion"); - HTable ht = createTable(tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - byte[] rowkey = Bytes.toBytes(j); - puts.add(createPut(rowkey, "v1")); - } - ht.put(puts); - // Deleting all the rows. - long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, new Scan(), 10, DeleteType.ROW, null); - assertEquals(100, noOfRowsDeleted); - - int rows = 0; - for (Result result : ht.getScanner(new Scan())) { - rows++; - } - assertEquals(0, rows); - } - - private long invokeBulkDeleteProtocol(byte[] tableName, final Scan scan, final int rowBatchSize, - final DeleteType deleteType, final Long timeStamp) throws Throwable { - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); - long noOfDeletedRows = 0L; - Batch.Call callable = - new Batch.Call() { - ServerRpcController controller = new ServerRpcController(); - BlockingRpcCallback rpcCallback = - new BlockingRpcCallback(); - - public BulkDeleteResponse call(BulkDeleteService service) throws IOException { - Builder builder = BulkDeleteRequest.newBuilder(); - builder.setScan(ProtobufUtil.toScan(scan)); - builder.setDeleteType(deleteType); - builder.setRowBatchSize(rowBatchSize); - if (timeStamp != null) { - builder.setTimestamp(timeStamp); - } - service.delete(controller, builder.build(), rpcCallback); - return rpcCallback.get(); - } - }; - Map result = ht.coprocessorService(BulkDeleteService.class, scan - .getStartRow(), scan.getStopRow(), callable); - for (BulkDeleteResponse response : result.values()) { - noOfDeletedRows += response.getRowsDeleted(); - } - return noOfDeletedRows; - } - - @Test - public void testBulkDeleteWithConditionBasedDelete() throws Throwable { - byte[] tableName = Bytes.toBytes("testBulkDeleteWithConditionBasedDelete"); - HTable ht = createTable(tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - byte[] rowkey = Bytes.toBytes(j); - String value = (j % 10 == 0) ? "v1" : "v2"; - puts.add(createPut(rowkey, value)); - } - ht.put(puts); - Scan scan = new Scan(); - FilterList fl = new FilterList(Operator.MUST_PASS_ALL); - SingleColumnValueFilter scvf = new SingleColumnValueFilter(FAMILY1, QUALIFIER3, - CompareOp.EQUAL, Bytes.toBytes("v1")); - // fl.addFilter(new FirstKeyOnlyFilter()); - fl.addFilter(scvf); - scan.setFilter(fl); - // Deleting all the rows where cf1:c1=v1 - long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.ROW, null); - assertEquals(10, noOfRowsDeleted); - - int rows = 0; - for (Result result : ht.getScanner(new Scan())) { - rows++; - } - assertEquals(90, rows); - } - - @Test - public void testBulkDeleteColumn() throws Throwable { - byte[] tableName = Bytes.toBytes("testBulkDeleteColumn"); - HTable ht = createTable(tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - byte[] rowkey = Bytes.toBytes(j); - String value = (j % 10 == 0) ? "v1" : "v2"; - puts.add(createPut(rowkey, value)); - } - ht.put(puts); - Scan scan = new Scan(); - scan.addColumn(FAMILY1, QUALIFIER2); - // Delete the column cf1:col2 - long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.COLUMN, null); - assertEquals(100, noOfRowsDeleted); - - int rows = 0; - for (Result result : ht.getScanner(new Scan())) { - assertEquals(2, result.getFamilyMap(FAMILY1).size()); - assertTrue(result.getColumn(FAMILY1, QUALIFIER2).isEmpty()); - assertEquals(1, result.getColumn(FAMILY1, QUALIFIER1).size()); - assertEquals(1, result.getColumn(FAMILY1, QUALIFIER3).size()); - rows++; - } - assertEquals(100, rows); - } - - @Test - public void testBulkDeleteFamily() throws Throwable { - byte[] tableName = Bytes.toBytes("testBulkDeleteFamily"); - HTableDescriptor htd = new HTableDescriptor(tableName); - htd.addFamily(new HColumnDescriptor(FAMILY1)); - htd.addFamily(new HColumnDescriptor(FAMILY2)); - TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - Put put = new Put(Bytes.toBytes(j)); - put.add(FAMILY1, QUALIFIER1, "v1".getBytes()); - put.add(FAMILY2, QUALIFIER2, "v2".getBytes()); - puts.add(put); - } - ht.put(puts); - Scan scan = new Scan(); - scan.addFamily(FAMILY1); - // Delete the column family cf1 - long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.FAMILY, null); - assertEquals(100, noOfRowsDeleted); - int rows = 0; - for (Result result : ht.getScanner(new Scan())) { - assertTrue(result.getFamilyMap(FAMILY1).isEmpty()); - assertEquals(1, result.getColumn(FAMILY2, QUALIFIER2).size()); - rows++; - } - assertEquals(100, rows); - } - - @Test - public void testBulkDeleteColumnVersion() throws Throwable { - byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersion"); - HTable ht = createTable(tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - Put put = new Put(Bytes.toBytes(j)); - byte[] value = "v1".getBytes(); - put.add(FAMILY1, QUALIFIER1, 1234L, value); - put.add(FAMILY1, QUALIFIER2, 1234L, value); - put.add(FAMILY1, QUALIFIER3, 1234L, value); - // Latest version values - value = "v2".getBytes(); - put.add(FAMILY1, QUALIFIER1, value); - put.add(FAMILY1, QUALIFIER2, value); - put.add(FAMILY1, QUALIFIER3, value); - put.add(FAMILY1, null, value); - puts.add(put); - } - ht.put(puts); - Scan scan = new Scan(); - scan.addFamily(FAMILY1); - // Delete the latest version values of all the columns in family cf1. - long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.VERSION, - HConstants.LATEST_TIMESTAMP); - assertEquals(100, noOfRowsDeleted); - int rows = 0; - scan = new Scan(); - scan.setMaxVersions(); - for (Result result : ht.getScanner(scan)) { - assertEquals(3, result.getFamilyMap(FAMILY1).size()); - List column = result.getColumn(FAMILY1, QUALIFIER1); - assertEquals(1, column.size()); - assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue())); - - column = result.getColumn(FAMILY1, QUALIFIER2); - assertEquals(1, column.size()); - assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue())); - - column = result.getColumn(FAMILY1, QUALIFIER3); - assertEquals(1, column.size()); - assertTrue(Bytes.equals("v1".getBytes(), column.get(0).getValue())); - rows++; - } - assertEquals(100, rows); - } - - @Test - public void testBulkDeleteColumnVersionBasedOnTS() throws Throwable { - byte[] tableName = Bytes.toBytes("testBulkDeleteColumnVersionBasedOnTS"); - HTable ht = createTable(tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - Put put = new Put(Bytes.toBytes(j)); - // TS = 1000L - byte[] value = "v1".getBytes(); - put.add(FAMILY1, QUALIFIER1, 1000L, value); - put.add(FAMILY1, QUALIFIER2, 1000L, value); - put.add(FAMILY1, QUALIFIER3, 1000L, value); - // TS = 1234L - value = "v2".getBytes(); - put.add(FAMILY1, QUALIFIER1, 1234L, value); - put.add(FAMILY1, QUALIFIER2, 1234L, value); - put.add(FAMILY1, QUALIFIER3, 1234L, value); - // Latest version values - value = "v3".getBytes(); - put.add(FAMILY1, QUALIFIER1, value); - put.add(FAMILY1, QUALIFIER2, value); - put.add(FAMILY1, QUALIFIER3, value); - puts.add(put); - } - ht.put(puts); - Scan scan = new Scan(); - scan.addColumn(FAMILY1, QUALIFIER3); - // Delete the column cf1:c3's one version at TS=1234 - long noOfRowsDeleted = invokeBulkDeleteProtocol(tableName, scan, 500, DeleteType.VERSION, 1234L); - assertEquals(100, noOfRowsDeleted); - int rows = 0; - scan = new Scan(); - scan.setMaxVersions(); - for (Result result : ht.getScanner(scan)) { - assertEquals(3, result.getFamilyMap(FAMILY1).size()); - assertEquals(3, result.getColumn(FAMILY1, QUALIFIER1).size()); - assertEquals(3, result.getColumn(FAMILY1, QUALIFIER2).size()); - List column = result.getColumn(FAMILY1, QUALIFIER3); - assertEquals(2, column.size()); - assertTrue(Bytes.equals("v3".getBytes(), column.get(0).getValue())); - assertTrue(Bytes.equals("v1".getBytes(), column.get(1).getValue())); - rows++; - } - assertEquals(100, rows); - } - - @Test - public void testBulkDeleteWithNumberOfVersions() throws Throwable { - byte[] tableName = Bytes.toBytes("testBulkDeleteWithNumberOfVersions"); - HTable ht = createTable(tableName); - List puts = new ArrayList(100); - for (int j = 0; j < 100; j++) { - Put put = new Put(Bytes.toBytes(j)); - // TS = 1000L - byte[] value = "v1".getBytes(); - put.add(FAMILY1, QUALIFIER1, 1000L, value); - put.add(FAMILY1, QUALIFIER2, 1000L, value); - put.add(FAMILY1, QUALIFIER3, 1000L, value); - // TS = 1234L - value = "v2".getBytes(); - put.add(FAMILY1, QUALIFIER1, 1234L, value); - put.add(FAMILY1, QUALIFIER2, 1234L, value); - put.add(FAMILY1, QUALIFIER3, 1234L, value); - // TS = 2000L - value = "v3".getBytes(); - put.add(FAMILY1, QUALIFIER1, 2000L, value); - put.add(FAMILY1, QUALIFIER2, 2000L, value); - put.add(FAMILY1, QUALIFIER3, 2000L, value); - // Latest version values - value = "v4".getBytes(); - put.add(FAMILY1, QUALIFIER1, value); - put.add(FAMILY1, QUALIFIER2, value); - put.add(FAMILY1, QUALIFIER3, value); - puts.add(put); - } - ht.put(puts); - - // Delete all the versions of columns cf1:c1 and cf1:c2 falling with the time range - // [1000,2000) - final Scan scan = new Scan(); - scan.addColumn(FAMILY1, QUALIFIER1); - scan.addColumn(FAMILY1, QUALIFIER2); - scan.setTimeRange(1000L, 2000L); - scan.setMaxVersions(); - - long noOfDeletedRows = 0L; - long noOfVersionsDeleted = 0L; - Batch.Call callable = - new Batch.Call() { - ServerRpcController controller = new ServerRpcController(); - BlockingRpcCallback rpcCallback = - new BlockingRpcCallback(); - - public BulkDeleteResponse call(BulkDeleteService service) throws IOException { - Builder builder = BulkDeleteRequest.newBuilder(); - builder.setScan(ProtobufUtil.toScan(scan)); - builder.setDeleteType(DeleteType.VERSION); - builder.setRowBatchSize(500); - service.delete(controller, builder.build(), rpcCallback); - return rpcCallback.get(); - } - }; - Map result = ht.coprocessorService(BulkDeleteService.class, scan - .getStartRow(), scan.getStopRow(), callable); - for (BulkDeleteResponse response : result.values()) { - noOfDeletedRows += response.getRowsDeleted(); - noOfVersionsDeleted += response.getVersionsDeleted(); - } - assertEquals(100, noOfDeletedRows); - assertEquals(400, noOfVersionsDeleted); - - int rows = 0; - Scan scan1 = new Scan(); - scan1.setMaxVersions(); - for (Result res : ht.getScanner(scan1)) { - assertEquals(3, res.getFamilyMap(FAMILY1).size()); - List column = res.getColumn(FAMILY1, QUALIFIER1); - assertEquals(2, column.size()); - assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue())); - assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue())); - column = res.getColumn(FAMILY1, QUALIFIER2); - assertEquals(2, column.size()); - assertTrue(Bytes.equals("v4".getBytes(), column.get(0).getValue())); - assertTrue(Bytes.equals("v3".getBytes(), column.get(1).getValue())); - assertEquals(4, res.getColumn(FAMILY1, QUALIFIER3).size()); - rows++; - } - assertEquals(100, rows); - } - - private HTable createTable(byte[] tableName) throws IOException { - HTableDescriptor htd = new HTableDescriptor(tableName); - HColumnDescriptor hcd = new HColumnDescriptor(FAMILY1); - hcd.setMaxVersions(10);// Just setting 10 as I am not testing with more than 10 versions here - htd.addFamily(hcd); - TEST_UTIL.getHBaseAdmin().createTable(htd, Bytes.toBytes(0), Bytes.toBytes(120), 5); - HTable ht = new HTable(TEST_UTIL.getConfiguration(), tableName); - return ht; - } - - private Put createPut(byte[] rowkey, String value) throws IOException { - Put put = new Put(rowkey); - put.add(FAMILY1, QUALIFIER1, value.getBytes()); - put.add(FAMILY1, QUALIFIER2, value.getBytes()); - put.add(FAMILY1, QUALIFIER3, value.getBytes()); - return put; - } -} \ No newline at end of file diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java deleted file mode 100644 index e582997..0000000 --- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestRowCountEndpoint.java +++ /dev/null @@ -1,108 +0,0 @@ -/* - * 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.coprocessor.example; - -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.coprocessor.Batch; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.coprocessor.example.generated.ExampleProtos; -import org.apache.hadoop.hbase.ipc.BlockingRpcCallback; -import org.apache.hadoop.hbase.ipc.ServerRpcController; -import org.apache.hadoop.hbase.util.Bytes; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; - -import static junit.framework.Assert.*; - -/** - * Test case demonstrating client interactions with the {@link RowCountEndpoint} - * sample coprocessor Service implementation. - */ -@Category(MediumTests.class) -public class TestRowCountEndpoint { - private static final byte[] TEST_TABLE = Bytes.toBytes("testrowcounter"); - private static final byte[] TEST_FAMILY = Bytes.toBytes("f"); - private static final byte[] TEST_COLUMN = Bytes.toBytes("col"); - - private static HBaseTestingUtility TEST_UTIL = null; - private static Configuration CONF = null; - - @BeforeClass - public static void setupBeforeClass() throws Exception { - TEST_UTIL = new HBaseTestingUtility(); - CONF = TEST_UTIL.getConfiguration(); - CONF.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, - RowCountEndpoint.class.getName()); - - TEST_UTIL.startMiniCluster(); - TEST_UTIL.createTable(TEST_TABLE, TEST_FAMILY); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Test - public void testEndpoint() throws Throwable { - HTable table = new HTable(CONF, TEST_TABLE); - - // insert some test rows - for (int i=0; i<5; i++) { - byte[] iBytes = Bytes.toBytes(i); - Put p = new Put(iBytes); - p.add(TEST_FAMILY, TEST_COLUMN, iBytes); - table.put(p); - } - - final ExampleProtos.CountRequest request = ExampleProtos.CountRequest.getDefaultInstance(); - Map results = table.coprocessorService(ExampleProtos.RowCountService.class, - null, null, - new Batch.Call() { - public Long call(ExampleProtos.RowCountService counter) throws IOException { - ServerRpcController controller = new ServerRpcController(); - BlockingRpcCallback rpcCallback = - new BlockingRpcCallback(); - counter.getRowCount(controller, request, rpcCallback); - ExampleProtos.CountResponse response = rpcCallback.get(); - if (controller.failedOnException()) { - throw controller.getFailedOn(); - } - return (response != null && response.hasCount()) ? response.getCount() : 0; - } - }); - // should be one region with results - assertEquals(1, results.size()); - Iterator iter = results.values().iterator(); - Long val = iter.next(); - assertNotNull(val); - assertEquals(5l, val.longValue()); - } - -} diff --git hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java deleted file mode 100644 index 83ac40c..0000000 --- hbase-server/src/test/java/org/apache/hadoop/hbase/coprocessor/example/TestZooKeeperScanPolicyObserver.java +++ /dev/null @@ -1,131 +0,0 @@ -/* - * Copyright The Apache Software Foundation - * - * 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.coprocessor.example; - -import static org.junit.Assert.assertEquals; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.hbase.HBaseTestingUtility; -import org.apache.hadoop.hbase.HColumnDescriptor; -import org.apache.hadoop.hbase.HTableDescriptor; -import org.apache.hadoop.hbase.MediumTests; -import org.apache.hadoop.hbase.client.Get; -import org.apache.hadoop.hbase.client.HConnectionManager; -import org.apache.hadoop.hbase.client.HTable; -import org.apache.hadoop.hbase.client.Put; -import org.apache.hadoop.hbase.client.Result; -import org.apache.hadoop.hbase.coprocessor.CoprocessorHost; -import org.apache.hadoop.hbase.util.Bytes; -import org.apache.hadoop.hbase.util.EnvironmentEdgeManager; -import org.apache.hadoop.hbase.zookeeper.ZKUtil; -import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; -import org.apache.zookeeper.ZooKeeper; -import org.junit.AfterClass; -import org.junit.BeforeClass; -import org.junit.Test; -import org.junit.experimental.categories.Category; - -@Category(MediumTests.class) -public class TestZooKeeperScanPolicyObserver { - private static final Log LOG = LogFactory.getLog(TestZooKeeperScanPolicyObserver.class); - private final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); - private static final byte[] F = Bytes.toBytes("fam"); - private static final byte[] Q = Bytes.toBytes("qual"); - private static final byte[] R = Bytes.toBytes("row"); - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - // Test we can first start the ZK cluster by itself - Configuration conf = TEST_UTIL.getConfiguration(); - conf.setStrings(CoprocessorHost.REGION_COPROCESSOR_CONF_KEY, - ZooKeeperScanPolicyObserver.class.getName()); - TEST_UTIL.startMiniZKCluster(); - TEST_UTIL.startMiniCluster(); - } - - @AfterClass - public static void tearDownAfterClass() throws Exception { - TEST_UTIL.shutdownMiniCluster(); - } - - @Test - public void testScanPolicyObserver() throws Exception { - byte[] tableName = Bytes.toBytes("testScanPolicyObserver"); - HTableDescriptor desc = new HTableDescriptor(tableName); - HColumnDescriptor hcd = new HColumnDescriptor(F) - .setMaxVersions(10) - .setTimeToLive(1); - desc.addFamily(hcd); - TEST_UTIL.getHBaseAdmin().createTable(desc); - HTable t = new HTable(new Configuration(TEST_UTIL.getConfiguration()), tableName); - long now = EnvironmentEdgeManager.currentTimeMillis(); - - ZooKeeperWatcher zkw = HConnectionManager.getConnection(TEST_UTIL.getConfiguration()) - .getZooKeeperWatcher(); - ZooKeeper zk = zkw.getRecoverableZooKeeper().getZooKeeper(); - ZKUtil.createWithParents(zkw, ZooKeeperScanPolicyObserver.node); - // let's say test last backup was 1h ago - // using plain ZK here, because RecoverableZooKeeper add extra encoding to the data - zk.setData(ZooKeeperScanPolicyObserver.node, Bytes.toBytes(now - 3600*1000), -1); - - LOG.debug("Set time: "+Bytes.toLong(Bytes.toBytes(now - 3600*1000))); - - // sleep for 1s to give the ZK change a chance to reach the watcher in the observer. - // TODO: Better to wait for the data to be propagated - Thread.sleep(1000); - - long ts = now - 2000; - Put p = new Put(R); - p.add(F, Q, ts, Q); - t.put(p); - p = new Put(R); - p.add(F, Q, ts+1, Q); - t.put(p); - - // these two should be expired but for the override - // (their ts was 2s in the past) - Get g = new Get(R); - g.setMaxVersions(10); - Result r = t.get(g); - // still there? - assertEquals(2, r.size()); - - TEST_UTIL.flush(tableName); - TEST_UTIL.compact(tableName, true); - - g = new Get(R); - g.setMaxVersions(10); - r = t.get(g); - // still there? - assertEquals(2, r.size()); - zk.setData(ZooKeeperScanPolicyObserver.node, Bytes.toBytes(now), -1); - LOG.debug("Set time: "+now); - - TEST_UTIL.compact(tableName, true); - - g = new Get(R); - g.setMaxVersions(10); - r = t.get(g); - // should be gone now - assertEquals(0, r.size()); - t.close(); - } -}