[accumulo] 01/01: Merge branch '1.9'

2018-09-20 Thread busbey
This is an automated email from the ASF dual-hosted git repository.

busbey pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 02f58119a283e2552f0577c732ef8aa0aa5226a1
Merge: 65639f3 d832765
Author: Sean Busbey 
AuthorDate: Thu Sep 20 13:58:44 2018 -0500

Merge branch '1.9'

 .../org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java  | 5 +++--
 .../apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java   | 5 +++--
 2 files changed, 6 insertions(+), 4 deletions(-)




[accumulo] branch master updated (65639f3 -> 02f5811)

2018-09-20 Thread busbey
This is an automated email from the ASF dual-hosted git repository.

busbey pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git.


from 65639f3  Fix performance of crypto (#652)
 add d832765  AccumuloFileOutputFormatITs need to look on local filesystem 
for results written into JUnit provided temp dir. (#654)
 new 02f5811  Merge branch '1.9'

The 1 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java  | 5 +++--
 .../apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java   | 5 +++--
 2 files changed, 6 insertions(+), 4 deletions(-)



[accumulo] branch 1.9 updated: AccumuloFileOutputFormatITs need to look on local filesystem for results written into JUnit provided temp dir. (#654)

2018-09-20 Thread busbey
This is an automated email from the ASF dual-hosted git repository.

busbey pushed a commit to branch 1.9
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/1.9 by this push:
 new d832765  AccumuloFileOutputFormatITs need to look on local filesystem 
for results written into JUnit provided temp dir. (#654)
d832765 is described below

commit d83276577a41e2c9c7cd4f5bd36ce91aa9631875
Author: Sean Busbey 
AuthorDate: Thu Sep 20 13:34:30 2018 -0500

AccumuloFileOutputFormatITs need to look on local filesystem for results 
written into JUnit provided temp dir. (#654)

Before this change the {mapred,mapreduce}.AccumuloFileOutputFormatITs would 
work against a minicluster backed by
Hadoop's LocalFileSystem, but fail if one followed the instructions for 
testing against a stand alone cluster.

The failure is because in the cluster case the FileSystem we get is for the 
HDFS instance underlying the
Accumulo cluster, which can't find the folder JUnit created for our test.
---
 .../org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java  | 5 +++--
 .../apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java   | 5 +++--
 2 files changed, 6 insertions(+), 4 deletions(-)

diff --git 
a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
 
b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
index 7f5fabb..e54d528 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/mapred/AccumuloFileOutputFormatIT.java
@@ -198,8 +198,9 @@ public class AccumuloFileOutputFormatIT extends 
AccumuloClusterHarness {
   Configuration conf = CachedConfiguration.getInstance();
   DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
   FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-  .forFile(files[0].toString(), FileSystem.get(conf), 
conf).withTableConfiguration(acuconf)
-  .build().getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
+  .forFile(files[0].toString(), FileSystem.getLocal(conf), conf)
+  .withTableConfiguration(acuconf).build()
+  .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
   assertNotNull(sample);
 } else {
   assertEquals(0, files.length);
diff --git 
a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
 
b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
index 8b292d0..e299151 100644
--- 
a/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
+++ 
b/test/src/main/java/org/apache/accumulo/test/mapreduce/AccumuloFileOutputFormatIT.java
@@ -212,8 +212,9 @@ public class AccumuloFileOutputFormatIT extends 
AccumuloClusterHarness {
   Configuration conf = CachedConfiguration.getInstance();
   DefaultConfiguration acuconf = DefaultConfiguration.getInstance();
   FileSKVIterator sample = RFileOperations.getInstance().newReaderBuilder()
-  .forFile(files[0].toString(), FileSystem.get(conf), 
conf).withTableConfiguration(acuconf)
-  .build().getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
+  .forFile(files[0].toString(), FileSystem.getLocal(conf), conf)
+  .withTableConfiguration(acuconf).build()
+  .getSample(new SamplerConfigurationImpl(SAMPLER_CONFIG));
   assertNotNull(sample);
 } else {
   assertEquals(0, files.length);



[accumulo] branch master updated: Fix performance of crypto (#652)

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git


The following commit(s) were added to refs/heads/master by this push:
 new 65639f3  Fix performance of crypto (#652)
65639f3 is described below

commit 65639f33c7d34e5278ef042afcc5d441aa3f033d
Author: Mike Miller 
AuthorDate: Thu Sep 20 13:51:37 2018 -0400

Fix performance of crypto (#652)

* Remove use of DiscardCloseOutputStream from WAL encryption
* Override inefficient write method in DiscardCloseOutputStream and 
NoFlushOutputStream
* Changed NoFlushOutputStream to extend FilterOutputStream, which doesn't 
have synchronized methods like DataOutputStream does
---
 .../core/security/crypto/impl/AESCryptoService.java|  8 +---
 .../security/crypto/streams/DiscardCloseOutputStream.java  |  9 +
 .../core/security/crypto/streams/NoFlushOutputStream.java  | 14 --
 3 files changed, 22 insertions(+), 9 deletions(-)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
 
b/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
index d10dbff..09c5648 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/security/crypto/impl/AESCryptoService.java
@@ -448,13 +448,7 @@ public class AESCryptoService implements CryptoService {
   throw new CryptoException("Unable to initialize cipher", e);
 }
 
-CipherOutputStream cos = new CipherOutputStream(new 
DiscardCloseOutputStream(outputStream),
-cipher);
-// Prevent underlying stream from being closed with 
DiscardCloseOutputStream
-// Without this, when the crypto stream is closed (in order to flush 
its last bytes)
-// the underlying RFile stream will *also* be closed, and that's 
undesirable as the
-// cipher
-// stream is closed for every block written.
+CipherOutputStream cos = new CipherOutputStream(outputStream, cipher);
 return new BlockedOutputStream(cos, cipher.getBlockSize(), 1024);
   }
 
diff --git 
a/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/DiscardCloseOutputStream.java
 
b/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/DiscardCloseOutputStream.java
index cf0db70..4870bbc 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/DiscardCloseOutputStream.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/DiscardCloseOutputStream.java
@@ -32,6 +32,15 @@ public class DiscardCloseOutputStream extends 
FilterOutputStream {
 super(out);
   }
 
+  /**
+   * It is very important to override this method!! The underlying method from 
FilterOutputStream
+   * calls write a single byte at a time and will kill performance.
+   */
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+out.write(b, off, len);
+  }
+
   @Override
   public void close() throws IOException {
 // Discard
diff --git 
a/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/NoFlushOutputStream.java
 
b/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/NoFlushOutputStream.java
index 1fa2af7..7edb1bf 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/NoFlushOutputStream.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/security/crypto/streams/NoFlushOutputStream.java
@@ -16,15 +16,25 @@
  */
 package org.apache.accumulo.core.security.crypto.streams;
 
-import java.io.DataOutputStream;
+import java.io.FilterOutputStream;
+import java.io.IOException;
 import java.io.OutputStream;
 
-public class NoFlushOutputStream extends DataOutputStream {
+public class NoFlushOutputStream extends FilterOutputStream {
 
   public NoFlushOutputStream(OutputStream out) {
 super(out);
   }
 
+  /**
+   * It is very important to override this method!! The underlying method from 
FilterOutputStream
+   * calls write a single byte at a time and will kill performance.
+   */
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+out.write(b, off, len);
+  }
+
   @Override
   public void flush() {}
 



[accumulo] 06/09: Replace AccumuloClient.builder() with Accumulo.newClient() #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 905f338d8203b927d6420665ab1fbda5ddeb3e86
Author: Mike Miller 
AuthorDate: Fri Sep 7 11:34:24 2018 -0400

Replace AccumuloClient.builder() with Accumulo.newClient() #636
---
 .../accumulo/core/client/mapred/AbstractInputFormat.java  |  3 ++-
 .../accumulo/core/client/mapred/AccumuloOutputFormat.java |  5 +++--
 .../accumulo/core/client/mapreduce/AbstractInputFormat.java   |  3 ++-
 .../accumulo/core/client/mapreduce/AccumuloOutputFormat.java  |  5 +++--
 .../core/client/mapreduce/lib/impl/ConfiguratorBase.java  | 10 +-
 .../core/client/mapreduce/lib/impl/MapReduceClientOpts.java   |  3 ++-
 .../core/client/mapreduce/lib/impl/ConfiguratorBaseTest.java  |  4 ++--
 .../main/java/org/apache/accumulo/core/cli/ClientOpts.java|  5 +++--
 .../apache/accumulo/core/client/impl/AccumuloClientImpl.java  |  3 ++-
 .../cluster/standalone/StandaloneAccumuloCluster.java |  5 +++--
 .../org/apache/accumulo/minicluster/MiniAccumuloCluster.java  |  3 ++-
 .../accumulo/minicluster/impl/MiniAccumuloClusterImpl.java| 11 ++-
 .../main/java/org/apache/accumulo/server/ServerContext.java   |  3 ++-
 .../src/main/java/org/apache/accumulo/tracer/TraceServer.java |  5 +++--
 shell/src/main/java/org/apache/accumulo/shell/Shell.java  |  3 ++-
 .../harness/conf/StandaloneAccumuloClusterConfiguration.java  |  4 ++--
 .../java/org/apache/accumulo/test/BatchWriterIterator.java|  3 ++-
 .../main/java/org/apache/accumulo/test/IMMLGBenchmark.java|  3 ++-
 .../src/main/java/org/apache/accumulo/test/ShellServerIT.java |  4 ++--
 .../org/apache/accumulo/test/functional/AccumuloClientIT.java |  9 +
 .../apache/accumulo/test/functional/ConfigurableMacBase.java  |  3 ++-
 .../java/org/apache/accumulo/test/functional/ReadWriteIT.java |  3 ++-
 .../apache/accumulo/test/mapred/AccumuloOutputFormatIT.java   |  3 ++-
 23 files changed, 61 insertions(+), 42 deletions(-)

diff --git 
a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
 
b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
index 83377b4..fc8e26b 100644
--- 
a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
+++ 
b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AbstractInputFormat.java
@@ -31,6 +31,7 @@ import java.util.Map;
 import java.util.Random;
 import java.util.concurrent.TimeUnit;
 
+import org.apache.accumulo.core.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -180,7 +181,7 @@ public abstract class AbstractInputFormat implements 
InputFormat {
 if (token instanceof KerberosToken) {
   log.info("Received KerberosToken, attempting to fetch DelegationToken");
   try {
-AccumuloClient conn = 
AccumuloClient.builder().usingClientInfo(getClientInfo(job))
+AccumuloClient conn = 
Accumulo.newClient().usingClientInfo(getClientInfo(job))
 .usingToken(principal, token).build();
 token = conn.securityOperations().getDelegationToken(new 
DelegationTokenConfig());
   } catch (Exception e) {
diff --git 
a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
 
b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
index e58d790..f00206a 100644
--- 
a/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
+++ 
b/client/mapreduce/src/main/java/org/apache/accumulo/core/client/mapred/AccumuloOutputFormat.java
@@ -22,6 +22,7 @@ import java.util.HashSet;
 import java.util.Map.Entry;
 import java.util.Set;
 
+import org.apache.accumulo.core.Accumulo;
 import org.apache.accumulo.core.client.AccumuloClient;
 import org.apache.accumulo.core.client.AccumuloException;
 import org.apache.accumulo.core.client.AccumuloSecurityException;
@@ -426,7 +427,7 @@ public class AccumuloOutputFormat implements 
OutputFormat {
   this.defaultTableName = (tname == null) ? null : new Text(tname);
 
   if (!simulate) {
-this.conn = 
AccumuloClient.builder().usingClientInfo(getClientInfo(job)).build();
+this.conn = 
Accumulo.newClient().usingClientInfo(getClientInfo(job)).build();
 mtbw = conn.createMultiTableBatchWriter(getBatchWriterOptions(job));
   }
 }
@@ -563,7 +564,7 @@ public class AccumuloOutputFormat implements 
OutputFormat {
   throw new IOException("Connector info has not been set.");
 try {
   // if the instance isn't configured, it will complain here
-  AccumuloClient c = 
AccumuloClient.builder().usingClientInfo(getClientInfo(job)).build(

[accumulo] 01/09: Created AccumuloClient from Connector #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 31a2de7f001ef3c5572fb17bf9f0db1c2802289f
Author: Mike Miller 
AuthorDate: Thu Sep 6 16:32:08 2018 -0400

Created AccumuloClient from Connector #636
---
 .../accumulo/core/client/AccumuloClient.java   | 532 +
 .../core/client/impl/AccumuloClientImpl.java   | 449 +
 .../accumulo/test/functional/AccumuloClientIT.java |  95 
 3 files changed, 1076 insertions(+)

diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java 
b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
new file mode 100644
index 000..1fe865a
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -0,0 +1,532 @@
+/*
+ * 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.accumulo.core.client;
+
+import java.util.Properties;
+
+import org.apache.accumulo.core.client.admin.InstanceOperations;
+import org.apache.accumulo.core.client.admin.NamespaceOperations;
+import org.apache.accumulo.core.client.admin.ReplicationOperations;
+import org.apache.accumulo.core.client.admin.SecurityOperations;
+import org.apache.accumulo.core.client.admin.TableOperations;
+import org.apache.accumulo.core.client.impl.AccumuloClientImpl;
+import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
+import org.apache.accumulo.core.security.Authorizations;
+
+/**
+ * Client connection to an Accumulo instance. Allows the user to request a 
scanner, deleter or
+ * writer for the instance as well as various objects that permit 
administrative operations.
+ * Enforces security on the client side with by requiring user credentials.
+ *
+ * Supports fluent API. Various options can be provided to {@link #builder()} 
and when finished a
+ * call to build() will return the AccumuloClient object. For example:
+ *
+ * {@code AccumuloClient.builder().forInstance(instanceName, zookeepers)
+ * .usingPassword(user, password).withZkTimeout(1234).build();}
+ *
+ * @since 2.0.0
+ */
+public interface AccumuloClient {
+
+  /**
+   * Factory method to create a BatchScanner connected to Accumulo.
+   *
+   * @param tableName
+   *  the name of the table to query
+   * @param authorizations
+   *  A set of authorization labels that will be checked against the 
column visibility of
+   *  each key in order to filter data. The authorizations passed in 
must be a subset of the
+   *  accumulo user's set of authorizations. If the accumulo user has 
authorizations (A1,
+   *  A2) and authorizations (A2, A3) are passed, then an exception 
will be thrown.
+   * @param numQueryThreads
+   *  the number of concurrent threads to spawn for querying
+   *
+   * @return BatchScanner object for configuring and querying
+   * @throws TableNotFoundException
+   *   when the specified table doesn't exist
+   */
+  public BatchScanner createBatchScanner(String tableName, Authorizations 
authorizations,
+  int numQueryThreads) throws TableNotFoundException;
+
+  /**
+   * Factory method to create a BatchScanner connected to Accumulo. This 
method uses the number of
+   * query threads configured when AccumuloClient was created. If none were 
configured, defaults
+   * will be used.
+   *
+   * @param tableName
+   *  the name of the table to query
+   * @param authorizations
+   *  A set of authorization labels that will be checked against the 
column visibility of
+   *  each key in order to filter data. The authorizations passed in 
must be a subset of the
+   *  accumulo user's set of authorizations. If the accumulo user has 
authorizations (A1,
+   *  A2) and authorizations (A2, A3) are passed, then an exception 
will be thrown.
+   *
+   * @return BatchScanner object for configuring and querying
+   * @throws TableNotFoundException
+   *   when the specified table doesn't exist
+   */
+  public BatchScanner createBatchScanner(String tableName, Authorizations 
authorizations)
+  throws Table

[accumulo] 07/09: Replace getConnector() with getClient() #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 3f828384fe952fc0dc8d74fba630d27dc50fcc1e
Author: Mike Miller 
AuthorDate: Tue Sep 11 14:03:21 2018 -0400

Replace getConnector() with getClient() #636
---
 .../core/client/mapred/AbstractInputFormat.java|  4 +-
 .../core/client/mapreduce/AbstractInputFormat.java |  4 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java   |  4 +-
 .../mapreduce/lib/impl/InputConfigurator.java  |  3 +-
 .../org/apache/accumulo/core/cli/ClientOpts.java   |  2 +-
 .../core/client/impl/AccumuloClientImpl.java   |  2 +-
 .../accumulo/core/client/impl/BulkImport.java  |  2 +-
 .../accumulo/core/client/impl/ClientContext.java   |  4 +-
 .../core/client/impl/ClientInfoFactory.java|  2 +-
 .../accumulo/core/client/impl/OfflineIterator.java |  2 +-
 .../client/impl/ReplicationOperationsImpl.java |  8 ++--
 .../core/client/impl/TableOperationsImpl.java  |  6 +--
 .../accumulo/core/metadata/MetadataServicer.java   |  2 +-
 .../core/metadata/TableMetadataServicer.java   |  2 +-
 .../core/metadata/schema/MetadataScanner.java  |  2 +-
 .../org/apache/accumulo/core/summary/Gatherer.java |  2 +-
 .../java/org/apache/accumulo/core/util/Merge.java  |  2 +-
 .../core/client/impl/TableOperationsImplTest.java  |  2 +-
 .../core/metadata/MetadataServicerTest.java|  2 +-
 .../org/apache/accumulo/server/ServerContext.java  |  4 +-
 .../server/client/ClientServiceHandler.java|  2 +-
 .../server/master/balancer/GroupBalancer.java  |  2 +-
 .../server/master/balancer/TableLoadBalancer.java  |  2 +-
 .../server/master/state/MetaDataStateStore.java|  2 +-
 .../server/master/state/MetaDataTableScanner.java  |  2 +-
 .../accumulo/server/problems/ProblemReports.java   |  4 +-
 .../server/replication/ReplicaSystemHelper.java|  2 +-
 .../server/replication/ReplicationUtil.java|  6 +--
 .../org/apache/accumulo/server/util/Admin.java |  8 ++--
 .../server/util/CheckForMetadataProblems.java  |  2 +-
 .../accumulo/server/util/ListVolumesUsed.java  |  2 +-
 .../apache/accumulo/server/util/LocalityCheck.java |  2 +-
 .../accumulo/server/util/MetadataTableUtil.java|  4 +-
 .../apache/accumulo/server/util/RandomWriter.java  |  2 +-
 .../accumulo/server/util/RandomizeVolumes.java |  4 +-
 .../server/util/RemoveEntriesForMissingFiles.java  |  2 +-
 .../accumulo/server/util/ReplicationTableUtil.java |  2 +-
 .../accumulo/server/util/TableDiskUsage.java   |  2 +-
 .../server/util/VerifyTabletAssignments.java   |  2 +-
 .../accumulo/gc/GarbageCollectWriteAheadLogs.java  |  2 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java | 16 +++
 .../replication/CloseWriteAheadLogReferences.java  |  2 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java   |  6 +--
 .../java/org/apache/accumulo/master/Master.java|  6 +--
 .../master/MasterClientServiceHandler.java |  4 +-
 .../apache/accumulo/master/TabletGroupWatcher.java | 12 ++---
 .../DistributedWorkQueueWorkAssigner.java  |  2 +-
 .../master/replication/ReplicationDriver.java  |  2 +-
 .../accumulo/master/replication/WorkDriver.java|  4 +-
 .../apache/accumulo/master/state/MergeStats.java   |  2 +-
 .../apache/accumulo/master/tableOps/CleanUp.java   |  4 +-
 .../accumulo/master/tableOps/CompactionDriver.java |  2 +-
 .../accumulo/master/tableOps/PopulateMetadata.java |  2 +-
 .../master/tableOps/PopulateMetadataTable.java |  2 +-
 .../accumulo/master/tableOps/WriteExportFiles.java |  6 +--
 .../tableOps/bulkVer1/CleanUpBulkImport.java   |  2 +-
 .../master/tableOps/bulkVer1/CopyFailed.java   |  2 +-
 .../tableOps/bulkVer2/CleanUpBulkImport.java   |  2 +-
 .../master/tableOps/bulkVer2/LoadFiles.java|  2 +-
 .../java/org/apache/accumulo/monitor/Monitor.java  |  2 +-
 .../rest/replication/ReplicationResource.java  |  2 +-
 .../monitor/rest/trace/TracesResource.java |  2 +-
 .../java/org/apache/accumulo/tracer/TraceDump.java |  4 +-
 .../apache/accumulo/tracer/TraceTableStats.java|  2 +-
 .../BatchWriterReplicationReplayer.java|  2 +-
 .../tserver/replication/ReplicationProcessor.java  |  2 +-
 .../BatchWriterReplicationReplayerTest.java|  2 +-
 .../accumulo/harness/AccumuloClusterHarness.java   |  2 +-
 .../apache/accumulo/test/QueryMetadataTable.java   |  2 +-
 .../org/apache/accumulo/test/ShellServerIT.java| 34 +++---
 .../org/apache/accumulo/test/TestBinaryRows.java   |  2 +-
 .../java/org/apache/accumulo/test/TestIngest.java  |  2 +-
 .../apache/accumulo/test/TestMultiTableIngest.java |  2 +-
 .../apache/accumulo/test/TestRandomDeletes.java|  4 +-
 .../org/apache/accumulo/test/VerifyIngest.java |  2 +-
 .../accumulo/test/functional/AccumuloClientIT.java | 52 +++---
 .../test/functional/CreateInitialSplitsIT.java   

[accumulo] branch master updated (4b7428f -> 4043c72)

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a change to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git.


from 4b7428f  Merge branch '1.9'
 new 31a2de7  Created AccumuloClient from Connector #636
 new 79b3468  Deprecate Connector #636
 new 3a6e76d  Replaced uses of Connector with AccumuloClient #636
 new 18287e7  Fix javadoc #636
 new 486b799  Create Accumulo class as main entry point #636
 new 905f338  Replace AccumuloClient.builder() with Accumulo.newClient() 
#636
 new 3f82838  Replace getConnector() with getClient() #636
 new 1d1a2a5  Updates from pull request #636
 new 4043c72  Merge branch 'accumulo-client' Closes #636

The 9 revisions listed above as "new" are entirely new to this
repository and will be described in separate emails.  The revisions
listed as "add" were already present in the repository and have only
been added to this reference.


Summary of changes:
 .../core/client/mapred/AbstractInputFormat.java|  11 +-
 .../core/client/mapred/AccumuloOutputFormat.java   |   9 +-
 .../core/client/mapreduce/AbstractInputFormat.java |  11 +-
 .../client/mapreduce/AccumuloOutputFormat.java |   9 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java   |  14 +-
 .../mapreduce/lib/impl/InputConfigurator.java  |   7 +-
 .../mapreduce/lib/impl/MapReduceClientOpts.java|   5 +-
 .../mapreduce/lib/impl/ConfiguratorBaseTest.java   |   4 +-
 .../java/org/apache/accumulo/core/Accumulo.java|  35 +--
 .../org/apache/accumulo/core/cli/ClientOpts.java   |  15 +-
 .../client/{Connector.java => AccumuloClient.java} | 228 
 .../accumulo/core/client/ClientConfiguration.java  |   3 +-
 .../apache/accumulo/core/client/ClientInfo.java|   3 +-
 .../org/apache/accumulo/core/client/Connector.java | 299 +
 .../org/apache/accumulo/core/client/Instance.java  |   3 +-
 .../accumulo/core/client/ZooKeeperInstance.java|  10 +-
 ...{ConnectorImpl.java => AccumuloClientImpl.java} |  22 +-
 .../accumulo/core/client/impl/BulkImport.java  |   2 +-
 .../accumulo/core/client/impl/ClientContext.java   |  10 +-
 .../core/client/impl/ClientInfoFactory.java|   6 +-
 .../accumulo/core/client/impl/Credentials.java |   6 +-
 .../accumulo/core/client/impl/OfflineIterator.java |   6 +-
 .../client/impl/ReplicationOperationsImpl.java |  12 +-
 .../core/client/impl/TableOperationsImpl.java  |   6 +-
 .../accumulo/core/metadata/MetadataServicer.java   |   2 +-
 .../core/metadata/TableMetadataServicer.java   |   2 +-
 .../core/metadata/schema/MetadataScanner.java  |  10 +-
 .../core/replication/ReplicationTable.java |  14 +-
 .../org/apache/accumulo/core/summary/Gatherer.java |   2 +-
 .../java/org/apache/accumulo/core/util/Merge.java  |  16 +-
 .../core/client/impl/TableOperationsImplTest.java  |  13 +-
 .../core/metadata/MetadataServicerTest.java|   6 +-
 .../org/apache/accumulo/core/util/MergeTest.java   |   8 +-
 .../apache/accumulo/cluster/AccumuloCluster.java   |  11 +
 .../standalone/StandaloneAccumuloCluster.java  |  11 +-
 .../accumulo/minicluster/MiniAccumuloCluster.java  |  18 +-
 .../minicluster/impl/MiniAccumuloClusterImpl.java  |  14 +-
 .../impl/MiniAccumuloClusterImplTest.java  |   4 +-
 .../org/apache/accumulo/proxy/ProxyServer.java |  26 +-
 .../org/apache/accumulo/server/ServerContext.java  |  14 +-
 .../server/client/ClientServiceHandler.java|   2 +-
 .../server/master/balancer/GroupBalancer.java  |   2 +-
 .../server/master/balancer/TableLoadBalancer.java  |   2 +-
 .../server/master/state/MetaDataStateStore.java|   2 +-
 .../server/master/state/MetaDataTableScanner.java  |   6 +-
 .../accumulo/server/problems/ProblemReports.java   |  11 +-
 .../replication/PrintReplicationRecords.java   |   6 +-
 .../server/replication/ReplicaSystemHelper.java|   2 +-
 .../server/replication/ReplicationUtil.java|  12 +-
 .../accumulo/server/replication/WorkAssigner.java  |   4 +-
 .../org/apache/accumulo/server/util/Admin.java |  59 ++--
 .../server/util/CheckForMetadataProblems.java  |   2 +-
 .../accumulo/server/util/ListVolumesUsed.java  |   2 +-
 .../apache/accumulo/server/util/LocalityCheck.java |   6 +-
 .../accumulo/server/util/MetadataTableUtil.java|  18 +-
 .../apache/accumulo/server/util/RandomWriter.java  |   6 +-
 .../accumulo/server/util/RandomizeVolumes.java |  10 +-
 .../server/util/RemoveEntriesForMissingFiles.java  |   8 +-
 .../accumulo/server/util/ReplicationTableUtil.java |   8 +-
 .../accumulo/server/util/TableDiskUsage.java   |  15 +-
 .../server/util/VerifyTabletAssignments.java   |   4 +-
 .../server/security/SystemCredentialsTest.java |   6 +-
 .../server/util/ReplicationTableUtilTest.java  |   4 +-
 .../accumulo/gc/GarbageCollectWriteAheadLogs.java  |   6 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |  22 +-
 ..

[accumulo] 05/09: Create Accumulo class as main entry point #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 486b79981c9a91a0852b0f2ff7402c91f37a17be
Author: Mike Miller 
AuthorDate: Fri Sep 7 11:34:02 2018 -0400

Create Accumulo class as main entry point #636
---
 .../java/org/apache/accumulo/core/Accumulo.java| 38 ++
 .../accumulo/core/client/AccumuloClient.java   | 12 +--
 2 files changed, 39 insertions(+), 11 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/Accumulo.java 
b/core/src/main/java/org/apache/accumulo/core/Accumulo.java
new file mode 100644
index 000..f0c8009
--- /dev/null
+++ b/core/src/main/java/org/apache/accumulo/core/Accumulo.java
@@ -0,0 +1,38 @@
+/*
+ * 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.accumulo.core;
+
+import org.apache.accumulo.core.client.AccumuloClient;
+import org.apache.accumulo.core.client.impl.AccumuloClientImpl;
+
+/**
+ * The main entry point for Accumulo public API.
+ */
+public final class Accumulo {
+
+  private Accumulo() {}
+
+  /**
+   * Create an Accumulo client builder, used to construct a client.
+   *
+   * @return a builder object for Accumulo clients
+   */
+  public static AccumuloClient.ClientInfoOptions newClient() {
+return new AccumuloClientImpl.AccumuloClientBuilderImpl();
+  }
+
+}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java 
b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
index 1fe865a..2676608 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -23,7 +23,6 @@ import 
org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.impl.AccumuloClientImpl;
 import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.Authorizations;
 
@@ -35,7 +34,7 @@ import org.apache.accumulo.core.security.Authorizations;
  * Supports fluent API. Various options can be provided to {@link #builder()} 
and when finished a
  * call to build() will return the AccumuloClient object. For example:
  *
- * {@code AccumuloClient.builder().forInstance(instanceName, zookeepers)
+ * {@code Accumulo.newClient().forInstance(instanceName, zookeepers)
  * .usingPassword(user, password).withZkTimeout(1234).build();}
  *
  * @since 2.0.0
@@ -520,13 +519,4 @@ public interface AccumuloClient {
   public interface FromOptions extends ConnectionOptions, PropertyOptions, 
AuthenticationArgs {
 
   }
-
-  /**
-   * Creates builder for AccumuloClient.
-   *
-   * @return this builder
-   */
-  public static ClientInfoOptions builder() {
-return new AccumuloClientImpl.AccumuloClientBuilderImpl();
-  }
 }



[accumulo] 08/09: Updates from pull request #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 1d1a2a5d0c447ef617cdfabb0624698b8de58500
Author: Mike Miller 
AuthorDate: Tue Sep 11 14:54:57 2018 -0400

Updates from pull request #636
---
 core/src/main/java/org/apache/accumulo/core/Accumulo.java | 7 ++-
 .../main/java/org/apache/accumulo/core/client/AccumuloClient.java | 5 +++--
 .../java/org/apache/accumulo/core/client/ClientConfiguration.java | 3 ++-
 .../src/main/java/org/apache/accumulo/core/client/ClientInfo.java | 3 ++-
 core/src/main/java/org/apache/accumulo/core/client/Connector.java | 3 ++-
 core/src/main/java/org/apache/accumulo/core/client/Instance.java  | 3 ++-
 .../java/org/apache/accumulo/core/client/ZooKeeperInstance.java   | 6 --
 .../java/org/apache/accumulo/minicluster/MiniAccumuloCluster.java | 5 +++--
 .../minicluster/MiniAccumuloClusterExistingZooKeepersTest.java| 4 ++--
 .../accumulo/minicluster/MiniAccumuloClusterStartStopTest.java| 4 ++--
 .../org/apache/accumulo/minicluster/MiniAccumuloClusterTest.java  | 8 
 .../java/org/apache/accumulo/harness/AccumuloClusterHarness.java  | 2 +-
 .../main/java/org/apache/accumulo/test/functional/BulkLoadIT.java | 1 +
 .../apache/accumulo/test/functional/CreateInitialSplitsIT.java| 2 +-
 14 files changed, 35 insertions(+), 21 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/Accumulo.java 
b/core/src/main/java/org/apache/accumulo/core/Accumulo.java
index f0c8009..67d8df6 100644
--- a/core/src/main/java/org/apache/accumulo/core/Accumulo.java
+++ b/core/src/main/java/org/apache/accumulo/core/Accumulo.java
@@ -21,13 +21,18 @@ import 
org.apache.accumulo.core.client.impl.AccumuloClientImpl;
 
 /**
  * The main entry point for Accumulo public API.
+ *
+ * @since 2.0.0
  */
 public final class Accumulo {
 
   private Accumulo() {}
 
   /**
-   * Create an Accumulo client builder, used to construct a client.
+   * Create an Accumulo client builder, used to construct a client. For 
example:
+   *
+   * {@code Accumulo.newClient().forInstance(instanceName, zookeepers)
+   * .usingPassword(user, password).withZkTimeout(1234).build();}
*
* @return a builder object for Accumulo clients
*/
diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java 
b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
index 2676608..7792fb0 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/AccumuloClient.java
@@ -18,6 +18,7 @@ package org.apache.accumulo.core.client;
 
 import java.util.Properties;
 
+import org.apache.accumulo.core.Accumulo;
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.ReplicationOperations;
@@ -31,8 +32,8 @@ import org.apache.accumulo.core.security.Authorizations;
  * writer for the instance as well as various objects that permit 
administrative operations.
  * Enforces security on the client side with by requiring user credentials.
  *
- * Supports fluent API. Various options can be provided to {@link #builder()} 
and when finished a
- * call to build() will return the AccumuloClient object. For example:
+ * Supports fluent API. Various options can be provided to {@link 
Accumulo#newClient()} and when
+ * finished a call to build() will return the AccumuloClient object. For 
example:
  *
  * {@code Accumulo.newClient().forInstance(instanceName, zookeepers)
  * .usingPassword(user, password).withZkTimeout(1234).build();}
diff --git 
a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java 
b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
index bf68155..09b4577 100644
--- 
a/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
+++ 
b/core/src/main/java/org/apache/accumulo/core/client/ClientConfiguration.java
@@ -30,6 +30,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.UUID;
 
+import org.apache.accumulo.core.Accumulo;
 import org.apache.accumulo.core.conf.Property;
 import org.apache.accumulo.core.conf.PropertyType;
 import org.apache.commons.configuration.CompositeConfiguration;
@@ -45,7 +46,7 @@ import org.slf4j.LoggerFactory;
  * setting them.
  *
  * @since 1.6.0
- * @deprecated since 2.0.0, replaced {@link AccumuloClient#builder()}
+ * @deprecated since 2.0.0, replaced by {@link Accumulo#newClient()}
  */
 @Deprecated
 public class ClientConfiguration {
diff --git a/core/src/main/java/org/apache/accumulo/core/client/ClientInfo.java 
b/core/src/main/java/org/apache/accumulo/core/client/ClientInfo.java
index c865518..df78b22 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/ClientInfo.java
+++ b/core/src/main/java/o

[accumulo] 09/09: Merge branch 'accumulo-client' Closes #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 4043c72ab2ad5ba7e984f9ba5cfd961eec66b9bd
Merge: 4b7428f 1d1a2a5
Author: Mike Miller 
AuthorDate: Thu Sep 20 13:47:10 2018 -0400

Merge branch 'accumulo-client' Closes #636

 .../core/client/mapred/AbstractInputFormat.java|  11 +-
 .../core/client/mapred/AccumuloOutputFormat.java   |   9 +-
 .../core/client/mapreduce/AbstractInputFormat.java |  11 +-
 .../client/mapreduce/AccumuloOutputFormat.java |   9 +-
 .../mapreduce/lib/impl/ConfiguratorBase.java   |  14 +-
 .../mapreduce/lib/impl/InputConfigurator.java  |   7 +-
 .../mapreduce/lib/impl/MapReduceClientOpts.java|   5 +-
 .../mapreduce/lib/impl/ConfiguratorBaseTest.java   |   4 +-
 .../java/org/apache/accumulo/core/Accumulo.java|  32 +--
 .../org/apache/accumulo/core/cli/ClientOpts.java   |  15 +-
 .../client/{Connector.java => AccumuloClient.java} | 228 
 .../accumulo/core/client/ClientConfiguration.java  |   3 +-
 .../apache/accumulo/core/client/ClientInfo.java|   3 +-
 .../org/apache/accumulo/core/client/Connector.java | 299 +
 .../org/apache/accumulo/core/client/Instance.java  |   3 +-
 .../accumulo/core/client/ZooKeeperInstance.java|  10 +-
 ...{ConnectorImpl.java => AccumuloClientImpl.java} |  22 +-
 .../accumulo/core/client/impl/BulkImport.java  |   2 +-
 .../accumulo/core/client/impl/ClientContext.java   |  10 +-
 .../core/client/impl/ClientInfoFactory.java|   6 +-
 .../accumulo/core/client/impl/Credentials.java |   6 +-
 .../accumulo/core/client/impl/OfflineIterator.java |   6 +-
 .../client/impl/ReplicationOperationsImpl.java |  12 +-
 .../core/client/impl/TableOperationsImpl.java  |   6 +-
 .../accumulo/core/metadata/MetadataServicer.java   |   2 +-
 .../core/metadata/TableMetadataServicer.java   |   2 +-
 .../core/metadata/schema/MetadataScanner.java  |  10 +-
 .../core/replication/ReplicationTable.java |  14 +-
 .../org/apache/accumulo/core/summary/Gatherer.java |   2 +-
 .../java/org/apache/accumulo/core/util/Merge.java  |  16 +-
 .../core/client/impl/TableOperationsImplTest.java  |  13 +-
 .../core/metadata/MetadataServicerTest.java|   6 +-
 .../org/apache/accumulo/core/util/MergeTest.java   |   8 +-
 .../apache/accumulo/cluster/AccumuloCluster.java   |  11 +
 .../standalone/StandaloneAccumuloCluster.java  |  11 +-
 .../accumulo/minicluster/MiniAccumuloCluster.java  |  18 +-
 .../minicluster/impl/MiniAccumuloClusterImpl.java  |  14 +-
 .../impl/MiniAccumuloClusterImplTest.java  |   4 +-
 .../org/apache/accumulo/proxy/ProxyServer.java |  26 +-
 .../org/apache/accumulo/server/ServerContext.java  |  14 +-
 .../server/client/ClientServiceHandler.java|   2 +-
 .../server/master/balancer/GroupBalancer.java  |   2 +-
 .../server/master/balancer/TableLoadBalancer.java  |   2 +-
 .../server/master/state/MetaDataStateStore.java|   2 +-
 .../server/master/state/MetaDataTableScanner.java  |   6 +-
 .../accumulo/server/problems/ProblemReports.java   |  11 +-
 .../replication/PrintReplicationRecords.java   |   6 +-
 .../server/replication/ReplicaSystemHelper.java|   2 +-
 .../server/replication/ReplicationUtil.java|  12 +-
 .../accumulo/server/replication/WorkAssigner.java  |   4 +-
 .../org/apache/accumulo/server/util/Admin.java |  59 ++--
 .../server/util/CheckForMetadataProblems.java  |   2 +-
 .../accumulo/server/util/ListVolumesUsed.java  |   2 +-
 .../apache/accumulo/server/util/LocalityCheck.java |   6 +-
 .../accumulo/server/util/MetadataTableUtil.java|  18 +-
 .../apache/accumulo/server/util/RandomWriter.java  |   6 +-
 .../accumulo/server/util/RandomizeVolumes.java |  10 +-
 .../server/util/RemoveEntriesForMissingFiles.java  |   8 +-
 .../accumulo/server/util/ReplicationTableUtil.java |   8 +-
 .../accumulo/server/util/TableDiskUsage.java   |  15 +-
 .../server/util/VerifyTabletAssignments.java   |   4 +-
 .../server/security/SystemCredentialsTest.java |   6 +-
 .../server/util/ReplicationTableUtilTest.java  |   4 +-
 .../accumulo/gc/GarbageCollectWriteAheadLogs.java  |   6 +-
 .../apache/accumulo/gc/SimpleGarbageCollector.java |  22 +-
 .../replication/CloseWriteAheadLogReferences.java  |   8 +-
 .../gc/GarbageCollectWriteAheadLogsTest.java   |  14 +-
 .../java/org/apache/accumulo/master/Master.java|  12 +-
 .../master/MasterClientServiceHandler.java |   8 +-
 .../apache/accumulo/master/TabletGroupWatcher.java |  16 +-
 .../DistributedWorkQueueWorkAssigner.java  |  10 +-
 .../master/replication/FinishedWorkUpdater.java|   6 +-
 .../RemoveCompleteReplicationRecords.java  |   8 +-
 .../master/replication/ReplicationDriver.java  |   6 +-
 .../master/replication/SequentialWorkAssigner.java |   4 +-
 .../accumulo/master/replicatio

[accumulo] 02/09: Deprecate Connector #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 79b346873f2072dc30063934c2027ddb0418e66a
Author: Mike Miller 
AuthorDate: Thu Sep 6 16:36:55 2018 -0400

Deprecate Connector #636

* Created Connector.from() as a bridge between it and AccumuloClient
* Moved new builder API to AccumuloClient
---
 .../org/apache/accumulo/core/client/Connector.java | 298 +
 1 file changed, 7 insertions(+), 291 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java 
b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index b4770d1..639fa67 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -16,15 +16,11 @@
  */
 package org.apache.accumulo.core.client;
 
-import java.util.Properties;
-
 import org.apache.accumulo.core.client.admin.InstanceOperations;
 import org.apache.accumulo.core.client.admin.NamespaceOperations;
 import org.apache.accumulo.core.client.admin.ReplicationOperations;
 import org.apache.accumulo.core.client.admin.SecurityOperations;
 import org.apache.accumulo.core.client.admin.TableOperations;
-import org.apache.accumulo.core.client.impl.ConnectorImpl;
-import org.apache.accumulo.core.client.security.tokens.AuthenticationToken;
 import org.apache.accumulo.core.security.Authorizations;
 
 /**
@@ -33,6 +29,8 @@ import org.apache.accumulo.core.security.Authorizations;
  *
  * The Connector enforces security on the client side by forcing all API calls 
to be accompanied by
  * user credentials.
+ *
+ * @deprecated For backwards compatibility only, use {@link AccumuloClient}
  */
 public abstract class Connector {
 
@@ -168,7 +166,7 @@ public abstract class Connector {
* @return BatchWriter object for configuring and writing data to
* @throws TableNotFoundException
*   when the specified table doesn't exist
-   * @deprecated since 1.5.0; Use {@link #createBatchWriter(String, 
BatchWriterConfig)} instead.
+   * @deprecated since 1.5.0; Use {@link createBatchWriter(String, 
BatchWriterConfig)} instead.
*/
   @Deprecated
   public abstract BatchWriter createBatchWriter(String tableName, long 
maxMemory, long maxLatency,
@@ -343,293 +341,11 @@ public abstract class Connector {
   public abstract ReplicationOperations replicationOperations();
 
   /**
-   * @return {@link ClientInfo} which contains information about client 
connection to Accumulo
-   * @since 2.0.0
-   */
-  public abstract ClientInfo info();
-
-  /**
-   * Change user
-   *
-   * @param principal
-   *  Principal/username
-   * @param token
-   *  Authentication token
-   * @return {@link Connector} for new user
-   * @since 2.0.0
-   */
-  public abstract Connector changeUser(String principal, AuthenticationToken 
token)
-  throws AccumuloSecurityException, AccumuloException;
-
-  /**
-   * Builds ClientInfo after all options have been specified
-   *
-   * @since 2.0.0
-   */
-  public interface ClientInfoFactory {
-
-/**
- * Builds ClientInfo after all options have been specified
- *
- * @return ClientInfo
- */
-ClientInfo info();
-  }
-
-  /**
-   * Builds Connector
-   *
-   * @since 2.0.0
-   */
-  public interface ConnectorFactory extends ClientInfoFactory {
-
-/**
- * Builds Connector after all options have been specified
- *
- * @return Connector
- */
-Connector build() throws AccumuloException, AccumuloSecurityException;
-
-  }
-
-  /**
-   * Builder method for setting Accumulo instance and zookeepers
-   *
-   * @since 2.0.0
-   */
-  public interface InstanceArgs {
-AuthenticationArgs forInstance(String instanceName, String zookeepers);
-  }
-
-  /**
-   * Builder methods for creating Connector using properties
-   *
-   * @since 2.0.0
-   */
-  public interface PropertyOptions extends InstanceArgs {
-
-/**
- * Build using properties file. An example properties file can be found at
- * conf/accumulo-client.properties in the Accumulo tarball distribution.
- *
- * @param propertiesFile
- *  Path to properties file
- * @return this builder
- */
-ConnectorFactory usingProperties(String propertiesFile);
-
-/**
- * Build using Java properties object. A list of available properties can 
be found in the
- * documentation on the project website (http://accumulo.apache.org) under 
'Development' ->
- * 'Client Properties'
- *
- * @param properties
- *  Properties object
- * @return this builder
- */
-ConnectorFactory usingProperties(Properties properties);
-  }
-
-  public interface ClientInfoOptions extends PropertyOptions {
-
-/**
- * Build using Accumulo client information
- *
- * @param clientInfo
- *  ClientInfo obje

[accumulo] 04/09: Fix javadoc #636

2018-09-20 Thread mmiller
This is an automated email from the ASF dual-hosted git repository.

mmiller pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/accumulo.git

commit 18287e73b95d1cfba6cfa7d6f193e149e90e1470
Author: Mike Miller 
AuthorDate: Thu Sep 6 17:02:25 2018 -0400

Fix javadoc #636
---
 core/src/main/java/org/apache/accumulo/core/client/Connector.java | 4 ++--
 .../src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java| 2 +-
 2 files changed, 3 insertions(+), 3 deletions(-)

diff --git a/core/src/main/java/org/apache/accumulo/core/client/Connector.java 
b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
index 639fa67..904a483 100644
--- a/core/src/main/java/org/apache/accumulo/core/client/Connector.java
+++ b/core/src/main/java/org/apache/accumulo/core/client/Connector.java
@@ -166,7 +166,7 @@ public abstract class Connector {
* @return BatchWriter object for configuring and writing data to
* @throws TableNotFoundException
*   when the specified table doesn't exist
-   * @deprecated since 1.5.0; Use {@link createBatchWriter(String, 
BatchWriterConfig)} instead.
+   * @deprecated since 1.5.0; Use {@link #createBatchWriter(String, 
BatchWriterConfig)} instead.
*/
   @Deprecated
   public abstract BatchWriter createBatchWriter(String tableName, long 
maxMemory, long maxLatency,
@@ -283,7 +283,7 @@ public abstract class Connector {
* Accessor method for internal instance object.
*
* @return the internal instance object
-   * @deprecated since 2.0.0, use {@link #info()} instead
+   * @deprecated since 2.0.0, use {@link AccumuloClient#info()} instead
*/
   @Deprecated
   public abstract Instance getInstance();
diff --git 
a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java 
b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
index 4f660e2..81107d1 100644
--- a/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
+++ b/minicluster/src/main/java/org/apache/accumulo/cluster/AccumuloCluster.java
@@ -56,7 +56,7 @@ public interface AccumuloCluster {
   /**
* Utility method to get a connector to the cluster.
*
-   * @deprecated since 2.0.0, replaced by {{@link #getAccumuloClient()}}
+   * @deprecated since 2.0.0, replaced by {@link #getAccumuloClient(String, 
AuthenticationToken)}
*/
   Connector getConnector(String user, AuthenticationToken token)
   throws AccumuloException, AccumuloSecurityException;