Index: conf/hbase-site.xml IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- conf/hbase-site.xml (revision 1403148) +++ conf/hbase-site.xml (revision ) @@ -21,4 +21,25 @@ */ --> + + + + hbase.rootdir + hdfs://localhost:9000/hbase-trunk0812 + + + hbase.cluster.distributed + true + + + hbase.dynamic.configuration.enabled + true + + Enables the dynamic cluster configuration ability.If enabled, you can make + runtime cluster configuration changes with out the need for restarting the cluster. + + Index: hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/RequestConverter.java (revision ) @@ -92,17 +92,19 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyColumnRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ModifyTableRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MoveRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.OfflineRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest; -import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetTableDescriptorsRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.SetBalancerRunningRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest; import org.apache.hadoop.hbase.protobuf.generated.RegionServerStatusProtos.GetLastFlushedSequenceIdRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest; import org.apache.hadoop.hbase.regionserver.wal.HLog; import org.apache.hadoop.hbase.regionserver.wal.HLogKey; import org.apache.hadoop.hbase.regionserver.wal.WALEdit; @@ -1159,4 +1161,19 @@ } return builder.build(); } + + public static GetConfigRequest buildGetConfigRequest(String configKey) { + GetConfigRequest.Builder builder = GetConfigRequest.newBuilder(); + builder.setConfigKey(configKey); + return builder.build(); + } + + public static UpdateConfigRequest buildUpdateConfigRequest(String configKey, String configValue) { + UpdateConfigRequest.Builder builder = UpdateConfigRequest.newBuilder(); + builder.setConfigKey(configKey); + builder.setConfigValue(configValue); + return builder.build(); + } + + } Index: hbase-server/src/main/ruby/shell.rb IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/ruby/shell.rb (revision 1403148) +++ hbase-server/src/main/ruby/shell.rb (revision ) @@ -289,6 +289,8 @@ catalogjanitor_run catalogjanitor_switch catalogjanitor_enabled + update_config + get_config ] ) Index: hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseInMemoryConfiguration.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseInMemoryConfiguration.java (revision ) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/HBaseInMemoryConfiguration.java (revision ) @@ -0,0 +1,267 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + + +package org.apache.hadoop.hbase; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.hadoop.hbase.zookeeper.ZKUtil; +import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.util.Collection; +import java.util.Collections; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ConcurrentHashMap; + +/** + * Holds current HBase cluster configuration in memory. Extends Configuration API and delegates to + * underlying configuration object where necessary. + * + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class HBaseInMemoryConfiguration extends Configuration { + + private static final Log LOG = LogFactory.getLog(HBaseInMemoryConfiguration.class); + + private ConcurrentHashMap clusterConfigMap + = new ConcurrentHashMap(100); + + public HBaseInMemoryConfiguration(Configuration configuration) { + super(configuration); + } + + @Override + public String get(String configKey) { + if (clusterConfigMap.containsKey(configKey)) { + return clusterConfigMap.get(configKey); + } + return super.get(configKey); + } + + @Override + public String getRaw(String name) { + return super.getRaw(name); + } + + public void put(String configKey, String configValue) { + this.clusterConfigMap.put(configKey, configValue); + } + + @Override + public long getLong(String name, long defaultValue) { + if (clusterConfigMap.containsKey(name)) { + String longValue = clusterConfigMap.get(name); + try { + return longValue == null ? defaultValue : Long.parseLong(longValue); + } catch (NumberFormatException nfe) { + return super.getLong(name, defaultValue); + } + } + return super.getLong(name, defaultValue); + } + + @Override + public int getInt(String name, int defaultValue) { + if (clusterConfigMap.containsKey(name)) { + String intValue = clusterConfigMap.get(name); + try { + return intValue == null ? defaultValue : Integer.parseInt(intValue); + } catch (NumberFormatException nfe) { + return super.getInt(name, defaultValue); + } + } + return super.getInt(name, defaultValue); + } + + @Override + public String get(String name, String defaultValue) { + if (clusterConfigMap.containsKey(name)) { + String stringValue = clusterConfigMap.get(name); + return stringValue == null ? defaultValue : stringValue; + } + return super.get(name, defaultValue); + } + + @Override + public float getFloat(String name, float defaultValue) { + if (clusterConfigMap.containsKey(name)) { + String floatValue = clusterConfigMap.get(name); + try { + return floatValue == null ? defaultValue : Float.parseFloat(floatValue); + } catch (NumberFormatException nfe) { + return super.getFloat(name, defaultValue); + } + } + return super.getFloat(name, defaultValue); + } + + @Override + public boolean getBoolean(String name, boolean defaultValue) { + if (clusterConfigMap.containsKey(name)) { + String booleanValue = clusterConfigMap.get(name); + if (booleanValue == null) { + return defaultValue; + } + booleanValue = booleanValue.trim(); + if ("true".equals(booleanValue)) { + return true; + } else if ("false".equals(booleanValue)) { + return false; + } else { + return defaultValue; + } + } + return super.getBoolean(name, defaultValue); + } + + + @Override + public void set(String name, String value) { + super.set(name, value); + clusterConfigMap.put(name, value); + } + + @Override + public void setIfUnset(String name, String value) { + if (clusterConfigMap.get(name) == null) { + set(name, value); + } + } + + @Override + public void setInt(String name, int value) { + set(name, Integer.toString(value)); + } + + @Override + public void setLong(String name, long value) { + set(name, Long.toString(value)); + } + + @Override + public void setFloat(String name, float value) { + set(name, Float.toString(value)); + } + + @Override + public void setBoolean(String name, boolean value) { + set(name, Boolean.toString(value)); + } + + @Override + public void setBooleanIfUnset(String name, boolean value) { + setIfUnset(name, Boolean.toString(value)); + } + + @Override + public > void setEnum(String name, T value) { + set(name, value.toString()); + } + + @Override + public > T getEnum(String name, T defaultValue) { + return super.getEnum(name, defaultValue); + } + + @Override + public IntegerRanges getRange(String name, String defaultValue) { + return super.getRange(name, defaultValue); + } + + @Override + public Collection getStringCollection(String name) { + return super.getStringCollection(name); + } + + @Override + public String[] getStrings(String name) { + return super.getStrings(name); + } + + @Override + public String[] getStrings(String name, String... defaultValue) { + return super.getStrings(name, defaultValue); + } + + @Override + public void setStrings(String name, String... values) { + set(name, StringUtils.arrayToString(values)); + } + + @Override + public Class getClassByName(String name) throws ClassNotFoundException { + return super.getClassByName(name); + } + + @Override + public Class[] getClasses(String name, Class... defaultValue) { + return super.getClasses(name, defaultValue); + } + + @Override + public Class getClass(String name, Class defaultValue) { + return super.getClass(name, defaultValue); + } + + @Override + public Class getClass(String name, Class defaultValue, + Class xface) { + return super.getClass(name, defaultValue, xface); + } + + @Override + public void setClass(String name, Class theClass, Class xface) { + super.setClass(name, theClass, xface); + clusterConfigMap.put(name, theClass.getName()); + } + + /** + * Rebuild the cluster configuration map from ZK node data. + * @param clusterData + */ + public void refreshClusterConfigMap(List clusterData) { + if (clusterData != null) { + LOG.debug("Found Cluster configuration in ZK. Refreshing the in memory " + + "configuration map"); + for (ZKUtil.NodeAndData nodeData : clusterData) { + String configNodePath = nodeData.getNode(); + String configKey = configNodePath.substring(configNodePath.lastIndexOf("/") + 1, + configNodePath.length()); + clusterConfigMap.put(configKey, Bytes.toString(nodeData.getData())); + } + LOG.debug("Refreshed cluster configuration map. Current size = " + + clusterConfigMap.size()); + } + } + + /** + * Get a read only version of the current configuration map. + * + * @return + */ + public Map getClusterConfigMap() { + return Collections.unmodifiableMap(clusterConfigMap); + } +} Index: hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseDynamicConfiguration.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseDynamicConfiguration.java (revision ) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/TestHBaseDynamicConfiguration.java (revision ) @@ -0,0 +1,292 @@ +package org.apache.hadoop.hbase; + +/** + * 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. + */ + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.master.HMaster; +import org.apache.hadoop.hbase.regionserver.HRegionServer; +import org.apache.hadoop.hbase.zookeeper.ClusterConfigTracker; +import org.codehaus.jackson.map.ObjectMapper; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.junit.experimental.categories.Category; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.Map; + +import static org.junit.Assert.*; + +@Category(MediumTests.class) +public class TestHBaseDynamicConfiguration { + + final Log LOG = LogFactory.getLog(getClass()); + protected final static HBaseTestingUtility TEST_UTIL = new HBaseTestingUtility(); + protected static MiniHBaseCluster miniHBaseCluster = null; + protected Configuration conf; + + @Before + public void setUpBeforeClass() throws Exception { + TEST_UTIL.getConfiguration().setBoolean("hbase.dynamic.configuration.enabled", true); + miniHBaseCluster = TEST_UTIL.startMiniCluster(1, 1); + } + + @After + public void tearDownAfterClass() throws Exception { + TEST_UTIL.shutdownMiniCluster(); + } + + @Test + public void testConfigurationChangesWhileMasterCrash() throws IOException, InterruptedException { + LOG.info("Start testConfigurationChangesWhileMasterCrash"); + + Configuration clusterConfig = TEST_UTIL.getConfiguration(); + int balancerPeriod = clusterConfig.getInt("hbase.balancer.period", 10); + assertEquals(balancerPeriod, 300000); + int handlerCount = clusterConfig.getInt("hbase.master.handler.count", 30); + assertEquals(handlerCount, 30); + long rsFatalsBuffer = clusterConfig.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024); + assertEquals(rsFatalsBuffer, 1 * 1024 * 1024); + long catalogTimeout = clusterConfig.getInt("hbase.master.catalog.timeout", Integer.MAX_VALUE); + assertEquals(catalogTimeout, 600000); + long catalogVerifyTimeout = clusterConfig.getInt("hbase.catalog.verification.timeout", 1000); + assertEquals(catalogVerifyTimeout, 1000); + + ClusterConfigTracker cct = miniHBaseCluster.getMaster().getClusterConfigTracker(); + updateClusterConfig(cct); + + Thread.sleep(5000); + HBaseInMemoryConfiguration masterConfig = (HBaseInMemoryConfiguration) + miniHBaseCluster.getMaster().getConfiguration(); + assertConfigChanges(masterConfig); + + TEST_UTIL.getHBaseCluster().startMaster(); + Thread.sleep(10000); + + HMaster activeMaster = TEST_UTIL.getHBaseCluster().getMaster(); + activeMaster.abort("Aborting primary master to test dynamic configuration overrides. ", + new Exception("Die now")); + Thread.sleep(15000); + + HMaster secondMaster = TEST_UTIL.getHBaseCluster().getMaster(); + HBaseInMemoryConfiguration secondMasterConfig = + (HBaseInMemoryConfiguration) secondMaster.getConfiguration(); + assertConfigChanges(secondMasterConfig); + + HRegionServer hRegionServer = TEST_UTIL.getHBaseCluster().getRegionServer(0); + HBaseInMemoryConfiguration regionConfig = + (HBaseInMemoryConfiguration) hRegionServer.getConfiguration(); + assertConfigChanges(regionConfig); + + LOG.info("End testConfigurationChangesWhileMasterCrash"); + } + + private void updateClusterConfig(ClusterConfigTracker cct) { + cct.updateClusterConfig("hbase.balancer.period", "200000"); + cct.updateClusterConfig("hbase.master.handler.count", "30"); + cct.updateClusterConfig("hbase.master.buffer.for.rs.fatals", Long.toString(2 * 1024 * 1024)); + cct.updateClusterConfig("hbase.master.catalog.timeout", Integer.toString(Integer.MAX_VALUE - 100)); + cct.updateClusterConfig("hbase.catalog.verification.timeout", Long.toString(2000)); + } + + private void assertConfigChanges(Configuration configuration) { + int balancePeriod = configuration.getInt("hbase.balancer.period", 40000); + LOG.info("Balancer period from second master config = " + balancePeriod); + assertEquals(balancePeriod, 200000); + int handlerCount = configuration.getInt("hbase.master.handler.count", 20); + assertEquals(handlerCount, 30); + long rsFatalsBuffer = configuration.getLong("hbase.master.buffer.for.rs.fatals", 1 * 1024 * 1024); + assertEquals(rsFatalsBuffer, 2 * 1024 * 1024); + int catalogTimeout = configuration.getInt("hbase.master.catalog.timeout", Integer.MAX_VALUE); + assertEquals(catalogTimeout, Integer.MAX_VALUE - 100); + int catalogVerifyTimeout = configuration.getInt("hbase.catalog.verification.timeout", 1000); + assertEquals(catalogVerifyTimeout, 2000); + } + + @Test + public void testDumpConfiguration() throws Exception { + Configuration conf = TEST_UTIL.getConfiguration(); + StringWriter outWriter = new StringWriter(); + conf.dumpConfiguration(conf, outWriter); + String jsonStr = outWriter.toString(); + ObjectMapper mapper = new ObjectMapper(); + JsonConfiguration jconf = mapper.readValue(jsonStr, JsonConfiguration.class); + JsonProperty[] props = jconf.getProperties(); + if (props != null) { + System.err.println("Props length = " + props.length); + } + System.err.println("Done"); + assertEquals("1", "1"); + } + + @Test + public void testConfigMemoryOverride() throws Exception { + LOG.info("Start testConfigMemoryOverride"); + + Configuration clusterConfig = TEST_UTIL.getConfiguration(); + int balancerPeriod = clusterConfig.getInt("hbase.balancer.period", 10); + assertEquals(balancerPeriod, 300000); + ClusterConfigTracker cct = miniHBaseCluster.getMaster().getClusterConfigTracker(); + cct.updateClusterConfig("hbase.balancer.period", "200000"); + Thread.sleep(1000); + HBaseInMemoryConfiguration masterConfig = (HBaseInMemoryConfiguration) + miniHBaseCluster.getMaster().getConfiguration(); + int newBalancerPeriod = masterConfig.getInt("hbase.balancer.period", 40000); + assertEquals(newBalancerPeriod, 200000); + LOG.info("End testConfigMemoryOverride"); + + } + + + @Test + public void testGetClusterConfigMap() throws Exception { + LOG.info("Start testGetClusterConfigMap"); + ClusterConfigTracker cct = miniHBaseCluster.getMaster().getClusterConfigTracker(); + Map configMap = cct.getClusterConfigMap(); + assertNotNull(configMap); + assertTrue(configMap.size() > 0); + LOG.info("End testGetClusterConfigMap"); + + } + + @Test + public void testConfigAddition() throws Exception { + LOG.info("Start testConfigAddition"); + + ClusterConfigTracker cct = miniHBaseCluster.getMaster().getClusterConfigTracker(); + int sizeBefore = cct.getClusterConfigMap().size(); + cct.updateClusterConfig("hbase.config.test.TestConfigKey", "TestConfigValue"); + Thread.sleep(1000); + int sizeAfter = cct.getClusterConfigMap().size(); + assertEquals(sizeBefore + 1, sizeAfter); + String configValue = + (String) cct.getClusterConfigMap().get("hbase.config.test.TestConfigKey"); + assertEquals(configValue, "TestConfigValue"); + LOG.info("End testConfigAddition"); + + } + + @Test + public void testConfigDuplicateAddition() throws Exception { + LOG.info("Start testConfigDuplicateAddition"); + + ClusterConfigTracker cct = miniHBaseCluster.getMaster().getClusterConfigTracker(); + int sizeBefore = cct.getClusterConfigMap().size(); + cct.updateClusterConfig("hbase.config.test.TestConfigKey", "TestConfigValue"); + Thread.sleep(1000); + int sizeAfter = cct.getClusterConfigMap().size(); + assertEquals(sizeAfter, sizeBefore + 1); + String configValue = + (String) cct.getClusterConfigMap().get("hbase.config.test.TestConfigKey"); + assertEquals(configValue, "TestConfigValue"); + // Now try to create the same config again. This should simply update the config + // value + cct.updateClusterConfig("hbase.config.test.TestConfigKey", "TestConfigValue201"); + Thread.sleep(1000); + sizeAfter = cct.getClusterConfigMap().size(); + assertEquals(sizeAfter, sizeBefore + 1); + configValue = + (String) cct.getClusterConfigMap().get("hbase.config.test.TestConfigKey"); + assertEquals(configValue, "TestConfigValue201"); + LOG.info("End testConfigDuplicateAddition"); + + } + + @Test + public void testConfigUpdate() throws Exception { + LOG.info("Start testConfigUpdate"); + + ClusterConfigTracker cct = miniHBaseCluster.getMaster().getClusterConfigTracker(); + int sizeBefore = cct.getClusterConfigMap().size(); + cct.updateClusterConfig("hbase.config.test.TestConfigKey", "TestConfigValue"); + Thread.sleep(1000); + int sizeAfter = cct.getClusterConfigMap().size(); + assertEquals(sizeAfter, sizeBefore + 1); + String configValue = + (String) cct.getClusterConfigMap().get("hbase.config.test.TestConfigKey"); + assertEquals(configValue, "TestConfigValue"); + sizeBefore = cct.getClusterConfigMap().size(); + cct.updateClusterConfig("hbase.config.test.TestConfigKey", "TestConfigValue101"); + Thread.sleep(1000); + sizeAfter = cct.getClusterConfigMap().size(); + // No change in config size map since we are only updating existing value + assertEquals(sizeAfter, sizeBefore); + configValue = + (String) cct.getClusterConfigMap().get("hbase.config.test.TestConfigKey"); + assertEquals(configValue, "TestConfigValue101"); + LOG.info("End testConfigUpdate"); + + } + + static class JsonConfiguration { + JsonProperty[] properties; + + public JsonProperty[] getProperties() { + return properties; + } + + public void setProperties(JsonProperty[] properties) { + this.properties = properties; + } + } + + static class JsonProperty { + String key; + + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + public boolean getIsFinal() { + return isFinal; + } + + public void setIsFinal(boolean isFinal) { + this.isFinal = isFinal; + } + + public String getResource() { + return resource; + } + + public void setResource(String resource) { + this.resource = resource; + } + + String value; + boolean isFinal; + String resource; + } + +} Index: hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterAdminProtos.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterAdminProtos.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/protobuf/generated/MasterAdminProtos.java (revision ) @@ -14364,6 +14364,1736 @@ // @@protoc_insertion_point(class_scope:IsCatalogJanitorEnabledResponse) } + public interface UpdateConfigRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string configKey = 1; + boolean hasConfigKey(); + String getConfigKey(); + + // required string configValue = 2; + boolean hasConfigValue(); + String getConfigValue(); + } + public static final class UpdateConfigRequest extends + com.google.protobuf.GeneratedMessage + implements UpdateConfigRequestOrBuilder { + // Use UpdateConfigRequest.newBuilder() to construct. + private UpdateConfigRequest(Builder builder) { + super(builder); + } + private UpdateConfigRequest(boolean noInit) {} + + private static final UpdateConfigRequest defaultInstance; + public static UpdateConfigRequest getDefaultInstance() { + return defaultInstance; + } + + public UpdateConfigRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigRequest_fieldAccessorTable; + } + + private int bitField0_; + // required string configKey = 1; + public static final int CONFIGKEY_FIELD_NUMBER = 1; + private java.lang.Object configKey_; + public boolean hasConfigKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getConfigKey() { + java.lang.Object ref = configKey_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + configKey_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getConfigKeyBytes() { + java.lang.Object ref = configKey_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + configKey_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + // required string configValue = 2; + public static final int CONFIGVALUE_FIELD_NUMBER = 2; + private java.lang.Object configValue_; + public boolean hasConfigValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getConfigValue() { + java.lang.Object ref = configValue_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + configValue_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getConfigValueBytes() { + java.lang.Object ref = configValue_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + configValue_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + configKey_ = ""; + configValue_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConfigKey()) { + memoizedIsInitialized = 0; + return false; + } + if (!hasConfigValue()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getConfigKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + output.writeBytes(2, getConfigValueBytes()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getConfigKeyBytes()); + } + if (((bitField0_ & 0x00000002) == 0x00000002)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(2, getConfigValueBytes()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest) obj; + + boolean result = true; + result = result && (hasConfigKey() == other.hasConfigKey()); + if (hasConfigKey()) { + result = result && getConfigKey() + .equals(other.getConfigKey()); + } + result = result && (hasConfigValue() == other.hasConfigValue()); + if (hasConfigValue()) { + result = result && getConfigValue() + .equals(other.getConfigValue()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasConfigKey()) { + hash = (37 * hash) + CONFIGKEY_FIELD_NUMBER; + hash = (53 * hash) + getConfigKey().hashCode(); + } + if (hasConfigValue()) { + hash = (37 * hash) + CONFIGVALUE_FIELD_NUMBER; + hash = (53 * hash) + getConfigValue().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + configKey_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + configValue_ = ""; + bitField0_ = (bitField0_ & ~0x00000002); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest build() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.configKey_ = configKey_; + if (((from_bitField0_ & 0x00000002) == 0x00000002)) { + to_bitField0_ |= 0x00000002; + } + result.configValue_ = configValue_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.getDefaultInstance()) return this; + if (other.hasConfigKey()) { + setConfigKey(other.getConfigKey()); + } + if (other.hasConfigValue()) { + setConfigValue(other.getConfigValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasConfigKey()) { + + return false; + } + if (!hasConfigValue()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + configKey_ = input.readBytes(); + break; + } + case 18: { + bitField0_ |= 0x00000002; + configValue_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string configKey = 1; + private java.lang.Object configKey_ = ""; + public boolean hasConfigKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getConfigKey() { + java.lang.Object ref = configKey_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + configKey_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setConfigKey(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + configKey_ = value; + onChanged(); + return this; + } + public Builder clearConfigKey() { + bitField0_ = (bitField0_ & ~0x00000001); + configKey_ = getDefaultInstance().getConfigKey(); + onChanged(); + return this; + } + void setConfigKey(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + configKey_ = value; + onChanged(); + } + + // required string configValue = 2; + private java.lang.Object configValue_ = ""; + public boolean hasConfigValue() { + return ((bitField0_ & 0x00000002) == 0x00000002); + } + public String getConfigValue() { + java.lang.Object ref = configValue_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + configValue_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setConfigValue(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000002; + configValue_ = value; + onChanged(); + return this; + } + public Builder clearConfigValue() { + bitField0_ = (bitField0_ & ~0x00000002); + configValue_ = getDefaultInstance().getConfigValue(); + onChanged(); + return this; + } + void setConfigValue(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000002; + configValue_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:UpdateConfigRequest) + } + + static { + defaultInstance = new UpdateConfigRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:UpdateConfigRequest) + } + + public interface UpdateConfigResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // optional bool updateConfigResponse = 1; + boolean hasUpdateConfigResponse(); + boolean getUpdateConfigResponse(); + } + public static final class UpdateConfigResponse extends + com.google.protobuf.GeneratedMessage + implements UpdateConfigResponseOrBuilder { + // Use UpdateConfigResponse.newBuilder() to construct. + private UpdateConfigResponse(Builder builder) { + super(builder); + } + private UpdateConfigResponse(boolean noInit) {} + + private static final UpdateConfigResponse defaultInstance; + public static UpdateConfigResponse getDefaultInstance() { + return defaultInstance; + } + + public UpdateConfigResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigResponse_fieldAccessorTable; + } + + private int bitField0_; + // optional bool updateConfigResponse = 1; + public static final int UPDATECONFIGRESPONSE_FIELD_NUMBER = 1; + private boolean updateConfigResponse_; + public boolean hasUpdateConfigResponse() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public boolean getUpdateConfigResponse() { + return updateConfigResponse_; + } + + private void initFields() { + updateConfigResponse_ = false; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBool(1, updateConfigResponse_); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBoolSize(1, updateConfigResponse_); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse) obj; + + boolean result = true; + result = result && (hasUpdateConfigResponse() == other.hasUpdateConfigResponse()); + if (hasUpdateConfigResponse()) { + result = result && (getUpdateConfigResponse() + == other.getUpdateConfigResponse()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasUpdateConfigResponse()) { + hash = (37 * hash) + UPDATECONFIGRESPONSE_FIELD_NUMBER; + hash = (53 * hash) + hashBoolean(getUpdateConfigResponse()); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_UpdateConfigResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + updateConfigResponse_ = false; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse build() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.updateConfigResponse_ = updateConfigResponse_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDefaultInstance()) return this; + if (other.hasUpdateConfigResponse()) { + setUpdateConfigResponse(other.getUpdateConfigResponse()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 8: { + bitField0_ |= 0x00000001; + updateConfigResponse_ = input.readBool(); + break; + } + } + } + } + + private int bitField0_; + + // optional bool updateConfigResponse = 1; + private boolean updateConfigResponse_ ; + public boolean hasUpdateConfigResponse() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public boolean getUpdateConfigResponse() { + return updateConfigResponse_; + } + public Builder setUpdateConfigResponse(boolean value) { + bitField0_ |= 0x00000001; + updateConfigResponse_ = value; + onChanged(); + return this; + } + public Builder clearUpdateConfigResponse() { + bitField0_ = (bitField0_ & ~0x00000001); + updateConfigResponse_ = false; + onChanged(); + return this; + } + + // @@protoc_insertion_point(builder_scope:UpdateConfigResponse) + } + + static { + defaultInstance = new UpdateConfigResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:UpdateConfigResponse) + } + + public interface GetConfigRequestOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string configKey = 1; + boolean hasConfigKey(); + String getConfigKey(); + } + public static final class GetConfigRequest extends + com.google.protobuf.GeneratedMessage + implements GetConfigRequestOrBuilder { + // Use GetConfigRequest.newBuilder() to construct. + private GetConfigRequest(Builder builder) { + super(builder); + } + private GetConfigRequest(boolean noInit) {} + + private static final GetConfigRequest defaultInstance; + public static GetConfigRequest getDefaultInstance() { + return defaultInstance; + } + + public GetConfigRequest getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigRequest_fieldAccessorTable; + } + + private int bitField0_; + // required string configKey = 1; + public static final int CONFIGKEY_FIELD_NUMBER = 1; + private java.lang.Object configKey_; + public boolean hasConfigKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getConfigKey() { + java.lang.Object ref = configKey_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + configKey_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getConfigKeyBytes() { + java.lang.Object ref = configKey_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + configKey_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + configKey_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConfigKey()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getConfigKeyBytes()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getConfigKeyBytes()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest other = (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest) obj; + + boolean result = true; + result = result && (hasConfigKey() == other.hasConfigKey()); + if (hasConfigKey()) { + result = result && getConfigKey() + .equals(other.getConfigKey()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasConfigKey()) { + hash = (37 * hash) + CONFIGKEY_FIELD_NUMBER; + hash = (53 * hash) + getConfigKey().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequestOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigRequest_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigRequest_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + configKey_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest build() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest result = new org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.configKey_ = configKey_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.getDefaultInstance()) return this; + if (other.hasConfigKey()) { + setConfigKey(other.getConfigKey()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasConfigKey()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + configKey_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string configKey = 1; + private java.lang.Object configKey_ = ""; + public boolean hasConfigKey() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getConfigKey() { + java.lang.Object ref = configKey_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + configKey_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setConfigKey(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + configKey_ = value; + onChanged(); + return this; + } + public Builder clearConfigKey() { + bitField0_ = (bitField0_ & ~0x00000001); + configKey_ = getDefaultInstance().getConfigKey(); + onChanged(); + return this; + } + void setConfigKey(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + configKey_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:GetConfigRequest) + } + + static { + defaultInstance = new GetConfigRequest(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GetConfigRequest) + } + + public interface GetConfigResponseOrBuilder + extends com.google.protobuf.MessageOrBuilder { + + // required string configValue = 1; + boolean hasConfigValue(); + String getConfigValue(); + } + public static final class GetConfigResponse extends + com.google.protobuf.GeneratedMessage + implements GetConfigResponseOrBuilder { + // Use GetConfigResponse.newBuilder() to construct. + private GetConfigResponse(Builder builder) { + super(builder); + } + private GetConfigResponse(boolean noInit) {} + + private static final GetConfigResponse defaultInstance; + public static GetConfigResponse getDefaultInstance() { + return defaultInstance; + } + + public GetConfigResponse getDefaultInstanceForType() { + return defaultInstance; + } + + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigResponse_fieldAccessorTable; + } + + private int bitField0_; + // required string configValue = 1; + public static final int CONFIGVALUE_FIELD_NUMBER = 1; + private java.lang.Object configValue_; + public boolean hasConfigValue() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getConfigValue() { + java.lang.Object ref = configValue_; + if (ref instanceof String) { + return (String) ref; + } else { + com.google.protobuf.ByteString bs = + (com.google.protobuf.ByteString) ref; + String s = bs.toStringUtf8(); + if (com.google.protobuf.Internal.isValidUtf8(bs)) { + configValue_ = s; + } + return s; + } + } + private com.google.protobuf.ByteString getConfigValueBytes() { + java.lang.Object ref = configValue_; + if (ref instanceof String) { + com.google.protobuf.ByteString b = + com.google.protobuf.ByteString.copyFromUtf8((String) ref); + configValue_ = b; + return b; + } else { + return (com.google.protobuf.ByteString) ref; + } + } + + private void initFields() { + configValue_ = ""; + } + private byte memoizedIsInitialized = -1; + public final boolean isInitialized() { + byte isInitialized = memoizedIsInitialized; + if (isInitialized != -1) return isInitialized == 1; + + if (!hasConfigValue()) { + memoizedIsInitialized = 0; + return false; + } + memoizedIsInitialized = 1; + return true; + } + + public void writeTo(com.google.protobuf.CodedOutputStream output) + throws java.io.IOException { + getSerializedSize(); + if (((bitField0_ & 0x00000001) == 0x00000001)) { + output.writeBytes(1, getConfigValueBytes()); + } + getUnknownFields().writeTo(output); + } + + private int memoizedSerializedSize = -1; + public int getSerializedSize() { + int size = memoizedSerializedSize; + if (size != -1) return size; + + size = 0; + if (((bitField0_ & 0x00000001) == 0x00000001)) { + size += com.google.protobuf.CodedOutputStream + .computeBytesSize(1, getConfigValueBytes()); + } + size += getUnknownFields().getSerializedSize(); + memoizedSerializedSize = size; + return size; + } + + private static final long serialVersionUID = 0L; + @java.lang.Override + protected java.lang.Object writeReplace() + throws java.io.ObjectStreamException { + return super.writeReplace(); + } + + @java.lang.Override + public boolean equals(final java.lang.Object obj) { + if (obj == this) { + return true; + } + if (!(obj instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse)) { + return super.equals(obj); + } + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse other = (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse) obj; + + boolean result = true; + result = result && (hasConfigValue() == other.hasConfigValue()); + if (hasConfigValue()) { + result = result && getConfigValue() + .equals(other.getConfigValue()); + } + result = result && + getUnknownFields().equals(other.getUnknownFields()); + return result; + } + + @java.lang.Override + public int hashCode() { + int hash = 41; + hash = (19 * hash) + getDescriptorForType().hashCode(); + if (hasConfigValue()) { + hash = (37 * hash) + CONFIGVALUE_FIELD_NUMBER; + hash = (53 * hash) + getConfigValue().hashCode(); + } + hash = (29 * hash) + getUnknownFields().hashCode(); + return hash; + } + + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom( + com.google.protobuf.ByteString data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom( + com.google.protobuf.ByteString data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom(byte[] data) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom( + byte[] data, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws com.google.protobuf.InvalidProtocolBufferException { + return newBuilder().mergeFrom(data, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom(java.io.InputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseDelimitedFrom(java.io.InputStream input) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseDelimitedFrom( + java.io.InputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + Builder builder = newBuilder(); + if (builder.mergeDelimitedFrom(input, extensionRegistry)) { + return builder.buildParsed(); + } else { + return null; + } + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom( + com.google.protobuf.CodedInputStream input) + throws java.io.IOException { + return newBuilder().mergeFrom(input).buildParsed(); + } + public static org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse parseFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + return newBuilder().mergeFrom(input, extensionRegistry) + .buildParsed(); + } + + public static Builder newBuilder() { return Builder.create(); } + public Builder newBuilderForType() { return newBuilder(); } + public static Builder newBuilder(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse prototype) { + return newBuilder().mergeFrom(prototype); + } + public Builder toBuilder() { return newBuilder(this); } + + @java.lang.Override + protected Builder newBuilderForType( + com.google.protobuf.GeneratedMessage.BuilderParent parent) { + Builder builder = new Builder(parent); + return builder; + } + public static final class Builder extends + com.google.protobuf.GeneratedMessage.Builder + implements org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponseOrBuilder { + public static final com.google.protobuf.Descriptors.Descriptor + getDescriptor() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigResponse_descriptor; + } + + protected com.google.protobuf.GeneratedMessage.FieldAccessorTable + internalGetFieldAccessorTable() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.internal_static_GetConfigResponse_fieldAccessorTable; + } + + // Construct using org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.newBuilder() + private Builder() { + maybeForceBuilderInitialization(); + } + + private Builder(BuilderParent parent) { + super(parent); + maybeForceBuilderInitialization(); + } + private void maybeForceBuilderInitialization() { + if (com.google.protobuf.GeneratedMessage.alwaysUseFieldBuilders) { + } + } + private static Builder create() { + return new Builder(); + } + + public Builder clear() { + super.clear(); + configValue_ = ""; + bitField0_ = (bitField0_ & ~0x00000001); + return this; + } + + public Builder clone() { + return create().mergeFrom(buildPartial()); + } + + public com.google.protobuf.Descriptors.Descriptor + getDescriptorForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDescriptor(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse getDefaultInstanceForType() { + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDefaultInstance(); + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse build() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException(result); + } + return result; + } + + private org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse buildParsed() + throws com.google.protobuf.InvalidProtocolBufferException { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse result = buildPartial(); + if (!result.isInitialized()) { + throw newUninitializedMessageException( + result).asInvalidProtocolBufferException(); + } + return result; + } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse buildPartial() { + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse result = new org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse(this); + int from_bitField0_ = bitField0_; + int to_bitField0_ = 0; + if (((from_bitField0_ & 0x00000001) == 0x00000001)) { + to_bitField0_ |= 0x00000001; + } + result.configValue_ = configValue_; + result.bitField0_ = to_bitField0_; + onBuilt(); + return result; + } + + public Builder mergeFrom(com.google.protobuf.Message other) { + if (other instanceof org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse) { + return mergeFrom((org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse)other); + } else { + super.mergeFrom(other); + return this; + } + } + + public Builder mergeFrom(org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse other) { + if (other == org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDefaultInstance()) return this; + if (other.hasConfigValue()) { + setConfigValue(other.getConfigValue()); + } + this.mergeUnknownFields(other.getUnknownFields()); + return this; + } + + public final boolean isInitialized() { + if (!hasConfigValue()) { + + return false; + } + return true; + } + + public Builder mergeFrom( + com.google.protobuf.CodedInputStream input, + com.google.protobuf.ExtensionRegistryLite extensionRegistry) + throws java.io.IOException { + com.google.protobuf.UnknownFieldSet.Builder unknownFields = + com.google.protobuf.UnknownFieldSet.newBuilder( + this.getUnknownFields()); + while (true) { + int tag = input.readTag(); + switch (tag) { + case 0: + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + default: { + if (!parseUnknownField(input, unknownFields, + extensionRegistry, tag)) { + this.setUnknownFields(unknownFields.build()); + onChanged(); + return this; + } + break; + } + case 10: { + bitField0_ |= 0x00000001; + configValue_ = input.readBytes(); + break; + } + } + } + } + + private int bitField0_; + + // required string configValue = 1; + private java.lang.Object configValue_ = ""; + public boolean hasConfigValue() { + return ((bitField0_ & 0x00000001) == 0x00000001); + } + public String getConfigValue() { + java.lang.Object ref = configValue_; + if (!(ref instanceof String)) { + String s = ((com.google.protobuf.ByteString) ref).toStringUtf8(); + configValue_ = s; + return s; + } else { + return (String) ref; + } + } + public Builder setConfigValue(String value) { + if (value == null) { + throw new NullPointerException(); + } + bitField0_ |= 0x00000001; + configValue_ = value; + onChanged(); + return this; + } + public Builder clearConfigValue() { + bitField0_ = (bitField0_ & ~0x00000001); + configValue_ = getDefaultInstance().getConfigValue(); + onChanged(); + return this; + } + void setConfigValue(com.google.protobuf.ByteString value) { + bitField0_ |= 0x00000001; + configValue_ = value; + onChanged(); + } + + // @@protoc_insertion_point(builder_scope:GetConfigResponse) + } + + static { + defaultInstance = new GetConfigResponse(true); + defaultInstance.initFields(); + } + + // @@protoc_insertion_point(class_scope:GetConfigResponse) + } + public static abstract class MasterAdminService implements com.google.protobuf.Service { protected MasterAdminService() {} @@ -14464,6 +16194,16 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest request, com.google.protobuf.RpcCallback done); + public abstract void updateConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest request, + com.google.protobuf.RpcCallback done); + } public static com.google.protobuf.Service newReflectiveService( @@ -14621,6 +16361,22 @@ impl.isCatalogJanitorEnabled(controller, request, done); } + @java.lang.Override + public void updateConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest request, + com.google.protobuf.RpcCallback done) { + impl.updateConfig(controller, request, done); + } + + @java.lang.Override + public void getConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest request, + com.google.protobuf.RpcCallback done) { + impl.getConfig(controller, request, done); + } + }; } @@ -14681,6 +16437,10 @@ return impl.enableCatalogJanitor(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest)request); case 18: return impl.isCatalogJanitorEnabled(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest)request); + case 19: + return impl.updateConfig(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest)request); + case 20: + return impl.getConfig(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest)request); default: throw new java.lang.AssertionError("Can't get here."); } @@ -14733,6 +16493,10 @@ return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest.getDefaultInstance(); case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest.getDefaultInstance(); + case 19: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.getDefaultInstance(); + case 20: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -14785,6 +16549,10 @@ return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.getDefaultInstance(); case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance(); + case 19: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDefaultInstance(); + case 20: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -14888,6 +16656,16 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest request, com.google.protobuf.RpcCallback done); + public abstract void updateConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest request, + com.google.protobuf.RpcCallback done); + + public abstract void getConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest request, + com.google.protobuf.RpcCallback done); + public static final com.google.protobuf.Descriptors.ServiceDescriptor getDescriptor() { @@ -15005,6 +16783,16 @@ com.google.protobuf.RpcUtil.specializeCallback( done)); return; + case 19: + this.updateConfig(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; + case 20: + this.getConfig(controller, (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest)request, + com.google.protobuf.RpcUtil.specializeCallback( + done)); + return; default: throw new java.lang.AssertionError("Can't get here."); } @@ -15057,6 +16845,10 @@ return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorRequest.getDefaultInstance(); case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest.getDefaultInstance(); + case 19: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.getDefaultInstance(); + case 20: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -15109,6 +16901,10 @@ return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.EnableCatalogJanitorResponse.getDefaultInstance(); case 18: return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance(); + case 19: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDefaultInstance(); + case 20: + return org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDefaultInstance(); default: throw new java.lang.AssertionError("Can't get here."); } @@ -15414,8 +17210,38 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance())); } + + public void updateConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(19), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDefaultInstance())); - } - + } + + public void getConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest request, + com.google.protobuf.RpcCallback done) { + channel.callMethod( + getDescriptor().getMethods().get(20), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDefaultInstance(), + com.google.protobuf.RpcUtil.generalizeCallback( + done, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDefaultInstance())); + } + } + public static BlockingInterface newBlockingStub( com.google.protobuf.BlockingRpcChannel channel) { return new BlockingStub(channel); @@ -15516,6 +17342,16 @@ com.google.protobuf.RpcController controller, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledRequest request) throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse updateConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest request) + throws com.google.protobuf.ServiceException; + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse getConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest request) + throws com.google.protobuf.ServiceException; } private static final class BlockingStub implements BlockingInterface { @@ -15752,9 +17588,33 @@ org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.getDefaultInstance()); } + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse updateConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(19), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.getDefaultInstance()); - } + } + + + public org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse getConfig( + com.google.protobuf.RpcController controller, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest request) + throws com.google.protobuf.ServiceException { + return (org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse) channel.callBlockingMethod( + getDescriptor().getMethods().get(20), + controller, + request, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.getDefaultInstance()); - } - + } + + } + } + private static com.google.protobuf.Descriptors.Descriptor internal_static_AddColumnRequest_descriptor; private static @@ -15945,6 +17805,26 @@ private static com.google.protobuf.GeneratedMessage.FieldAccessorTable internal_static_IsCatalogJanitorEnabledResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_UpdateConfigRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_UpdateConfigRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_UpdateConfigResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_UpdateConfigResponse_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GetConfigRequest_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GetConfigRequest_fieldAccessorTable; + private static com.google.protobuf.Descriptors.Descriptor + internal_static_GetConfigResponse_descriptor; + private static + com.google.protobuf.GeneratedMessage.FieldAccessorTable + internal_static_GetConfigResponse_fieldAccessorTable; public static com.google.protobuf.Descriptors.FileDescriptor getDescriptor() { @@ -15994,39 +17874,47 @@ "\010\"1\n\034EnableCatalogJanitorResponse\022\021\n\tpre" + "vValue\030\001 \001(\010\" \n\036IsCatalogJanitorEnabledR" + "equest\"0\n\037IsCatalogJanitorEnabledRespons", - "e\022\r\n\005value\030\001 \002(\0102\263\t\n\022MasterAdminService\022" + - "2\n\taddColumn\022\021.AddColumnRequest\032\022.AddCol" + - "umnResponse\022;\n\014deleteColumn\022\024.DeleteColu" + - "mnRequest\032\025.DeleteColumnResponse\022;\n\014modi" + - "fyColumn\022\024.ModifyColumnRequest\032\025.ModifyC" + - "olumnResponse\0225\n\nmoveRegion\022\022.MoveRegion" + - "Request\032\023.MoveRegionResponse\022;\n\014assignRe" + - "gion\022\024.AssignRegionRequest\032\025.AssignRegio" + - "nResponse\022A\n\016unassignRegion\022\026.UnassignRe" + - "gionRequest\032\027.UnassignRegionResponse\022>\n\r", - "offlineRegion\022\025.OfflineRegionRequest\032\026.O" + - "fflineRegionResponse\0228\n\013deleteTable\022\023.De" + - "leteTableRequest\032\024.DeleteTableResponse\0228" + - "\n\013enableTable\022\023.EnableTableRequest\032\024.Ena" + - "bleTableResponse\022;\n\014disableTable\022\024.Disab" + - "leTableRequest\032\025.DisableTableResponse\0228\n" + - "\013modifyTable\022\023.ModifyTableRequest\032\024.Modi" + - "fyTableResponse\0228\n\013createTable\022\023.CreateT" + - "ableRequest\032\024.CreateTableResponse\022/\n\010shu" + - "tdown\022\020.ShutdownRequest\032\021.ShutdownRespon", - "se\0225\n\nstopMaster\022\022.StopMasterRequest\032\023.S" + - "topMasterResponse\022,\n\007balance\022\017.BalanceRe" + - "quest\032\020.BalanceResponse\022M\n\022setBalancerRu" + - "nning\022\032.SetBalancerRunningRequest\032\033.SetB" + - "alancerRunningResponse\022;\n\016runCatalogScan" + - "\022\023.CatalogScanRequest\032\024.CatalogScanRespo" + - "nse\022S\n\024enableCatalogJanitor\022\034.EnableCata" + - "logJanitorRequest\032\035.EnableCatalogJanitor" + - "Response\022\\\n\027isCatalogJanitorEnabled\022\037.Is" + - "CatalogJanitorEnabledRequest\032 .IsCatalog", - "JanitorEnabledResponseBG\n*org.apache.had" + - "oop.hbase.protobuf.generatedB\021MasterAdmi" + - "nProtosH\001\210\001\001\240\001\001" + "e\022\r\n\005value\030\001 \002(\010\"=\n\023UpdateConfigRequest\022" + + "\021\n\tconfigKey\030\001 \002(\t\022\023\n\013configValue\030\002 \002(\t\"" + + "4\n\024UpdateConfigResponse\022\034\n\024updateConfigR" + + "esponse\030\001 \001(\010\"%\n\020GetConfigRequest\022\021\n\tcon" + + "figKey\030\001 \002(\t\"(\n\021GetConfigResponse\022\023\n\013con" + + "figValue\030\001 \002(\t2\244\n\n\022MasterAdminService\0222\n" + + "\taddColumn\022\021.AddColumnRequest\032\022.AddColum" + + "nResponse\022;\n\014deleteColumn\022\024.DeleteColumn" + + "Request\032\025.DeleteColumnResponse\022;\n\014modify" + + "Column\022\024.ModifyColumnRequest\032\025.ModifyCol", + "umnResponse\0225\n\nmoveRegion\022\022.MoveRegionRe" + + "quest\032\023.MoveRegionResponse\022;\n\014assignRegi" + + "on\022\024.AssignRegionRequest\032\025.AssignRegionR" + + "esponse\022A\n\016unassignRegion\022\026.UnassignRegi" + + "onRequest\032\027.UnassignRegionResponse\022>\n\rof" + + "flineRegion\022\025.OfflineRegionRequest\032\026.Off" + + "lineRegionResponse\0228\n\013deleteTable\022\023.Dele" + + "teTableRequest\032\024.DeleteTableResponse\0228\n\013" + + "enableTable\022\023.EnableTableRequest\032\024.Enabl" + + "eTableResponse\022;\n\014disableTable\022\024.Disable", + "TableRequest\032\025.DisableTableResponse\0228\n\013m" + + "odifyTable\022\023.ModifyTableRequest\032\024.Modify" + + "TableResponse\0228\n\013createTable\022\023.CreateTab" + + "leRequest\032\024.CreateTableResponse\022/\n\010shutd" + + "own\022\020.ShutdownRequest\032\021.ShutdownResponse" + + "\0225\n\nstopMaster\022\022.StopMasterRequest\032\023.Sto" + + "pMasterResponse\022,\n\007balance\022\017.BalanceRequ" + + "est\032\020.BalanceResponse\022M\n\022setBalancerRunn" + + "ing\022\032.SetBalancerRunningRequest\032\033.SetBal" + + "ancerRunningResponse\022;\n\016runCatalogScan\022\023", + ".CatalogScanRequest\032\024.CatalogScanRespons" + + "e\022S\n\024enableCatalogJanitor\022\034.EnableCatalo" + + "gJanitorRequest\032\035.EnableCatalogJanitorRe" + + "sponse\022\\\n\027isCatalogJanitorEnabled\022\037.IsCa" + + "talogJanitorEnabledRequest\032 .IsCatalogJa" + + "nitorEnabledResponse\022;\n\014updateConfig\022\024.U" + + "pdateConfigRequest\032\025.UpdateConfigRespons" + + "e\0222\n\tgetConfig\022\021.GetConfigRequest\032\022.GetC" + + "onfigResponseBG\n*org.apache.hadoop.hbase" + + ".protobuf.generatedB\021MasterAdminProtosH\001", + "\210\001\001\240\001\001" }; com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner assigner = new com.google.protobuf.Descriptors.FileDescriptor.InternalDescriptorAssigner() { @@ -16337,6 +18225,38 @@ new java.lang.String[] { "Value", }, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.class, org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.IsCatalogJanitorEnabledResponse.Builder.class); + internal_static_UpdateConfigRequest_descriptor = + getDescriptor().getMessageTypes().get(38); + internal_static_UpdateConfigRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UpdateConfigRequest_descriptor, + new java.lang.String[] { "ConfigKey", "ConfigValue", }, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest.Builder.class); + internal_static_UpdateConfigResponse_descriptor = + getDescriptor().getMessageTypes().get(39); + internal_static_UpdateConfigResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_UpdateConfigResponse_descriptor, + new java.lang.String[] { "UpdateConfigResponse", }, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse.Builder.class); + internal_static_GetConfigRequest_descriptor = + getDescriptor().getMessageTypes().get(40); + internal_static_GetConfigRequest_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GetConfigRequest_descriptor, + new java.lang.String[] { "ConfigKey", }, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest.Builder.class); + internal_static_GetConfigResponse_descriptor = + getDescriptor().getMessageTypes().get(41); + internal_static_GetConfigResponse_fieldAccessorTable = new + com.google.protobuf.GeneratedMessage.FieldAccessorTable( + internal_static_GetConfigResponse_descriptor, + new java.lang.String[] { "ConfigValue", }, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.class, + org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse.Builder.class); return null; } }; Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/HMaster.java (revision ) @@ -144,6 +144,10 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest; @@ -178,6 +182,7 @@ import org.apache.hadoop.hbase.zookeeper.LoadBalancerTracker; import org.apache.hadoop.hbase.zookeeper.RegionServerTracker; import org.apache.hadoop.hbase.zookeeper.ZKClusterId; +import org.apache.hadoop.hbase.zookeeper.ClusterConfigTracker; import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; import org.apache.hadoop.metrics.util.MBeanUtil; @@ -190,6 +195,7 @@ import com.google.protobuf.RpcController; import com.google.protobuf.ServiceException; + /** * HMaster is the "master server" for HBase. An HBase cluster has one active * master. If many masters are started, all compete. Whichever wins goes on to @@ -260,7 +266,10 @@ private CatalogTracker catalogTracker; // Cluster status zk tracker and local setter private ClusterStatusTracker clusterStatusTracker; + // Cluster config tracker + private ClusterConfigTracker clusterConfigTracker = null; + private boolean dynamicConfigEnabled = false; - + // buffer for "fatal error" notices from region servers // in the cluster. This is only used for assisting // operations/debugging. @@ -324,7 +333,9 @@ */ public HMaster(final Configuration conf) throws IOException, KeeperException, InterruptedException { - this.conf = new Configuration(conf); + this.zooKeeper = new ZooKeeperWatcher(conf, MASTER, this, true); + this.conf = getClusterConfiguration(conf); + LOG.info("hbase.rootdir=" + FSUtils.getRootDir(this.conf) + ", hbase.cluster.distributed=" + this.conf.getBoolean("hbase.cluster.distributed", false)); // Disable the block cache on the master @@ -386,6 +397,22 @@ this.metrics = new MasterMetrics( new MasterMetricsWrapperImpl(this)); } + private Configuration getClusterConfiguration(Configuration configuration) { + dynamicConfigEnabled = configuration.getBoolean( + "hbase.dynamic.configuration.enabled", false); + if (!dynamicConfigEnabled) { + LOG.info("HBase dynamic cluster configuration is disabled. Configuration" + + "changes require a config file change and cluster restart."); + return configuration; + } + LOG.info("HBase dynamic cluster configuration is enabled. Configuration changes" + + " can be done through the shell and does not require cluster restart"); + // Initialize cluster config tracker + this.clusterConfigTracker = new ClusterConfigTracker(getZooKeeper(), this, configuration); + this.clusterConfigTracker.start(); + return clusterConfigTracker.getClusterConfiguration(); + } + /** * Stall startup if we are designated a backup master; i.e. we want someone * else to become the master before proceeding. @@ -1239,6 +1266,17 @@ return IsCatalogJanitorEnabledResponse.newBuilder().setValue(isEnabled).build(); } + @Override + public UpdateConfigResponse updateConfig(RpcController c, UpdateConfigRequest req) throws ServiceException { + boolean result = this.clusterConfigTracker.updateClusterConfig(req.getConfigKey(), req.getConfigValue()); + return UpdateConfigResponse.newBuilder().setUpdateConfigResponse(result).build(); + } + + @Override + public GetConfigResponse getConfig(RpcController c, GetConfigRequest req) throws ServiceException { + return GetConfigResponse.newBuilder().setConfigValue(conf.get(req.getConfigKey())).build(); + } + /** * @return Maximum time we should run balancer for */ @@ -1914,6 +1952,11 @@ IOException, KeeperException, ExecutionException { this.zooKeeper.reconnectAfterExpiration(); + // Re initialize clusterConfigTracker if required. + if (dynamicConfigEnabled && clusterConfigTracker != null) { + this.clusterConfigTracker = new ClusterConfigTracker(getZooKeeper(), this, conf); + this.clusterConfigTracker.start(); + } Callable callable = new Callable () { public Boolean call() throws InterruptedException, @@ -1994,6 +2037,11 @@ @Override public CatalogTracker getCatalogTracker() { return catalogTracker; + } + + @Override + public ClusterConfigTracker getClusterConfigTracker() { + return this.clusterConfigTracker; } @Override Index: hbase-server/src/main/ruby/hbase/admin.rb IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/ruby/hbase/admin.rb (revision 1403148) +++ hbase-server/src/main/ruby/hbase/admin.rb (revision ) @@ -114,6 +114,20 @@ end #---------------------------------------------------------------------------------------------- + # Update a cluster configuration + # Returns true if update cluster config is successful. + def update_config(configKey, configValue) + @admin.updateConfig(configKey, configValue) + end + + #---------------------------------------------------------------------------------------------- + # Get current cluster configuration for a key + # Returns value of cluster config key. + def get_config(configKey) + @admin.getConfig(configKey) + end + + #---------------------------------------------------------------------------------------------- # Enables a table def enable(table_name) tableExists(table_name) Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/MasterServices.java (revision ) @@ -25,6 +25,7 @@ import org.apache.hadoop.hbase.Server; import org.apache.hadoop.hbase.TableDescriptors; import org.apache.hadoop.hbase.executor.ExecutorService; +import org.apache.hadoop.hbase.zookeeper.ClusterConfigTracker; /** * Services Master supplies @@ -77,4 +78,11 @@ * @return true if master enables ServerShutdownHandler; */ public boolean isServerShutdownHandlerEnabled(); + + /** + * Return the current cluster config tracker + * @return + */ + public ClusterConfigTracker getClusterConfigTracker(); + } Index: hbase-server/src/main/resources/hbase-default.xml IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/resources/hbase-default.xml (revision 1403148) +++ hbase-server/src/main/resources/hbase-default.xml (revision ) @@ -859,6 +859,7 @@ files when hbase.data.umask.enable is true + hbase.metrics.showTableName true @@ -868,6 +869,16 @@ In both cases, the aggregated metric M across tables and cfs will be reported. + + hbase.dynamic.configuration.enabled + false + + Enables the dynamic cluster configuration ability.If enabled, you can make + runtime cluster configuration changes without the need for restarting the cluster. + + + + hbase.metrics.exposeOperationTimes true Index: hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/regionserver/HRegionServer.java (revision ) @@ -219,6 +219,8 @@ import org.apache.hadoop.hbase.zookeeper.ZKUtil; import org.apache.hadoop.hbase.zookeeper.ZooKeeperNodeTracker; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.hadoop.hbase.zookeeper.ClusterConfigTracker; +import org.apache.hadoop.io.Writable; import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.metrics.util.MBeanUtil; import org.apache.hadoop.net.DNS; @@ -395,6 +397,9 @@ // Log Splitting Worker private SplitLogWorker splitLogWorker; + private ClusterConfigTracker clusterConfigTracker; + private boolean dynamicConfigEnabled = false; + // A sleeper that sleeps for msgInterval. private final Sleeper sleeper; @@ -457,7 +462,9 @@ public HRegionServer(Configuration conf) throws IOException, InterruptedException { this.fsOk = true; - this.conf = conf; + this.zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER, this); + this.conf = getClusterConfiguration(conf); + // Set how many times to retry talking to another server over HConnection. HConnectionManager.setServerSideHConnectionRetries(this.conf, LOG); this.isOnline = false; @@ -533,6 +540,22 @@ cacheConfig = new CacheConfig(conf); } + private Configuration getClusterConfiguration(Configuration configuration) { + dynamicConfigEnabled = configuration.getBoolean( + "hbase.dynamic.configuration.enabled", false); + if (!dynamicConfigEnabled) { + LOG.info("HBase runtime dynamic cluster configuration is disabled. Configuration" + + "changes require a config file change and cluster restart."); + return configuration; + } + LOG.info("HBase runtime dynamic cluster configuration is enabled. Configuration changes" + + " can be done through the shell and does not require cluster restart"); + // Initialize cluster config tracker + this.clusterConfigTracker = new ClusterConfigTracker(getZooKeeper(), this, configuration); + this.clusterConfigTracker.start(); + return clusterConfigTracker.getClusterConfiguration(); + } + /** * Run test on configured codecs to make sure supporting libs are in place. * @param c @@ -748,9 +771,6 @@ */ private void initializeZooKeeper() throws IOException, InterruptedException { // Open connection to zookeeper and set primary watcher - this.zooKeeper = new ZooKeeperWatcher(conf, REGIONSERVER + ":" + - this.isa.getPort(), this); - // Create the master address manager, register with zk, and start it. Then // block until a master is available. No point in starting up if no master // running. Index: hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterConfigTracker.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterConfigTracker.java (revision ) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ClusterConfigTracker.java (revision ) @@ -0,0 +1,277 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.hbase.zookeeper; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hbase.Abortable; +import org.apache.hadoop.hbase.HBaseInMemoryConfiguration; +import org.apache.hadoop.hbase.util.Bytes; +import org.apache.zookeeper.KeeperException; +import org.codehaus.jackson.map.ObjectMapper; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.io.IOException; +import java.io.StringWriter; +import java.util.Map; + +/** + * ZK tracker to monitor and dynamically manage cluster configuration changes. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public class ClusterConfigTracker extends ZooKeeperNodeTracker { + public static final Log LOG = LogFactory.getLog(ClusterConfigTracker.class); + // The original cluster configuration that we will use to initially populate the + // ZK cluster configuration node. + private Configuration configuration; + private HBaseInMemoryConfiguration clusterConfiguration = null; + + /** + * Constructs a new ClusterConfig ZK node tracker. + *

+ *

After construction, use {@link #start} to kick off tracking. + * + * @param watcher + * @param abortable + */ + public ClusterConfigTracker(ZooKeeperWatcher watcher, + Abortable abortable, Configuration configuration) { + super(watcher, watcher.clusterConfigZNode, abortable); + this.configuration = configuration; + this.clusterConfiguration = new HBaseInMemoryConfiguration(configuration); + } + + @Override + public void start() { + try { + LOG.info("Starting cluster configuration tracker"); + watcher.registerListener(this); + initClusterConfig(); + } catch (KeeperException e) { + LOG.error("ClusterConfigTracker startup failed.", e); + abortable.abort("ClusterConfigTracker startup failed", e); + } catch (IOException e) { + LOG.error("ClusterConfigTracker startup failed.", e); + abortable.abort("ClusterConfigTracker startup failed", e); + } + } + + private boolean isClusterConfigAvailableInZK() throws KeeperException { + int checkExists = ZKUtil.checkExists(watcher, watcher.clusterConfigZNode); + if (checkExists != -1) { + int configNodes = ZKUtil.getNumberOfChildren(watcher, watcher.clusterConfigZNode); + byte[] data = ZKUtil.getData(watcher, watcher.clusterConfigZNode); + if (data != null && data.length > 0 && configNodes > 0) { + // Cluster configuration already exists in ZK. + LOG.info("ZK Cluster Config already available. Skipping config ZK node creation"); + return true; + } + } + LOG.info("ZK Cluster Config node not available. "); + return false; + } + + private void initClusterConfig() throws KeeperException, IOException { + + if (!isClusterConfigAvailableInZK()) { + createClusterConfig(); + ZKUtil.getChildDataAndWatchForNewChildren(watcher, watcher.clusterConfigZNode); + } else { + clusterConfiguration.refreshClusterConfigMap( + ZKUtil.getChildDataAndWatchForNewChildren(watcher, watcher.clusterConfigZNode)); + } + } + + /** + * Get the current cluster configuration map. + * + * @return Map of current cluster config K,V pairs + */ + public Map getClusterConfigMap() { + return this.clusterConfiguration.getClusterConfigMap(); + } + + /** + * Get current in memory cluster configuration + * + * @return HBaseInMemoryConfiguration + */ + public HBaseInMemoryConfiguration getClusterConfiguration() { + return clusterConfiguration; + } + + public void setClusterConfiguration(HBaseInMemoryConfiguration clusterConfiguration) { + this.clusterConfiguration = clusterConfiguration; + } + + /** + * Update an existing HBase cluster configuration key. Creates a new configuration node, + * if the config does not exist. + * + * @param configKey + * @param configValue + */ + public boolean updateClusterConfig(String configKey, String configValue) { + try { + if (ZKUtil.checkExists(watcher, + getConfigNodePath(configKey)) > 0) { + LOG.debug("Updating cluster configuration. Config Key = " + configKey + + " Value = " + configValue); + ZKUtil.setData(watcher, getConfigNodePath(configKey), Bytes.toBytes(configValue)); + } else { + LOG.debug("Cluster configuration does not exist for Config Key = " + + configKey + " Creating new cluster config node"); + ZKUtil.createSetData(watcher, getConfigNodePath(configKey), + Bytes.toBytes(configValue)); + } + } catch (KeeperException e) { + LOG.error("Failure during update cluster configuration", e); + return false; + } + return true; + } + + private String getConfigNodePath(String configKey) { + return ZKUtil.joinZNode(watcher.clusterConfigZNode, configKey); + } + + private void createClusterConfig() throws IOException, KeeperException { + LOG.info("Creating Cluster Configuration ZK nodes"); + JsonConfiguration jsonConfiguration = readClusterConfiguration(configuration); + JsonProperty[] jsonProperties = jsonConfiguration.getProperties(); + if (jsonProperties.length > 0) { + LOG.info("Number of cluster configuration properties = " + jsonProperties.length); + for (JsonProperty jsonProperty : jsonProperties) { + ZKUtil.createSetData(watcher, getZKConfigNodeName(jsonProperty.getKey()), + Bytes.toBytes(jsonProperty.getValue())); + clusterConfiguration.put(jsonProperty.getKey(), jsonProperty.getValue()); + } + ZKUtil.setData(watcher, watcher.clusterConfigZNode, Bytes.toBytes(jsonProperties.length)); + } else { + // This should never happen + LOG.error("Current cluster config property set is empty."); + } + } + + + private String getZKConfigNodeName(String key) { + return ZKUtil.joinZNode(watcher.clusterConfigZNode, key); + } + + private JsonConfiguration readClusterConfiguration(Configuration configuration) throws IOException { + StringWriter outWriter = new StringWriter(); + Configuration.dumpConfiguration(configuration, outWriter); + String jsonStr = outWriter.toString(); + ObjectMapper mapper = new ObjectMapper(); + return mapper.readValue(jsonStr, JsonConfiguration.class); + } + + @Override + public void nodeCreated(String path) { + if (path.startsWith(watcher.clusterConfigZNode)) { + processConfigChange(); + } + } + + @Override + public void nodeDeleted(String path) { + if (path.startsWith(watcher.clusterConfigZNode)) { + processConfigChange(); + } + } + + private void processConfigChange() { + try { + clusterConfiguration.refreshClusterConfigMap( + ZKUtil.getChildDataAndWatchForNewChildren(watcher, watcher.clusterConfigZNode)); + } catch (KeeperException ke) { + LOG.error("Cluster config refresh error ", ke); + } + } + + @Override + public void nodeDataChanged(String path) { + if (path.startsWith(watcher.clusterConfigZNode)) { + processConfigChange(); + } + } + + @Override + public void nodeChildrenChanged(String path) { + if (path.startsWith(watcher.clusterConfigZNode)) { + processConfigChange(); + } + } + + static class JsonConfiguration { + JsonProperty[] properties; + + public JsonProperty[] getProperties() { + return properties; + } + + public void setProperties(JsonProperty[] properties) { + this.properties = properties; + } + } + + static class JsonProperty { + String key; + String value; + boolean isFinal; + String resource; + + public String getKey() { + return key; + } + + public void setKey(String key) { + this.key = key; + } + + public String getValue() { + return value; + } + + public void setValue(String value) { + this.value = value; + } + + public boolean getIsFinal() { + return isFinal; + } + + public void setIsFinal(boolean isFinal) { + this.isFinal = isFinal; + } + + public String getResource() { + return resource; + } + + public void setResource(String resource) { + this.resource = resource; + } + } + +} + Index: hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/master/ServerManager.java (revision ) @@ -739,7 +739,6 @@ lastCountChange = now; } } - LOG.info("Finished waiting for region servers count to settle;" + " checked in " + count + ", slept for " + slept + " ms," + " expecting minimum of " + minToStart + ", maximum of "+ maxToStart+","+ Index: hbase-server/src/main/java/org/apache/hadoop/hbase/MasterAdminProtocol.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/MasterAdminProtocol.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/MasterAdminProtocol.java (revision ) @@ -20,6 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.MasterAdminService; import org.apache.hadoop.hbase.security.TokenInfo; import org.apache.hadoop.hbase.security.KerberosInfo; @@ -61,6 +62,10 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterProtos.IsMasterRunningResponse; @@ -308,7 +313,27 @@ */ @Override public IsMasterRunningResponse isMasterRunning(RpcController c, IsMasterRunningRequest req) + throws ServiceException; + + /** + * Update cluster configuration. + * @param c Controller Unused (set to null). + * @param req UpdateConfigRequest + * @return + * @throws ServiceException + */ + public UpdateConfigResponse updateConfig(RpcController c, UpdateConfigRequest req) + throws ServiceException; + + /** + * Get current cluster configuration for a given config key. + * @param c Controller unused (set to null) + * @param req GetConfigRequest + * @return + * @throws ServiceException + */ + public GetConfigResponse getConfig(RpcController c, GetConfigRequest req) - throws ServiceException; + throws ServiceException; /** * Run a scan of the catalog table Index: hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/zookeeper/ZooKeeperWatcher.java (revision ) @@ -103,6 +103,8 @@ public String splitLogZNode; // znode containing the state of the load balancer public String balancerZNode; + // znode to monitor online cluster configuration changes + public String clusterConfigZNode; // Certain ZooKeeper nodes need to be world-readable public static final ArrayList CREATOR_ALL_AND_WORLD_READABLE = @@ -166,6 +168,7 @@ ZKUtil.createAndFailSilent(this, tableZNode); ZKUtil.createAndFailSilent(this, splitLogZNode); ZKUtil.createAndFailSilent(this, backupMasterAddressesZNode); + ZKUtil.createAndFailSilent(this, clusterConfigZNode); } catch (KeeperException e) { throw new ZooKeeperConnectionException( prefix("Unexpected KeeperException creating base node"), e); @@ -215,6 +218,9 @@ conf.get("zookeeper.znode.splitlog", HConstants.SPLIT_LOGDIR_NAME)); balancerZNode = ZKUtil.joinZNode(baseZNode, conf.get("zookeeper.znode.balancer", "balancer")); + clusterConfigZNode = ZKUtil.joinZNode(baseZNode, + conf.get("zookeeper.znode.cluster.config", "clusterConfig")); + } /** Index: hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (revision 1403148) +++ hbase-server/src/test/java/org/apache/hadoop/hbase/master/TestCatalogJanitor.java (revision ) @@ -67,6 +67,8 @@ import org.apache.hadoop.hbase.util.HFileArchiveUtil; import org.apache.hadoop.hbase.util.Pair; import org.apache.hadoop.hbase.zookeeper.ZooKeeperWatcher; +import org.apache.hadoop.hbase.zookeeper.ClusterConfigTracker; +import org.junit.Ignore; import org.junit.Test; import org.junit.experimental.categories.Category; import org.mockito.Mockito; @@ -288,6 +290,12 @@ public boolean isServerShutdownHandlerEnabled() { return true; } + + @Override + public ClusterConfigTracker getClusterConfigTracker() { + return null; + } + } @Test Index: hbase-server/src/main/protobuf/MasterAdmin.proto IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/protobuf/MasterAdmin.proto (revision 1403148) +++ hbase-server/src/main/protobuf/MasterAdmin.proto (revision ) @@ -176,6 +176,23 @@ required bool value = 1; } +message UpdateConfigRequest { + required string configKey = 1; + required string configValue = 2; +} + +message UpdateConfigResponse { + optional bool updateConfigResponse = 1; +} + +message GetConfigRequest { + required string configKey = 1; +} + +message GetConfigResponse { + required string configValue = 1; +} + service MasterAdminService { /** Adds a column to the specified table. */ rpc addColumn(AddColumnRequest) @@ -273,4 +290,10 @@ */ rpc isCatalogJanitorEnabled(IsCatalogJanitorEnabledRequest) returns(IsCatalogJanitorEnabledResponse); + + rpc updateConfig(UpdateConfigRequest) + returns(UpdateConfigResponse); + + rpc getConfig(GetConfigRequest) + returns(GetConfigResponse); } Index: hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (revision 1403148) +++ hbase-server/src/main/java/org/apache/hadoop/hbase/client/HBaseAdmin.java (revision ) @@ -87,6 +87,10 @@ import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.ShutdownRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.StopMasterRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UnassignRegionRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.GetConfigResponse; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigRequest; +import org.apache.hadoop.hbase.protobuf.generated.MasterAdminProtos.UpdateConfigResponse; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetClusterStatusRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusRequest; import org.apache.hadoop.hbase.protobuf.generated.MasterMonitorProtos.GetSchemaAlterStatusResponse; @@ -2082,4 +2086,29 @@ throw new IOException("Unexpected exception when calling master", e); } } + + + public boolean updateConfig(final String configKey, final String configValue) throws IOException { + return execute(new MasterAdminCallable() { + @Override + public Boolean call() throws ServiceException { + UpdateConfigRequest request = RequestConverter.buildUpdateConfigRequest(configKey, configValue); + UpdateConfigResponse response = masterAdmin.updateConfig(null, request); + return response.getUpdateConfigResponse(); + } + }); + + } + + public String getConfig(final String configKey) throws IOException { + return execute(new MasterAdminCallable() { + @Override + public String call() throws ServiceException { + GetConfigRequest request = RequestConverter.buildGetConfigRequest(configKey); + GetConfigResponse response = masterAdmin.getConfig(null, request); + return response.getConfigValue(); + } + }); + } + } Index: hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java IDEA additional info: Subsystem: com.intellij.openapi.diff.impl.patch.CharsetEP <+>UTF-8 =================================================================== --- hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (revision 1403148) +++ hbase-common/src/main/java/org/apache/hadoop/hbase/HConstants.java (revision ) @@ -601,7 +601,7 @@ "hbase.regionserver.hostname.seen.by.master"; public static final String HBASE_MASTER_LOGCLEANER_PLUGINS = - "hbase.master.logcleaner.plugins"; + "hbase.master.logcleaner.plugins"; public static final String HBASE_REGION_SPLIT_POLICY_KEY = "hbase.regionserver.region.split.policy";