Return-Path: X-Original-To: archive-asf-public-internal@cust-asf2.ponee.io Delivered-To: archive-asf-public-internal@cust-asf2.ponee.io Received: from cust-asf.ponee.io (cust-asf.ponee.io [163.172.22.183]) by cust-asf2.ponee.io (Postfix) with ESMTP id 8466B200D2F for ; Wed, 1 Nov 2017 21:27:22 +0100 (CET) Received: by cust-asf.ponee.io (Postfix) id 830131609EC; Wed, 1 Nov 2017 20:27:22 +0000 (UTC) Delivered-To: archive-asf-public@cust-asf.ponee.io Received: from mail.apache.org (hermes.apache.org [140.211.11.3]) by cust-asf.ponee.io (Postfix) with SMTP id 53500160BFE for ; Wed, 1 Nov 2017 21:27:21 +0100 (CET) Received: (qmail 98400 invoked by uid 500); 1 Nov 2017 20:27:20 -0000 Mailing-List: contact notifications-help@accumulo.apache.org; run by ezmlm Precedence: bulk List-Help: List-Unsubscribe: List-Post: List-Id: Reply-To: jira@apache.org Delivered-To: mailing list notifications@accumulo.apache.org Received: (qmail 98389 invoked by uid 99); 1 Nov 2017 20:27:20 -0000 Received: from ec2-52-202-80-70.compute-1.amazonaws.com (HELO gitbox.apache.org) (52.202.80.70) by apache.org (qpsmtpd/0.29) with ESMTP; Wed, 01 Nov 2017 20:27:20 +0000 From: GitBox To: notifications@accumulo.apache.org Subject: [GitHub] mikewalch closed pull request #316: ACCUMULO-4706 Updates to support Accumulo docker image Message-ID: <150956803988.22160.4951533608687907342.gitbox@gitbox.apache.org> archived-at: Wed, 01 Nov 2017 20:27:22 -0000 mikewalch closed pull request #316: ACCUMULO-4706 Updates to support Accumulo docker image URL: https://github.com/apache/accumulo/pull/316 This is a PR merged from a forked repository. As GitHub hides the original diff on merge, it is displayed below for the sake of provenance: As this is a foreign pull request (from a fork), the diff is supplied below (as it won't show otherwise due to GitHub magic): diff --git a/core/src/main/java/org/apache/accumulo/core/conf/CliConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/CliConfiguration.java new file mode 100644 index 0000000000..7be10530f2 --- /dev/null +++ b/core/src/main/java/org/apache/accumulo/core/conf/CliConfiguration.java @@ -0,0 +1,63 @@ +/* + * 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.conf; + +import java.util.HashMap; +import java.util.Map; +import java.util.Objects; +import java.util.function.Predicate; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class CliConfiguration { + + private static final Logger log = LoggerFactory.getLogger(CliConfiguration.class); + private static volatile Map config = new HashMap<>(); + + /** + * Sets CliConfiguration with map of configuration. Additional calls will overwrite existing properties and values. + * + * @param conf + * Map of configuration + */ + public static void set(Map conf) { + Objects.requireNonNull(conf); + config = conf; + } + + public static void print() { + log.info("The following configuration was set on the command line:"); + for (Map.Entry entry : config.entrySet()) { + String key = entry.getKey(); + log.info(key + " = " + (Property.isSensitive(key) ? "" : entry.getValue())); + } + } + + public static String get(Property property) { + return config.get(property.getKey()); + } + + public static void getProperties(Map props, Predicate filter) { + for (Map.Entry entry : config.entrySet()) { + if (filter.test(entry.getKey())) { + props.put(entry.getKey(), entry.getValue()); + } + } + } +} diff --git a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java index cd35ad8a2a..4135de0111 100644 --- a/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java +++ b/core/src/main/java/org/apache/accumulo/core/conf/SiteConfiguration.java @@ -34,8 +34,8 @@ import org.slf4j.LoggerFactory; /** - * An {@link AccumuloConfiguration} which loads properties from an XML file, usually accumulo-site.xml. This implementation supports defaulting undefined - * property values to a parent configuration's definitions. + * An {@link AccumuloConfiguration} which first loads any properties set on the command-line (using the -o option) and then from an XML file, usually + * accumulo-site.xml. This implementation supports defaulting undefined property values to a parent configuration's definitions. *

* The system property "accumulo.configuration" can be used to specify the location of the XML configuration file on the classpath or filesystem if the path is * prefixed with 'file://'. If the system property is not defined, it defaults to "accumulo-site.xml" and will look on classpath for file. @@ -119,8 +119,11 @@ synchronized private static Configuration getXmlConfig() { @Override public String get(Property property) { - String key = property.getKey(); + if (CliConfiguration.get(property) != null) { + return CliConfiguration.get(property); + } + String key = property.getKey(); // If the property is sensitive, see if CredentialProvider was configured. if (property.isSensitive()) { Configuration hadoopConf = getHadoopConfiguration(); @@ -145,12 +148,19 @@ public String get(Property property) { log.error("Using default value for {} due to improperly formatted {}: {}", key, property.getType(), value); value = parent.get(property); } + return value; } @Override public void getProperties(Map props, Predicate filter) { - parent.getProperties(props, filter); + getProperties(props, filter, true); + } + + public void getProperties(Map props, Predicate filter, boolean useDefaults) { + if (useDefaults) { + parent.getProperties(props, filter); + } for (Entry entry : getXmlConfig()) if (filter.test(entry.getKey())) @@ -176,6 +186,7 @@ public void getProperties(Map props, Predicate filter) { log.warn("Failed to extract sensitive properties from Hadoop CredentialProvider, falling back to accumulo-site.xml", e); } } + CliConfiguration.getProperties(props, filter); } protected Configuration getHadoopConfiguration() { diff --git a/core/src/test/java/org/apache/accumulo/core/conf/CliConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/conf/CliConfigurationTest.java new file mode 100644 index 0000000000..e7f76dc795 --- /dev/null +++ b/core/src/test/java/org/apache/accumulo/core/conf/CliConfigurationTest.java @@ -0,0 +1,50 @@ +/* + * 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.conf; + +import java.util.HashMap; +import java.util.Map; + +import org.junit.Assert; +import org.junit.Test; + +import com.google.common.collect.ImmutableMap; + +public class CliConfigurationTest { + + @Test + public void testBasic() { + try { + CliConfiguration.set(null); + Assert.fail(); + } catch (NullPointerException e) { + // expected + } + CliConfiguration.set(new HashMap<>()); + + Map expected = new HashMap<>(); + expected.put(Property.TRACE_USER.getKey(), "test"); + expected.put(Property.TSERV_CLIENTPORT.getKey(), "123"); + CliConfiguration.set(expected); + + Assert.assertEquals("test", CliConfiguration.get(Property.TRACE_USER)); + + Map results = new HashMap<>(); + CliConfiguration.getProperties(results, p -> p.startsWith("trace")); + Assert.assertEquals(ImmutableMap.of(Property.TRACE_USER.getKey(), "test"), results); + } +} diff --git a/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java b/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java index 92a0da4df6..ee51d2868b 100644 --- a/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java +++ b/core/src/test/java/org/apache/accumulo/core/conf/SiteConfigurationTest.java @@ -28,6 +28,8 @@ import org.junit.BeforeClass; import org.junit.Test; +import com.google.common.collect.ImmutableMap; + public class SiteConfigurationTest { private static boolean isCredentialProviderAvailable; @@ -71,4 +73,13 @@ public void testOnlySensitivePropertiesExtractedFromCredentialProvider() throws Assert.assertEquals(null, props.get("ignored.property")); Assert.assertEquals(Property.GENERAL_RPC_TIMEOUT.getDefaultValue(), props.get(Property.GENERAL_RPC_TIMEOUT.getKey())); } + + @Test + public void testCliConfig() { + SiteConfiguration conf = SiteConfiguration.getInstance(); + Assert.assertEquals("localhost:2181", conf.get(Property.INSTANCE_ZK_HOST)); + + CliConfiguration.set(ImmutableMap.of(Property.INSTANCE_ZK_HOST.getKey(), "myhost:2181")); + Assert.assertEquals("myhost:2181", conf.get(Property.INSTANCE_ZK_HOST)); + } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java b/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java index bbe0dd28d0..b173ae33de 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java +++ b/server/base/src/main/java/org/apache/accumulo/server/ServerOpts.java @@ -16,17 +16,68 @@ */ package org.apache.accumulo.server; +import java.util.ArrayList; +import java.util.Collections; +import java.util.HashMap; +import java.util.List; +import java.util.Map; + import org.apache.accumulo.core.cli.Help; +import org.apache.accumulo.core.conf.CliConfiguration; import com.beust.jcommander.Parameter; +import com.beust.jcommander.converters.IParameterSplitter; public class ServerOpts extends Help { @Parameter(names = {"-a", "--address"}, description = "address to bind to") - String address = null; + private String address = null; + + public static class NullSplitter implements IParameterSplitter { + @Override + public List split(String value) { + return Collections.singletonList(value); + } + } + + @Parameter(names = "-o", splitter = NullSplitter.class, + description = "Overrides configuration set in accumulo-site.xml (but NOT system-wide config set in Zookeeper). Expected format: -o =") + private List properties = new ArrayList<>(); public String getAddress() { if (address != null) return address; return "0.0.0.0"; } + + public List getProperties() { + return properties; + } + + public Map getConfig() { + Map config = new HashMap<>(); + for (String prop : getProperties()) { + String[] propArgs = prop.split("=", 2); + if (propArgs.length == 2) { + String key = propArgs[0].trim(); + String value = propArgs[1].trim(); + if (key.isEmpty() || value.isEmpty()) { + throw new IllegalArgumentException("Invalid command line -o option: " + prop); + } else { + config.put(key, value); + } + } else { + throw new IllegalArgumentException("Invalid command line -o option: " + prop); + } + } + return config; + } + + @Override + public void parseArgs(String programName, String[] args, Object... others) { + super.parseArgs(programName, args, others); + CliConfiguration.set(getConfig()); + if (getConfig().size() > 0) { + CliConfiguration.print(); + } + } } diff --git a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java index be3869b105..819841872a 100644 --- a/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java +++ b/server/base/src/main/java/org/apache/accumulo/server/init/Initialize.java @@ -29,6 +29,7 @@ import java.util.HashMap; import java.util.HashSet; import java.util.Locale; +import java.util.Map; import java.util.Map.Entry; import java.util.Set; import java.util.TreeMap; @@ -100,6 +101,7 @@ import org.apache.accumulo.server.tables.TableManager; import org.apache.accumulo.server.tablets.TabletTime; import org.apache.accumulo.server.util.ReplicationTableUtil; +import org.apache.accumulo.server.util.SystemPropUtil; import org.apache.accumulo.server.util.TablePropUtil; import org.apache.accumulo.server.zookeeper.ZooReaderWriter; import org.apache.accumulo.start.spi.KeywordExecutable; @@ -393,6 +395,28 @@ private boolean initialize(Opts opts, String instanceNamePath, VolumeManager fs, log.error("FATAL: Failed to initialize security", e); return false; } + + if (opts.uploadAccumuloSite) { + try { + log.info("Uploading properties in accumulo-site.xml to Zookeeper. Properties that cannot be set in Zookeeper will be skipped:"); + Map entries = new TreeMap<>(); + SiteConfiguration.getInstance().getProperties(entries, x -> true, false); + for (Map.Entry entry : entries.entrySet()) { + String key = entry.getKey(); + String value = entry.getValue(); + if (Property.isValidZooPropertyKey(key)) { + SystemPropUtil.setSystemProperty(key, value); + log.info("Uploaded - {} = {}", key, Property.isSensitive(key) ? "" : value); + } else { + log.info("Skipped - {} = {}", key, Property.isSensitive(key) ? "" : value); + } + } + } catch (Exception e) { + log.error("FATAL: Failed to upload accumulo-site.xml to Zookeeper", e); + return false; + } + } + return true; } @@ -760,6 +784,8 @@ private static void addVolumes(VolumeManager fs) throws IOException { boolean forceResetSecurity = false; @Parameter(names = "--clear-instance-name", description = "delete any existing instance name without prompting") boolean clearInstanceName = false; + @Parameter(names = "--upload-accumulo-site", description = "Uploads properties in accumulo-site.xml to Zookeeper") + boolean uploadAccumuloSite = false; @Parameter(names = "--instance-name", description = "the instance name, if not provided, will prompt") String cliInstanceName; @Parameter(names = "--password", description = "set the password on the command line") diff --git a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java b/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java index 2c6889b0f3..fbd8e14bda 100644 --- a/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java +++ b/server/base/src/test/java/org/apache/accumulo/server/ServerOptsTest.java @@ -18,6 +18,9 @@ import static org.junit.Assert.assertEquals; +import org.apache.accumulo.core.conf.Property; +import org.apache.accumulo.core.conf.SiteConfiguration; +import org.junit.Assert; import org.junit.Before; import org.junit.Test; @@ -40,4 +43,12 @@ public void testGetAddress_NOne() { opts.parseArgs(ServerOptsTest.class.getName(), new String[] {}); assertEquals("0.0.0.0", opts.getAddress()); } + + @Test + public void testOverrideConfig() { + SiteConfiguration siteConf = SiteConfiguration.getInstance(); + Assert.assertEquals("localhost:2181", siteConf.get(Property.INSTANCE_ZK_HOST)); + opts.parseArgs(ServerOptsTest.class.getName(), new String[] {"-o", "instance.zookeeper.host=test:123"}); + Assert.assertEquals("test:123", siteConf.get(Property.INSTANCE_ZK_HOST)); + } } diff --git a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java index ba23505ca5..74ef6c91fe 100644 --- a/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java +++ b/server/gc/src/main/java/org/apache/accumulo/gc/SimpleGarbageCollector.java @@ -144,8 +144,10 @@ private GCStatus status = new GCStatus(new GcCycleStats(), new GcCycleStats(), new GcCycleStats(), new GcCycleStats()); - public static void main(String[] args) throws UnknownHostException, IOException { + public static void main(String[] args) throws IOException { final String app = "gc"; + Opts opts = new Opts(); + opts.parseArgs(app, args); SecurityUtil.serverLogin(SiteConfiguration.getInstance()); Instance instance = HdfsZooInstance.getInstance(); ServerConfigurationFactory conf = new ServerConfigurationFactory(instance); @@ -153,8 +155,6 @@ public static void main(String[] args) throws UnknownHostException, IOException log.info("Instance " + instance.getInstanceID()); final VolumeManager fs = VolumeManagerImpl.get(); Accumulo.init(fs, instance, conf, app); - Opts opts = new Opts(); - opts.parseArgs(app, args); SimpleGarbageCollector gc = new SimpleGarbageCollector(opts, instance, fs, conf); DistributedTrace.enable(opts.getAddress(), app, conf.getSystemConfiguration()); diff --git a/server/master/src/main/java/org/apache/accumulo/master/Master.java b/server/master/src/main/java/org/apache/accumulo/master/Master.java index 8660751789..1e72e4bf2b 100644 --- a/server/master/src/main/java/org/apache/accumulo/master/Master.java +++ b/server/master/src/main/java/org/apache/accumulo/master/Master.java @@ -1438,10 +1438,9 @@ private void getMasterLock(final String zMasterLoc) throws KeeperException, Inte public static void main(String[] args) throws Exception { try { final String app = "master"; - SecurityUtil.serverLogin(SiteConfiguration.getInstance()); - ServerOpts opts = new ServerOpts(); opts.parseArgs(app, args); + SecurityUtil.serverLogin(SiteConfiguration.getInstance()); String hostname = opts.getAddress(); Instance instance = HdfsZooInstance.getInstance(); ServerConfigurationFactory conf = new ServerConfigurationFactory(instance); diff --git a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java index b1d6db48ba..68317b376e 100644 --- a/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java +++ b/server/monitor/src/main/java/org/apache/accumulo/monitor/Monitor.java @@ -418,11 +418,10 @@ private static GCStatus fetchGcStatus() { public static void main(String[] args) throws Exception { final String app = "monitor"; - SecurityUtil.serverLogin(SiteConfiguration.getInstance()); - ServerOpts opts = new ServerOpts(); opts.parseArgs(app, args); String hostname = opts.getAddress(); + SecurityUtil.serverLogin(SiteConfiguration.getInstance()); VolumeManager fs = VolumeManagerImpl.get(); instance = HdfsZooInstance.getInstance(); diff --git a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java index c3e6d5f22d..16d72db0ec 100644 --- a/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java +++ b/server/tracer/src/main/java/org/apache/accumulo/tracer/TraceServer.java @@ -379,9 +379,9 @@ private static void loginTracer(AccumuloConfiguration acuConf) { public static void main(String[] args) throws Exception { final String app = "tracer"; - loginTracer(SiteConfiguration.getInstance()); ServerOpts opts = new ServerOpts(); opts.parseArgs(app, args); + loginTracer(SiteConfiguration.getInstance()); Instance instance = HdfsZooInstance.getInstance(); ServerConfigurationFactory conf = new ServerConfigurationFactory(instance); VolumeManager fs = VolumeManagerImpl.get(); diff --git a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java index 5e24e595e5..6bdcfd0c09 100644 --- a/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java +++ b/server/tserver/src/main/java/org/apache/accumulo/tserver/TabletServer.java @@ -2441,7 +2441,8 @@ private HostAndPort startReplicationService() throws UnknownHostException { AccumuloConfiguration conf = getServerConfigurationFactory().getSystemConfiguration(); Property maxMessageSizeProperty = (conf.get(Property.TSERV_MAX_MESSAGE_SIZE) != null ? Property.TSERV_MAX_MESSAGE_SIZE : Property.GENERAL_MAX_MESSAGE_SIZE); ServerAddress sp = TServerUtils.startServer(this, clientAddress.getHost(), Property.REPLICATION_RECEIPT_SERVICE_PORT, processor, - "ReplicationServicerHandler", "Replication Servicer", null, Property.REPLICATION_MIN_THREADS, Property.REPLICATION_THREADCHECK, maxMessageSizeProperty); + "ReplicationServicerHandler", "Replication Servicer", Property.TSERV_PORTSEARCH, Property.REPLICATION_MIN_THREADS, Property.REPLICATION_THREADCHECK, + maxMessageSizeProperty); this.replServer = sp.server; log.info("Started replication service on {}", sp.address); @@ -3061,9 +3062,9 @@ public TabletServerStatus getStats(Map> scanCo public static void main(String[] args) throws IOException { try { final String app = "tserver"; - SecurityUtil.serverLogin(SiteConfiguration.getInstance()); ServerOpts opts = new ServerOpts(); opts.parseArgs(app, args); + SecurityUtil.serverLogin(SiteConfiguration.getInstance()); String hostname = opts.getAddress(); Instance instance = HdfsZooInstance.getInstance(); ServerConfigurationFactory conf = new ServerConfigurationFactory(instance); ---------------------------------------------------------------- This is an automated message from the Apache Git Service. To respond to the message, please log on GitHub and use the URL above to go to the specific comment. For queries about this service, please contact Infrastructure at: users@infra.apache.org With regards, Apache Git Services