Return-Path: X-Original-To: apmail-accumulo-commits-archive@www.apache.org Delivered-To: apmail-accumulo-commits-archive@www.apache.org Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by minotaur.apache.org (Postfix) with SMTP id 95EEB17D25 for ; Sat, 11 Oct 2014 01:41:37 +0000 (UTC) Received: (qmail 34489 invoked by uid 500); 11 Oct 2014 01:41:37 -0000 Delivered-To: apmail-accumulo-commits-archive@accumulo.apache.org Received: (qmail 34453 invoked by uid 500); 11 Oct 2014 01:41:37 -0000 Mailing-List: contact commits-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: dev@accumulo.apache.org Delivered-To: mailing list commits@accumulo.apache.org Received: (qmail 34444 invoked by uid 99); 11 Oct 2014 01:41:37 -0000 Received: from tyr.zones.apache.org (HELO tyr.zones.apache.org) (140.211.11.114) by apache.org (qpsmtpd/0.29) with ESMTP; Sat, 11 Oct 2014 01:41:37 +0000 Received: by tyr.zones.apache.org (Postfix, from userid 65534) id 3494A81E7BE; Sat, 11 Oct 2014 01:41:37 +0000 (UTC) Content-Type: text/plain; charset="us-ascii" MIME-Version: 1.0 Content-Transfer-Encoding: 7bit From: ctubbsii@apache.org To: commits@accumulo.apache.org Message-Id: <2cbf0dbe199a4c2fa95e1a4a733697e9@git.apache.org> X-Mailer: ASF-Git Admin Mailer Subject: git commit: ACCUMULO-3197 Remove deprecated Instance.getConfiguration() Date: Sat, 11 Oct 2014 01:41:37 +0000 (UTC) Repository: accumulo Updated Branches: refs/heads/master 3fba0b543 -> 8fe826dd1 ACCUMULO-3197 Remove deprecated Instance.getConfiguration() This patch removes the deprecated setter/getter which had poorly-defined behavior in previous versions, and was prone to incorrect use. It replaces it with a less risky getter on ZooKeeperInstance only, and refactors utility code with better names and documentation to help clarify correct usage and limitations. This mitigates some of the problems identified in ACCUMULO-3199, by removing the problematic deprecated public API methods while that issue awaits completion. Project: http://git-wip-us.apache.org/repos/asf/accumulo/repo Commit: http://git-wip-us.apache.org/repos/asf/accumulo/commit/8fe826dd Tree: http://git-wip-us.apache.org/repos/asf/accumulo/tree/8fe826dd Diff: http://git-wip-us.apache.org/repos/asf/accumulo/diff/8fe826dd Branch: refs/heads/master Commit: 8fe826dd119289165cb19e9cdf922bb4ce933324 Parents: 3fba0b5 Author: Christopher Tubbs Authored: Fri Oct 10 20:45:08 2014 -0400 Committer: Christopher Tubbs Committed: Fri Oct 10 20:45:08 2014 -0400 ---------------------------------------------------------------------- .../apache/accumulo/core/client/Instance.java | 23 --- .../accumulo/core/client/ZooKeeperInstance.java | 18 +-- .../client/impl/ClientConfigurationHelper.java | 143 +++++++++++++++++++ .../core/client/impl/ConditionalWriterImpl.java | 8 +- .../client/impl/InstanceOperationsImpl.java | 9 +- .../accumulo/core/client/impl/MasterClient.java | 2 +- .../core/client/impl/ReplicationClient.java | 4 +- .../core/client/impl/ScannerIterator.java | 2 +- .../accumulo/core/client/impl/ServerClient.java | 9 +- .../client/impl/ServerConfigurationUtil.java | 120 ---------------- .../core/client/impl/TableOperationsImpl.java | 8 +- .../impl/TabletServerBatchReaderIterator.java | 2 +- .../client/impl/TabletServerBatchWriter.java | 8 +- .../accumulo/core/client/impl/Writer.java | 2 +- .../accumulo/core/client/mock/MockInstance.java | 17 --- .../core/metadata/MetadataLocationObtainer.java | 11 +- .../impl/ClientConfigurationHelperTest.java | 110 ++++++++++++++ .../impl/ServerConfigurationUtilTest.java | 110 -------------- .../core/client/impl/TabletLocatorImplTest.java | 15 -- .../minicluster/MiniAccumuloRunner.java | 1 - .../accumulo/server/client/HdfsZooInstance.java | 17 --- .../replication/AccumuloReplicaSystem.java | 3 +- .../replication/ReplicationServicerHandler.java | 3 +- .../java/org/apache/accumulo/shell/Shell.java | 4 +- .../server/security/SystemCredentialsIT.java | 12 -- 25 files changed, 301 insertions(+), 360 deletions(-) ---------------------------------------------------------------------- http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/Instance.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/Instance.java b/core/src/main/java/org/apache/accumulo/core/client/Instance.java index 8a70d4c..ff6375b 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/Instance.java +++ b/core/src/main/java/org/apache/accumulo/core/client/Instance.java @@ -19,10 +19,8 @@ package org.apache.accumulo.core.client; import java.nio.ByteBuffer; import java.util.List; -import org.apache.accumulo.core.client.admin.InstanceOperations; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.client.security.tokens.PasswordToken; -import org.apache.accumulo.core.conf.AccumuloConfiguration; /** * This class represents the information a client needs to know to connect to an instance of accumulo. @@ -123,27 +121,6 @@ public interface Instance { Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException; /** - * Returns the AccumuloConfiguration to use when interacting with this instance. - * - * @return the AccumuloConfiguration that specifies properties related to interacting with this instance - * @deprecated since 1.6.0. This method makes very little sense in the context of the client API and never should have been exposed. - * @see InstanceOperations#getSystemConfiguration() for client-side reading of the server-side configuration. - */ - @Deprecated - AccumuloConfiguration getConfiguration(); - - /** - * Set the AccumuloConfiguration to use when interacting with this instance. - * - * @param conf - * accumulo configuration - * @deprecated since 1.6.0. This method makes very little sense in the context of the client API and never should have been exposed. - * @see InstanceOperations#setProperty(String, String) - */ - @Deprecated - void setConfiguration(AccumuloConfiguration conf); - - /** * Returns a connection to this instance of accumulo. * * @param principal http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java index 0d19e01..6b41c1d 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java +++ b/core/src/main/java/org/apache/accumulo/core/client/ZooKeeperInstance.java @@ -27,11 +27,9 @@ import java.util.UUID; import org.apache.accumulo.core.Constants; import org.apache.accumulo.core.client.ClientConfiguration.ClientProperty; import org.apache.accumulo.core.client.impl.ConnectorImpl; -import org.apache.accumulo.core.client.impl.ServerConfigurationUtil; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.DefaultConfiguration; import org.apache.accumulo.core.metadata.RootTable; import org.apache.accumulo.core.security.Credentials; import org.apache.accumulo.core.util.ByteBufferUtil; @@ -72,7 +70,6 @@ public class ZooKeeperInstance implements Instance { private final int zooKeepersSessionTimeOut; - private AccumuloConfiguration accumuloConf; private ClientConfiguration clientConf; /** @@ -248,16 +245,13 @@ public class ZooKeeperInstance implements Instance { return getConnector(principal, new PasswordToken(pass)); } - @Override - @Deprecated - public AccumuloConfiguration getConfiguration() { - return ServerConfigurationUtil.convertClientConfig(accumuloConf == null ? DefaultConfiguration.getInstance() : accumuloConf, clientConf); - } - - @Override + /** + * Used for retrieving the clientConfiguration which was provided (if any); should not be considered public API. + */ @Deprecated - public void setConfiguration(AccumuloConfiguration conf) { - this.accumuloConf = conf; + public final Configuration getClientConfiguration() { + // TODO ACCUMULO-3199 + return clientConf; } /** http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelper.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelper.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelper.java new file mode 100644 index 0000000..603172f --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelper.java @@ -0,0 +1,143 @@ +/* + * 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.impl; + +import java.io.IOException; +import java.util.Iterator; +import java.util.Map; + +import org.apache.accumulo.core.client.ClientConfiguration; +import org.apache.accumulo.core.client.Instance; +import org.apache.accumulo.core.client.ZooKeeperInstance; +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.CredentialProviderFactoryShim; +import org.apache.accumulo.core.conf.DefaultConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.apache.commons.configuration.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +/** + * This class exists to get client RPC configuration available as an {@link AccumuloConfiguration} object, which is expected by the RPC layer. + *

+ * This exists as a workaround for ACCUMULO-3199 (TODO). + */ +public class ClientConfigurationHelper { + private static final Logger log = LoggerFactory.getLogger(ClientConfigurationHelper.class); + + /** + * This retrieves a the RPC-related client-side configuration in the {@link ClientConfiguration} provided to + * {@link ZooKeeperInstance#ZooKeeperInstance(Configuration)}, if one was provided, and converts it to a generic {@link AccumuloConfiguration} instance, which + * is used by the RPC layer. + * + *

+ * If any other {@link Instance} is provided, it will return only defaults. + * + *

+ * Servers can, and should, retrieve their RPC configuration from their own configuration, which can be obtained from the ServerConfigurationFactory in the + * server module. It should NOT use this method, because it will probably not do what you expect. + * + */ + public static AccumuloConfiguration getClientRpcConfiguration(Instance instance) { + if (instance instanceof ZooKeeperInstance) { + @SuppressWarnings("deprecation") + Configuration clientConf = ((ZooKeeperInstance) instance).getClientConfiguration(); + return convertClientConfig(DefaultConfiguration.getInstance(), clientConf); + } + return DefaultConfiguration.getInstance(); + } + + public static AccumuloConfiguration convertClientConfig(final AccumuloConfiguration base, final Configuration config) { + + return new AccumuloConfiguration() { + @Override + public String get(Property property) { + final String key = property.getKey(); + + // Attempt to load sensitive properties from a CredentialProvider, if configured + if (property.isSensitive()) { + org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration(); + if (null != hadoopConf) { + try { + char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key); + if (null != value) { + log.trace("Loaded sensitive value for {} from CredentialProvider", key); + return new String(value); + } else { + log.trace("Tried to load sensitive value for {} from CredentialProvider, but none was found", key); + } + } catch (IOException e) { + log.warn("Failed to extract sensitive property ({}) from Hadoop CredentialProvider, falling back to base AccumuloConfiguration", key, e); + } + } + } + if (config.containsKey(key)) + return config.getString(key); + else + return base.get(property); + } + + @Override + public void getProperties(Map props, PropertyFilter filter) { + base.getProperties(props, filter); + + @SuppressWarnings("unchecked") + Iterator keyIter = config.getKeys(); + while (keyIter.hasNext()) { + String key = keyIter.next(); + if (filter.accept(key)) + props.put(key, config.getString(key)); + } + + // Attempt to load sensitive properties from a CredentialProvider, if configured + org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration(); + if (null != hadoopConf) { + try { + for (String key : CredentialProviderFactoryShim.getKeys(hadoopConf)) { + if (!Property.isValidPropertyKey(key) || !Property.isSensitive(key)) { + continue; + } + + if (filter.accept(key)) { + char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key); + if (null != value) { + props.put(key, new String(value)); + } + } + } + } catch (IOException e) { + log.warn("Failed to extract sensitive properties from Hadoop CredentialProvider, falling back to accumulo-site.xml", e); + } + } + } + + private org.apache.hadoop.conf.Configuration getHadoopConfiguration() { + String credProviderPaths = config.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey()); + if (null != credProviderPaths && !credProviderPaths.isEmpty()) { + org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); + hadoopConf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProviderPaths); + return hadoopConf; + } + + log.trace("Did not find credential provider configuration in ClientConfiguration"); + + return null; + } + }; + + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java index e8af187..a896be8 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ConditionalWriterImpl.java @@ -48,6 +48,7 @@ import org.apache.accumulo.core.client.TableOfflineException; import org.apache.accumulo.core.client.TimedOutException; import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations; import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException; +import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.data.ByteSequence; import org.apache.accumulo.core.data.Condition; @@ -546,10 +547,11 @@ class ConditionalWriterImpl implements ConditionalWriter { private TabletClientService.Iface getClient(String location) throws TTransportException { TabletClientService.Iface client; - if (timeout < ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)) - client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance), timeout); + AccumuloConfiguration rpcConfig = ClientConfigurationHelper.getClientRpcConfiguration(instance); + if (timeout < rpcConfig.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)) + client = ThriftUtil.getTServerClient(location, rpcConfig, timeout); else - client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance)); + client = ThriftUtil.getTServerClient(location, rpcConfig); return client; } http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java index c5f7634..1a9af63 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/InstanceOperationsImpl.java @@ -35,6 +35,7 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.impl.thrift.ClientService; import org.apache.accumulo.core.client.impl.thrift.ConfigurationType; import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException; +import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.master.thrift.MasterClientService; import org.apache.accumulo.core.security.Credentials; import org.apache.accumulo.core.tabletserver.thrift.TabletClientService; @@ -55,6 +56,7 @@ import org.apache.thrift.transport.TTransportException; public class InstanceOperationsImpl implements InstanceOperations { private Instance instance; private Credentials credentials; + private AccumuloConfiguration rpcConfig; /** * @param instance @@ -66,6 +68,7 @@ public class InstanceOperationsImpl implements InstanceOperations { checkArgument(instance != null, "instance is null"); checkArgument(credentials != null, "credentials is null"); this.instance = instance; + this.rpcConfig = ClientConfigurationHelper.getClientRpcConfiguration(instance); this.credentials = credentials; } @@ -135,7 +138,7 @@ public class InstanceOperationsImpl implements InstanceOperations { public List getActiveScans(String tserver) throws AccumuloException, AccumuloSecurityException { Client client = null; try { - client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance)); + client = ThriftUtil.getTServerClient(tserver, rpcConfig); List as = new ArrayList(); for (org.apache.accumulo.core.tabletserver.thrift.ActiveScan activeScan : client.getActiveScans(Tracer.traceInfo(), credentials.toThrift(instance))) { @@ -172,7 +175,7 @@ public class InstanceOperationsImpl implements InstanceOperations { public List getActiveCompactions(String tserver) throws AccumuloException, AccumuloSecurityException { Client client = null; try { - client = ThriftUtil.getTServerClient(tserver, ServerConfigurationUtil.getConfiguration(instance)); + client = ThriftUtil.getTServerClient(tserver, rpcConfig); List as = new ArrayList(); for (org.apache.accumulo.core.tabletserver.thrift.ActiveCompaction activeCompaction : client.getActiveCompactions(Tracer.traceInfo(), @@ -196,7 +199,7 @@ public class InstanceOperationsImpl implements InstanceOperations { public void ping(String tserver) throws AccumuloException { TTransport transport = null; try { - transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver, false), ServerConfigurationUtil.getConfiguration(instance)); + transport = ThriftUtil.createTransport(AddressUtil.parseAddress(tserver, false), rpcConfig); TabletClientService.Client client = ThriftUtil.createClient(new TabletClientService.Client.Factory(), transport); client.getTabletServerStatus(Tracer.traceInfo(), credentials.toThrift(instance)); } catch (TTransportException e) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java index 9807a82..53f178a 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/MasterClient.java @@ -66,7 +66,7 @@ public class MasterClient { try { // Master requests can take a long time: don't ever time out MasterClientService.Client client = ThriftUtil.getClientNoTimeout(new MasterClientService.Client.Factory(), master, - ServerConfigurationUtil.getConfiguration(instance)); + ClientConfigurationHelper.getClientRpcConfiguration(instance)); return client; } catch (TTransportException tte) { if (tte.getCause().getClass().equals(UnknownHostException.class)) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java index 7e9dc60..937dc7c 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ReplicationClient.java @@ -84,7 +84,7 @@ public class ReplicationClient { } - AccumuloConfiguration conf = ServerConfigurationUtil.getConfiguration(instance); + AccumuloConfiguration conf = ClientConfigurationHelper.getClientRpcConfiguration(instance); String zkPath = ZooUtil.getRoot(instance) + Constants.ZMASTER_REPLICATION_COORDINATOR_ADDR; String replCoordinatorAddr; @@ -130,7 +130,7 @@ public class ReplicationClient { checkNotNull(server); try { - return ThriftUtil.getClientNoTimeout(new ReplicationServicer.Client.Factory(), server, ServerConfigurationUtil.getConfiguration(inst)); + return ThriftUtil.getClientNoTimeout(new ReplicationServicer.Client.Factory(), server, ClientConfigurationHelper.getClientRpcConfiguration(inst)); } catch (TTransportException tte) { log.debug("Failed to connect to servicer ({}), will retry...", server, tte); throw tte; http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java index 5ea3662..b0bfc20 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ScannerIterator.java @@ -81,7 +81,7 @@ public class ScannerIterator implements Iterator> { try { while (true) { - List currentBatch = ThriftScanner.scan(instance, credentials, scanState, timeOut, ServerConfigurationUtil.getConfiguration(instance)); + List currentBatch = ThriftScanner.scan(instance, credentials, scanState, timeOut, ClientConfigurationHelper.getClientRpcConfiguration(instance)); if (currentBatch == null) { synchQ.add(EMPTY_LIST); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java index d3822f4..fcbb104 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerClient.java @@ -116,8 +116,8 @@ public class ServerClient { } public static Pair getConnection(Instance instance, boolean preferCachedConnections) throws TTransportException { - AccumuloConfiguration conf = ServerConfigurationUtil.getConfiguration(instance); - return getConnection(instance, preferCachedConnections, conf.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)); + AccumuloConfiguration rpcConfig = ClientConfigurationHelper.getClientRpcConfiguration(instance); + return getConnection(instance, preferCachedConnections, rpcConfig.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)); } public static Pair getConnection(Instance instance, boolean preferCachedConnections, long rpcTimeout) throws TTransportException { @@ -131,9 +131,8 @@ public class ServerClient { String path = ZooUtil.getRoot(instance) + Constants.ZTSERVERS + "/" + tserver; byte[] data = ZooUtil.getLockData(zc, path); if (data != null && !new String(data, StandardCharsets.UTF_8).equals("master")) - servers.add(new ThriftTransportKey( - new ServerServices(new String(data)).getAddressString(Service.TSERV_CLIENT), - rpcTimeout, SslConnectionParams.forClient(ServerConfigurationUtil.getConfiguration(instance)))); + servers.add(new ThriftTransportKey(new ServerServices(new String(data)).getAddressString(Service.TSERV_CLIENT), rpcTimeout, SslConnectionParams + .forClient(ClientConfigurationHelper.getClientRpcConfiguration(instance)))); } boolean opened = false; http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java b/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java deleted file mode 100644 index 507b071..0000000 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtil.java +++ /dev/null @@ -1,120 +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.accumulo.core.client.impl; - -import java.io.IOException; -import java.util.Iterator; -import java.util.Map; - -import org.apache.accumulo.core.client.Instance; -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.CredentialProviderFactoryShim; -import org.apache.accumulo.core.conf.Property; -import org.apache.commons.configuration.Configuration; -import org.slf4j.Logger; -import org.slf4j.LoggerFactory; - -/** - * All client side code that needs a server side configuration object should obtain it from here. - */ -public class ServerConfigurationUtil { - private static final Logger log = LoggerFactory.getLogger(ServerConfigurationUtil.class); - @SuppressWarnings("deprecation") - public static AccumuloConfiguration getConfiguration(Instance instance) { - return instance.getConfiguration(); - } - - public static AccumuloConfiguration convertClientConfig(final AccumuloConfiguration base, final Configuration config) { - - return new AccumuloConfiguration() { - @Override - public String get(Property property) { - final String key = property.getKey(); - - // Attempt to load sensitive properties from a CredentialProvider, if configured - if (property.isSensitive()) { - org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration(); - if (null != hadoopConf) { - try { - char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key); - if (null != value) { - log.trace("Loaded sensitive value for {} from CredentialProvider", key); - return new String(value); - } else { - log.trace("Tried to load sensitive value for {} from CredentialProvider, but none was found", key); - } - } catch (IOException e) { - log.warn("Failed to extract sensitive property ({}) from Hadoop CredentialProvider, falling back to base AccumuloConfiguration", key, e); - } - } - } - if (config.containsKey(key)) - return config.getString(key); - else - return base.get(property); - } - - @Override - public void getProperties(Map props, PropertyFilter filter) { - base.getProperties(props, filter); - - @SuppressWarnings("unchecked") - Iterator keyIter = config.getKeys(); - while (keyIter.hasNext()) { - String key = keyIter.next(); - if (filter.accept(key)) - props.put(key, config.getString(key)); - } - - // Attempt to load sensitive properties from a CredentialProvider, if configured - org.apache.hadoop.conf.Configuration hadoopConf = getHadoopConfiguration(); - if (null != hadoopConf) { - try { - for (String key : CredentialProviderFactoryShim.getKeys(hadoopConf)) { - if (!Property.isValidPropertyKey(key) || !Property.isSensitive(key)) { - continue; - } - - if (filter.accept(key)) { - char[] value = CredentialProviderFactoryShim.getValueFromCredentialProvider(hadoopConf, key); - if (null != value) { - props.put(key, new String(value)); - } - } - } - } catch (IOException e) { - log.warn("Failed to extract sensitive properties from Hadoop CredentialProvider, falling back to accumulo-site.xml", e); - } - } - } - - private org.apache.hadoop.conf.Configuration getHadoopConfiguration() { - String credProviderPaths = config.getString(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey()); - if (null != credProviderPaths && !credProviderPaths.isEmpty()) { - org.apache.hadoop.conf.Configuration hadoopConf = new org.apache.hadoop.conf.Configuration(); - hadoopConf.set(CredentialProviderFactoryShim.CREDENTIAL_PROVIDER_PATH, credProviderPaths); - return hadoopConf; - } - - log.trace("Did not find credential provider configuration in ClientConfiguration"); - - return null; - } - }; - - } -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java index e46b9c9..9889413 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TableOperationsImpl.java @@ -122,6 +122,7 @@ import com.google.common.base.Joiner; public class TableOperationsImpl extends TableOperationsHelper { private Instance instance; private Credentials credentials; + private AccumuloConfiguration rpcConfig; public static final String CLONE_EXCLUDE_PREFIX = "!"; @@ -137,6 +138,7 @@ public class TableOperationsImpl extends TableOperationsHelper { checkArgument(instance != null, "instance is null"); checkArgument(credentials != null, "credentials is null"); this.instance = instance; + this.rpcConfig = ClientConfigurationHelper.getClientRpcConfiguration(instance); this.credentials = credentials; } @@ -484,7 +486,7 @@ public class TableOperationsImpl extends TableOperationsHelper { } try { - TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, ServerConfigurationUtil.getConfiguration(instance)); + TabletClientService.Client client = ThriftUtil.getTServerClient(tl.tablet_location, rpcConfig); try { OpTimer opTimer = null; if (log.isTraceEnabled()) @@ -1139,11 +1141,11 @@ public class TableOperationsImpl extends TableOperationsHelper { return ranges; } - // TODO Remove deprecation warning surppression when Hadoop1 support is dropped + // TODO Remove deprecation warning suppression when Hadoop1 support is dropped @SuppressWarnings("deprecation") private Path checkPath(String dir, String kind, String type) throws IOException, AccumuloException { Path ret; - FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), ServerConfigurationUtil.getConfiguration(instance)).getFileSystem(); + FileSystem fs = VolumeConfiguration.getVolume(dir, CachedConfiguration.getInstance(), rpcConfig).getFileSystem(); if (dir.contains(":")) { ret = new Path(dir); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java index d2ca60e..e0aa30a 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchReaderIterator.java @@ -344,7 +344,7 @@ public class TabletServerBatchReaderIterator implements Iterator 0) { locator.invalidateCache(tsFailures.keySet()); synchronized (failures) { http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java index 5eec397..475d04d 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/TabletServerBatchWriter.java @@ -48,6 +48,7 @@ import org.apache.accumulo.core.client.TimedOutException; import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations; import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode; import org.apache.accumulo.core.client.impl.thrift.ThriftSecurityException; +import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; import org.apache.accumulo.core.constraints.Violations; import org.apache.accumulo.core.data.ConstraintViolationSummary; @@ -848,10 +849,11 @@ public class TabletServerBatchWriter { try { TabletClientService.Iface client; - if (timeoutTracker.getTimeOut() < ServerConfigurationUtil.getConfiguration(instance).getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)) - client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance), timeoutTracker.getTimeOut()); + AccumuloConfiguration rpcConfig = ClientConfigurationHelper.getClientRpcConfiguration(instance); + if (timeoutTracker.getTimeOut() < rpcConfig.getTimeInMillis(Property.GENERAL_RPC_TIMEOUT)) + client = ThriftUtil.getTServerClient(location, rpcConfig, timeoutTracker.getTimeOut()); else - client = ThriftUtil.getTServerClient(location, ServerConfigurationUtil.getConfiguration(instance)); + client = ThriftUtil.getTServerClient(location, rpcConfig); try { MutationSet allFailures = new MutationSet(); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java index 0358a88..e76e2d9 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java +++ b/core/src/main/java/org/apache/accumulo/core/client/impl/Writer.java @@ -98,7 +98,7 @@ public class Writer { } try { - updateServer(instance, m, tabLoc.tablet_extent, tabLoc.tablet_location, credentials, ServerConfigurationUtil.getConfiguration(instance)); + updateServer(instance, m, tabLoc.tablet_extent, tabLoc.tablet_location, credentials, ClientConfigurationHelper.getClientRpcConfiguration(instance)); return; } catch (NotServingTabletException e) { log.trace("Not serving tablet, server = " + tabLoc.tablet_location); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java index f48d5b4..9b07d49 100644 --- a/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java +++ b/core/src/main/java/org/apache/accumulo/core/client/mock/MockInstance.java @@ -30,7 +30,6 @@ import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.impl.thrift.SecurityErrorCode; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; import org.apache.accumulo.core.client.security.tokens.PasswordToken; -import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.security.Credentials; import org.apache.accumulo.core.util.ByteBufferUtil; import org.apache.accumulo.core.util.CachedConfiguration; @@ -135,22 +134,6 @@ public class MockInstance implements Instance { return getConnector(user, TextUtil.getBytes(new Text(pass.toString()))); } - AccumuloConfiguration conf = null; - - @Deprecated - @Override - public AccumuloConfiguration getConfiguration() { - if (conf == null) - conf = AccumuloConfiguration.getDefaultConfiguration(); - return conf; - } - - @Override - @Deprecated - public void setConfiguration(AccumuloConfiguration conf) { - this.conf = conf; - } - @Override public Connector getConnector(String principal, AuthenticationToken token) throws AccumuloException, AccumuloSecurityException { Connector conn = new MockConnector(new Credentials(principal, token), acu, this); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java ---------------------------------------------------------------------- diff --git a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java index 25007f5..c7fe137 100644 --- a/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java +++ b/core/src/main/java/org/apache/accumulo/core/metadata/MetadataLocationObtainer.java @@ -34,7 +34,7 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.impl.AccumuloServerException; import org.apache.accumulo.core.client.impl.ScannerOptions; -import org.apache.accumulo.core.client.impl.ServerConfigurationUtil; +import org.apache.accumulo.core.client.impl.ClientConfigurationHelper; import org.apache.accumulo.core.client.impl.TabletLocator; import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocation; import org.apache.accumulo.core.client.impl.TabletLocator.TabletLocations; @@ -42,6 +42,7 @@ import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletLocationObta import org.apache.accumulo.core.client.impl.TabletServerBatchReaderIterator; import org.apache.accumulo.core.client.impl.TabletServerBatchReaderIterator.ResultReceiver; import org.apache.accumulo.core.client.impl.ThriftScanner; +import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.Column; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.KeyExtent; @@ -65,10 +66,12 @@ public class MetadataLocationObtainer implements TabletLocationObtainer { private SortedSet locCols; private ArrayList columns; private Instance instance; + private AccumuloConfiguration rpcConfig; public MetadataLocationObtainer(Instance instance) { this.instance = instance; + this.rpcConfig = ClientConfigurationHelper.getClientRpcConfiguration(instance); locCols = new TreeSet(); locCols.add(new Column(TextUtil.getBytes(TabletsSection.CurrentLocationColumnFamily.NAME), null, null)); @@ -98,7 +101,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer { Map> serverSideIteratorOptions = Collections.emptyMap(); boolean more = ThriftScanner.getBatchFromServer(instance, credentials, range, src.tablet_extent, src.tablet_location, encodedResults, locCols, - serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationUtil.getConfiguration(instance)); + serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, rpcConfig); decodeRows(encodedResults, results); @@ -106,7 +109,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer { range = new Range(results.lastKey().followingKey(PartialKey.ROW_COLFAM_COLQUAL_COLVIS_TIME), true, new Key(stopRow).followingKey(PartialKey.ROW), false); encodedResults.clear(); more = ThriftScanner.getBatchFromServer(instance, credentials, range, src.tablet_extent, src.tablet_location, encodedResults, locCols, - serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, ServerConfigurationUtil.getConfiguration(instance)); + serverSideIteratorList, serverSideIteratorOptions, Constants.SCAN_BATCH_SIZE, Authorizations.EMPTY, false, rpcConfig); decodeRows(encodedResults, results); } @@ -179,7 +182,7 @@ public class MetadataLocationObtainer implements TabletLocationObtainer { Map> failures = new HashMap>(); try { TabletServerBatchReaderIterator.doLookup(instance, credentials, tserver, tabletsRanges, failures, unscanned, rr, columns, opts, Authorizations.EMPTY, - ServerConfigurationUtil.getConfiguration(instance)); + ClientConfigurationHelper.getClientRpcConfiguration(instance)); if (failures.size() > 0) { // invalidate extents in parents cache if (log.isTraceEnabled()) http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/test/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelperTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelperTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelperTest.java new file mode 100644 index 0000000..663ce22 --- /dev/null +++ b/core/src/test/java/org/apache/accumulo/core/client/impl/ClientConfigurationHelperTest.java @@ -0,0 +1,110 @@ +/* + * 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.impl; + +import java.io.File; +import java.net.URL; +import java.util.HashMap; +import java.util.Map; + +import org.apache.accumulo.core.client.ClientConfiguration; +import org.apache.accumulo.core.conf.AccumuloConfiguration; +import org.apache.accumulo.core.conf.AccumuloConfiguration.AllFilter; +import org.apache.accumulo.core.conf.CredentialProviderFactoryShim; +import org.apache.accumulo.core.conf.DefaultConfiguration; +import org.apache.accumulo.core.conf.Property; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Test; + +public class ClientConfigurationHelperTest { + + private static boolean isCredentialProviderAvailable = false; + private static final String keystoreName = "/site-cfg.jceks"; + + //site-cfg.jceks={'ignored.property'=>'ignored', 'instance.secret'=>'mysecret', 'general.rpc.timeout'=>'timeout'} + private static File keystore; + + @BeforeClass + public static void setUpBeforeClass() throws Exception { + try { + Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME); + isCredentialProviderAvailable = true; + } catch (Exception e) { + isCredentialProviderAvailable = false; + } + + if (isCredentialProviderAvailable) { + URL keystoreUrl = ClientConfigurationHelperTest.class.getResource(keystoreName); + + Assert.assertNotNull("Could not find " + keystoreName, keystoreUrl); + + keystore = new File(keystoreUrl.getFile()); + } + } + + protected String getKeyStoreUrl(File absoluteFilePath) { + return "jceks://file" + absoluteFilePath.getAbsolutePath(); + } + + @Test + public void loadSensitivePropertyFromCredentialProvider() { + if (!isCredentialProviderAvailable) { + return; + } + + String absPath = getKeyStoreUrl(keystore); + ClientConfiguration clientConf = new ClientConfiguration(); + clientConf.addProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), absPath); + + AccumuloConfiguration accClientConf = ClientConfigurationHelper.convertClientConfig(DefaultConfiguration.getInstance(), clientConf); + Assert.assertEquals("mysecret", accClientConf.get(Property.INSTANCE_SECRET)); + } + + @Test + public void defaultValueForSensitiveProperty() { + if (!isCredentialProviderAvailable) { + return; + } + + ClientConfiguration clientConf = new ClientConfiguration(); + + AccumuloConfiguration accClientConf = ClientConfigurationHelper.convertClientConfig(DefaultConfiguration.getInstance(), clientConf); + Assert.assertEquals(Property.INSTANCE_SECRET.getDefaultValue(), accClientConf.get(Property.INSTANCE_SECRET)); + } + + @Test + public void sensitivePropertiesIncludedInProperties() { + if (!isCredentialProviderAvailable) { + return; + } + + String absPath = getKeyStoreUrl(keystore); + ClientConfiguration clientConf = new ClientConfiguration(); + clientConf.addProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), absPath); + + AccumuloConfiguration accClientConf = ClientConfigurationHelper.convertClientConfig(DefaultConfiguration.getInstance(), clientConf); + Map props = new HashMap(); + accClientConf.getProperties(props, new AllFilter()); + + // Only sensitive properties are added + Assert.assertEquals(Property.GENERAL_RPC_TIMEOUT.getDefaultValue(), props.get(Property.GENERAL_RPC_TIMEOUT.getKey())); + // Only known properties are added + Assert.assertFalse(props.containsKey("ignored.property")); + Assert.assertEquals("mysecret", props.get(Property.INSTANCE_SECRET.getKey())); + } +} http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/test/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtilTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtilTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtilTest.java deleted file mode 100644 index 1124283..0000000 --- a/core/src/test/java/org/apache/accumulo/core/client/impl/ServerConfigurationUtilTest.java +++ /dev/null @@ -1,110 +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.accumulo.core.client.impl; - -import java.io.File; -import java.net.URL; -import java.util.HashMap; -import java.util.Map; - -import org.apache.accumulo.core.client.ClientConfiguration; -import org.apache.accumulo.core.conf.AccumuloConfiguration; -import org.apache.accumulo.core.conf.AccumuloConfiguration.AllFilter; -import org.apache.accumulo.core.conf.CredentialProviderFactoryShim; -import org.apache.accumulo.core.conf.DefaultConfiguration; -import org.apache.accumulo.core.conf.Property; -import org.junit.Assert; -import org.junit.BeforeClass; -import org.junit.Test; - -public class ServerConfigurationUtilTest { - - private static boolean isCredentialProviderAvailable = false; - private static final String keystoreName = "/site-cfg.jceks"; - - //site-cfg.jceks={'ignored.property'=>'ignored', 'instance.secret'=>'mysecret', 'general.rpc.timeout'=>'timeout'} - private static File keystore; - - @BeforeClass - public static void setUpBeforeClass() throws Exception { - try { - Class.forName(CredentialProviderFactoryShim.HADOOP_CRED_PROVIDER_CLASS_NAME); - isCredentialProviderAvailable = true; - } catch (Exception e) { - isCredentialProviderAvailable = false; - } - - if (isCredentialProviderAvailable) { - URL keystoreUrl = ServerConfigurationUtilTest.class.getResource(keystoreName); - - Assert.assertNotNull("Could not find " + keystoreName, keystoreUrl); - - keystore = new File(keystoreUrl.getFile()); - } - } - - protected String getKeyStoreUrl(File absoluteFilePath) { - return "jceks://file" + absoluteFilePath.getAbsolutePath(); - } - - @Test - public void loadSensitivePropertyFromCredentialProvider() { - if (!isCredentialProviderAvailable) { - return; - } - - String absPath = getKeyStoreUrl(keystore); - ClientConfiguration clientConf = new ClientConfiguration(); - clientConf.addProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), absPath); - - AccumuloConfiguration accClientConf = ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(), clientConf); - Assert.assertEquals("mysecret", accClientConf.get(Property.INSTANCE_SECRET)); - } - - @Test - public void defaultValueForSensitiveProperty() { - if (!isCredentialProviderAvailable) { - return; - } - - ClientConfiguration clientConf = new ClientConfiguration(); - - AccumuloConfiguration accClientConf = ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(), clientConf); - Assert.assertEquals(Property.INSTANCE_SECRET.getDefaultValue(), accClientConf.get(Property.INSTANCE_SECRET)); - } - - @Test - public void sensitivePropertiesIncludedInProperties() { - if (!isCredentialProviderAvailable) { - return; - } - - String absPath = getKeyStoreUrl(keystore); - ClientConfiguration clientConf = new ClientConfiguration(); - clientConf.addProperty(Property.GENERAL_SECURITY_CREDENTIAL_PROVIDER_PATHS.getKey(), absPath); - - AccumuloConfiguration accClientConf = ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(), clientConf); - Map props = new HashMap(); - accClientConf.getProperties(props, new AllFilter()); - - // Only sensitive properties are added - Assert.assertEquals(Property.GENERAL_RPC_TIMEOUT.getDefaultValue(), props.get(Property.GENERAL_RPC_TIMEOUT.getKey())); - // Only known properties are added - Assert.assertFalse(props.containsKey("ignored.property")); - Assert.assertEquals("mysecret", props.get(Property.INSTANCE_SECRET.getKey())); - } -} http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java ---------------------------------------------------------------------- diff --git a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java index a594d19..21bf2b9 100644 --- a/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java +++ b/core/src/test/java/org/apache/accumulo/core/client/impl/TabletLocatorImplTest.java @@ -41,7 +41,6 @@ import org.apache.accumulo.core.client.impl.TabletLocator.TabletServerMutations; import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletLocationObtainer; import org.apache.accumulo.core.client.impl.TabletLocatorImpl.TabletServerLockChecker; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; -import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.KeyExtent; import org.apache.accumulo.core.data.Mutation; @@ -437,20 +436,6 @@ public class TabletLocatorImplTest extends TestCase { throw new UnsupportedOperationException(); } - AccumuloConfiguration conf = AccumuloConfiguration.getDefaultConfiguration(); - - @Deprecated - @Override - public AccumuloConfiguration getConfiguration() { - return conf; - } - - @Override - @Deprecated - public void setConfiguration(AccumuloConfiguration conf) { - this.conf = conf; - } - @Override @Deprecated public Connector getConnector(String user, CharSequence pass) throws AccumuloException, AccumuloSecurityException { http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java ---------------------------------------------------------------------- diff --git a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java index 6b618e3..4a3100d 100644 --- a/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java +++ b/minicluster/src/main/java/org/apache/accumulo/minicluster/MiniAccumuloRunner.java @@ -21,7 +21,6 @@ import java.io.FileInputStream; import java.io.IOException; import java.io.InputStream; import java.net.ServerSocket; -import java.net.Socket; import java.util.Date; import java.util.HashMap; import java.util.Map; http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java ---------------------------------------------------------------------- diff --git a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java index 050a31a..7169215 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java +++ b/server/base/src/main/java/org/apache/accumulo/server/client/HdfsZooInstance.java @@ -44,7 +44,6 @@ import org.apache.accumulo.core.zookeeper.ZooUtil; import org.apache.accumulo.fate.zookeeper.ZooCache; import org.apache.accumulo.fate.zookeeper.ZooCacheFactory; import org.apache.accumulo.server.Accumulo; -import org.apache.accumulo.server.conf.ServerConfigurationFactory; import org.apache.accumulo.server.fs.VolumeManager; import org.apache.accumulo.server.fs.VolumeManagerImpl; import org.apache.accumulo.server.zookeeper.ZooLock; @@ -175,22 +174,6 @@ public class HdfsZooInstance implements Instance { return getConnector(user, TextUtil.getBytes(new Text(pass.toString()))); } - private AccumuloConfiguration conf = null; - - @Deprecated - @Override - public AccumuloConfiguration getConfiguration() { - if (conf == null) - conf = new ServerConfigurationFactory(this).getConfiguration(); - return conf; - } - - @Override - @Deprecated - public void setConfiguration(AccumuloConfiguration conf) { - this.conf = conf; - } - public static void main(String[] args) { Instance instance = HdfsZooInstance.getInstance(); System.out.println("Instance Name: " + instance.getInstanceName()); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java index 4079a65..732907d 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/AccumuloReplicaSystem.java @@ -34,7 +34,6 @@ import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.ZooKeeperInstance; import org.apache.accumulo.core.client.impl.ClientExecReturn; import org.apache.accumulo.core.client.impl.ReplicationClient; -import org.apache.accumulo.core.client.impl.ServerConfigurationUtil; import org.apache.accumulo.core.client.replication.ReplicaSystem; import org.apache.accumulo.core.client.security.tokens.PasswordToken; import org.apache.accumulo.core.conf.AccumuloConfiguration; @@ -154,7 +153,7 @@ public class AccumuloReplicaSystem implements ReplicaSystem { @Override public Status replicate(final Path p, final Status status, final ReplicationTarget target, final ReplicaSystemHelper helper) { final Instance localInstance = HdfsZooInstance.getInstance(); - final AccumuloConfiguration localConf = ServerConfigurationUtil.getConfiguration(localInstance); + final AccumuloConfiguration localConf = new ServerConfigurationFactory(localInstance).getConfiguration(); Credentials credentialsForPeer = getCredentialsForPeer(localConf, target); final TCredentials tCredsForPeer = credentialsForPeer.toThrift(localInstance); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java ---------------------------------------------------------------------- diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java index 3ae0bd7..30361b1 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/replication/ReplicationServicerHandler.java @@ -23,7 +23,6 @@ import org.apache.accumulo.core.client.AccumuloSecurityException; import org.apache.accumulo.core.client.Connector; import org.apache.accumulo.core.client.Instance; import org.apache.accumulo.core.client.TableNotFoundException; -import org.apache.accumulo.core.client.impl.ServerConfigurationUtil; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.conf.Property; @@ -75,7 +74,7 @@ public class ReplicationServicerHandler implements Iface { throw new RemoteReplicationException(RemoteReplicationErrorCode.TABLE_DOES_NOT_EXIST, "Table with id " + tableId + " does not exist"); } - AccumuloConfiguration conf = ServerConfigurationUtil.getConfiguration(inst); + AccumuloConfiguration conf = new ServerConfigurationFactory(inst).getConfiguration(); Map replicationHandlers = conf.getAllPropertiesWithPrefix(Property.TSERV_REPLICATION_REPLAYERS); String propertyForHandlerTable = Property.TSERV_REPLICATION_REPLAYERS.getKey() + tableId; http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/shell/src/main/java/org/apache/accumulo/shell/Shell.java ---------------------------------------------------------------------- diff --git a/shell/src/main/java/org/apache/accumulo/shell/Shell.java b/shell/src/main/java/org/apache/accumulo/shell/Shell.java index 37bf052..2ebdb6c 100644 --- a/shell/src/main/java/org/apache/accumulo/shell/Shell.java +++ b/shell/src/main/java/org/apache/accumulo/shell/Shell.java @@ -55,7 +55,7 @@ import org.apache.accumulo.core.client.IteratorSetting; import org.apache.accumulo.core.client.NamespaceNotFoundException; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.ZooKeeperInstance; -import org.apache.accumulo.core.client.impl.ServerConfigurationUtil; +import org.apache.accumulo.core.client.impl.ClientConfigurationHelper; import org.apache.accumulo.core.client.impl.Tables; import org.apache.accumulo.core.client.mock.MockInstance; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; @@ -438,7 +438,7 @@ public class Shell extends ShellOptions { instanceName = clientConfig.get(ClientProperty.INSTANCE_NAME); } if (instanceName == null || keepers == null) { - AccumuloConfiguration conf = SiteConfiguration.getInstance(ServerConfigurationUtil.convertClientConfig(DefaultConfiguration.getInstance(), clientConfig)); + AccumuloConfiguration conf = SiteConfiguration.getInstance(ClientConfigurationHelper.convertClientConfig(DefaultConfiguration.getInstance(), clientConfig)); if (instanceName == null) { Path instanceDir = new Path(VolumeConfiguration.getVolumeUris(conf)[0], "instance_id"); instanceId = UUID.fromString(ZooUtil.getInstanceIDFromHdfs(instanceDir, conf)); http://git-wip-us.apache.org/repos/asf/accumulo/blob/8fe826dd/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java ---------------------------------------------------------------------- diff --git a/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java b/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java index f5296c0..7524943 100644 --- a/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java +++ b/test/src/test/java/org/apache/accumulo/server/security/SystemCredentialsIT.java @@ -30,7 +30,6 @@ import org.apache.accumulo.core.client.Scanner; import org.apache.accumulo.core.client.TableNotFoundException; import org.apache.accumulo.core.client.security.SecurityErrorCode; import org.apache.accumulo.core.client.security.tokens.AuthenticationToken; -import org.apache.accumulo.core.conf.AccumuloConfiguration; import org.apache.accumulo.core.data.Key; import org.apache.accumulo.core.data.Value; import org.apache.accumulo.core.metadata.RootTable; @@ -62,12 +61,6 @@ public class SystemCredentialsIT extends ConfigurableMacIT { if (args[0].equals("bad")) { creds = new SystemCredentials(new Instance() { - @Deprecated - @Override - public void setConfiguration(AccumuloConfiguration conf) { - throw new UnsupportedOperationException(); - } - @Override public int getZooKeepersSessionTimeOut() { throw new UnsupportedOperationException(); @@ -121,11 +114,6 @@ public class SystemCredentialsIT extends ConfigurableMacIT { throw new UnsupportedOperationException(); } - @Deprecated - @Override - public AccumuloConfiguration getConfiguration() { - throw new UnsupportedOperationException(); - } }); } else if (args[0].equals("good")) { creds = SystemCredentials.get();