hbase git commit: HBASE-18347 Implement a BufferedMutator for async client

2017-08-24 Thread zhangduo
Repository: hbase
Updated Branches:
  refs/heads/branch-2 45b20da23 -> 1ae9a3901


HBASE-18347 Implement a BufferedMutator for async client


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/1ae9a390
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/1ae9a390
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/1ae9a390

Branch: refs/heads/branch-2
Commit: 1ae9a39011f2ee2dfe013a32a990cff34f6ea428
Parents: 45b20da
Author: zhangduo 
Authored: Mon Aug 21 18:37:26 2017 +0800
Committer: zhangduo 
Committed: Thu Aug 24 16:01:30 2017 +0800

--
 .../hbase/client/AsyncBufferedMutator.java  |  84 +++
 .../client/AsyncBufferedMutatorBuilder.java |  85 +++
 .../client/AsyncBufferedMutatorBuilderImpl.java |  85 +++
 .../hbase/client/AsyncBufferedMutatorImpl.java  | 144 +++
 .../hadoop/hbase/client/AsyncConnection.java|  39 +
 .../client/AsyncConnectionConfiguration.java|   9 ++
 .../hbase/client/AsyncConnectionImpl.java   |  11 ++
 .../hbase/client/TestAsyncBufferMutator.java| 128 +
 8 files changed, 585 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/1ae9a390/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
new file mode 100644
index 000..ad9279b
--- /dev/null
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
@@ -0,0 +1,84 @@
+/**
+ * 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.client;
+
+import java.io.Closeable;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Used to communicate with a single HBase table in batches. Obtain an 
instance from a
+ * {@link AsyncConnection} and call {@link #close()} afterwards.
+ * 
+ * The implementation is required to be thread safe.
+ */
+@InterfaceAudience.Public
+public interface AsyncBufferedMutator extends Closeable {
+
+  /**
+   * Gets the fully qualified table name instance of the table that this
+   * {@code AsyncBufferedMutator} writes to.
+   */
+  TableName getName();
+
+  /**
+   * Returns the {@link org.apache.hadoop.conf.Configuration} object used by 
this instance.
+   * 
+   * The reference returned is not a copy, so any change made to it will 
affect this instance.
+   */
+  Configuration getConfiguration();
+
+  /**
+   * Sends a {@link Mutation} to the table. The mutations will be buffered and 
sent over the wire as
+   * part of a batch. Currently only supports {@link Put} and {@link Delete} 
mutations.
+   * @param mutation The data to send.
+   */
+  CompletableFuture mutate(Mutation mutation);
+
+  /**
+   * Send some {@link Mutation}s to the table. The mutations will be buffered 
and sent over the wire
+   * as part of a batch. There is no guarantee of sending entire content of 
{@code mutations} in a
+   * single batch, the implementations are free to break it up according to 
the write buffer
+   * capacity.
+   * @param mutations The data to send.
+   */
+  List mutate(List mutations);
+
+  /**
+   * Executes all the buffered, asynchronous operations.
+   */
+  void flush();
+
+  /**
+   * Performs a {@link #flush()} and releases any resources held.
+   */
+  @Override
+  void close();
+
+  /**
+   * Returns the maximum size in bytes of the write buffer.
+   * 
+   * The default value comes from the configuration parameter {@code 
hbase.client.write.buffer}.
+   * @return The size of the write buffer in bytes.
+   */
+  long getWriteBufferSize();
+}


hbase git commit: HBASE-18347 Implement a BufferedMutator for async client

2017-08-24 Thread zhangduo
Repository: hbase
Updated Branches:
  refs/heads/master 25ee5f7f8 -> d12eb7a4a


HBASE-18347 Implement a BufferedMutator for async client


Project: http://git-wip-us.apache.org/repos/asf/hbase/repo
Commit: http://git-wip-us.apache.org/repos/asf/hbase/commit/d12eb7a4
Tree: http://git-wip-us.apache.org/repos/asf/hbase/tree/d12eb7a4
Diff: http://git-wip-us.apache.org/repos/asf/hbase/diff/d12eb7a4

Branch: refs/heads/master
Commit: d12eb7a4aae5c2dc7b230bf2a12d2313b93b8ba9
Parents: 25ee5f7
Author: zhangduo 
Authored: Mon Aug 21 18:37:26 2017 +0800
Committer: zhangduo 
Committed: Thu Aug 24 15:55:43 2017 +0800

--
 .../hbase/client/AsyncBufferedMutator.java  |  84 +++
 .../client/AsyncBufferedMutatorBuilder.java |  85 +++
 .../client/AsyncBufferedMutatorBuilderImpl.java |  85 +++
 .../hbase/client/AsyncBufferedMutatorImpl.java  | 144 +++
 .../hadoop/hbase/client/AsyncConnection.java|  39 +
 .../client/AsyncConnectionConfiguration.java|   9 ++
 .../hbase/client/AsyncConnectionImpl.java   |  11 ++
 .../hbase/client/TestAsyncBufferMutator.java| 128 +
 8 files changed, 585 insertions(+)
--


http://git-wip-us.apache.org/repos/asf/hbase/blob/d12eb7a4/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
--
diff --git 
a/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
new file mode 100644
index 000..ad9279b
--- /dev/null
+++ 
b/hbase-client/src/main/java/org/apache/hadoop/hbase/client/AsyncBufferedMutator.java
@@ -0,0 +1,84 @@
+/**
+ * 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.client;
+
+import java.io.Closeable;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hbase.TableName;
+import org.apache.hadoop.hbase.classification.InterfaceAudience;
+
+/**
+ * Used to communicate with a single HBase table in batches. Obtain an 
instance from a
+ * {@link AsyncConnection} and call {@link #close()} afterwards.
+ * 
+ * The implementation is required to be thread safe.
+ */
+@InterfaceAudience.Public
+public interface AsyncBufferedMutator extends Closeable {
+
+  /**
+   * Gets the fully qualified table name instance of the table that this
+   * {@code AsyncBufferedMutator} writes to.
+   */
+  TableName getName();
+
+  /**
+   * Returns the {@link org.apache.hadoop.conf.Configuration} object used by 
this instance.
+   * 
+   * The reference returned is not a copy, so any change made to it will 
affect this instance.
+   */
+  Configuration getConfiguration();
+
+  /**
+   * Sends a {@link Mutation} to the table. The mutations will be buffered and 
sent over the wire as
+   * part of a batch. Currently only supports {@link Put} and {@link Delete} 
mutations.
+   * @param mutation The data to send.
+   */
+  CompletableFuture mutate(Mutation mutation);
+
+  /**
+   * Send some {@link Mutation}s to the table. The mutations will be buffered 
and sent over the wire
+   * as part of a batch. There is no guarantee of sending entire content of 
{@code mutations} in a
+   * single batch, the implementations are free to break it up according to 
the write buffer
+   * capacity.
+   * @param mutations The data to send.
+   */
+  List mutate(List mutations);
+
+  /**
+   * Executes all the buffered, asynchronous operations.
+   */
+  void flush();
+
+  /**
+   * Performs a {@link #flush()} and releases any resources held.
+   */
+  @Override
+  void close();
+
+  /**
+   * Returns the maximum size in bytes of the write buffer.
+   * 
+   * The default value comes from the configuration parameter {@code 
hbase.client.write.buffer}.
+   * @return The size of the write buffer in bytes.
+   */
+  long getWriteBufferSize();
+}