Add options to the properties converter to update from 0.6.x to 0.7.x

* Removed 0.5 --> 0.6 converter
This commit is contained in:
Charles Allen 2015-01-12 17:05:02 -08:00
parent 95e0429eb1
commit e055a7e869
2 changed files with 156 additions and 83 deletions

View File

@ -17,13 +17,20 @@
package io.druid.cli.convert;
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.core.type.TypeReference;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.api.client.util.Sets;
import com.google.common.base.Charsets;
import com.google.common.base.Throwables;
import com.google.common.collect.Lists;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.metamx.common.guava.CloseQuietly;
import com.metamx.common.logger.Logger;
import io.airlift.command.Command;
import io.airlift.command.Option;
import io.druid.jackson.DefaultObjectMapper;
import java.io.BufferedWriter;
import java.io.File;
@ -33,94 +40,104 @@ import java.io.IOException;
import java.io.InputStreamReader;
import java.io.OutputStreamWriter;
import java.io.Reader;
import java.net.URI;
import java.util.Arrays;
import java.util.List;
import java.util.Map;
import java.util.Properties;
import java.util.Set;
import java.util.TreeMap;
/**
*/
@Command(
name = "convertProps",
description = "Converts runtime.properties files from version 0.5 to 0.6"
description = "Converts runtime.properties files from version to version"
)
public class ConvertProperties implements Runnable
{
private static final Logger log = new Logger(ConvertProperties.class);
private static final List<PropertyConverter> converters = Lists.newArrayList(
new DatabasePropertiesConverter(),
new Rename("druid.database.rules.defaultDatasource", "druid.manager.rules.defaultTier"),
new Rename("druid.zk.paths.discoveryPath", "druid.discovery.curator.path"),
new Rename("druid.http.numThreads", "druid.server.http.numThreads"),
new Rename("druid.http.maxIdleTimeMillis", "druid.server.http.maxIdleTime"),
new Rename("druid.database.connectURI", "druid.db.connector.connectURI"),
new Rename("druid.database.user", "druid.db.connector.user"),
new Rename("druid.database.password", "druid.db.connector.password"),
new Rename("druid.database.poll.duration", "druid.manager.segment.pollDuration"),
new Rename("druid.database.password", "druid.db.connector.password"),
new Rename("druid.database.validation", "druid.db.connector.useValidationQuery"),
new Rename("com.metamx.emitter", "druid.emitter"),
new Rename("com.metamx.emitter.logging", "druid.emitter.logging"),
new Rename("com.metamx.emitter.logging.level", "druid.emitter.logging.logLevel"),
new Rename("com.metamx.emitter.http", "druid.emitter.http"),
new Rename("com.metamx.emitter.http.url", "druid.emitter.http.recipientBaseUrl"),
new Rename("com.metamx.emitter.period", "druid.monitoring.emissionPeriod"),
new Rename("com.metamx.druid.emitter.period", "druid.monitoring.emissionPeriod"),
new Rename("com.metamx.metrics.emitter.period", "druid.monitoring.emissionPeriod"),
new PrefixRename("com.metamx.emitter", "druid.emitter"),
new PrefixRename("com.metamx.druid.emitter", "druid.emitter"),
new IndexCacheConverter(),
new Rename("druid.paths.segmentInfoCache", "druid.segmentCache.infoDir"),
new Rename("com.metamx.aws.accessKey", "druid.s3.accessKey"),
new Rename("com.metamx.aws.secretKey", "druid.s3.secretKey"),
new Rename("druid.bard.maxIntervalDuration", "druid.query.chunkDuration"),
new PrefixRename("druid.bard.cache", "druid.broker.cache"),
new Rename("druid.client.http.connections", "druid.broker.http.numConnections"),
new Rename("com.metamx.query.groupBy.maxResults", "druid.query.groupBy.maxResults"),
new Rename("com.metamx.query.search.maxSearchLimit", "druid.query.search.maxSearchLimit"),
new Rename("druid.indexer.runner", "druid.indexer.runner.type"),
new Rename("druid.indexer.storage", "druid.indexer.storage.type"),
new Rename("druid.indexer.threads", "druid.indexer.runner.forks"),
new Rename("druid.indexer.taskDir", "druid.indexer.runner.taskDir"),
new Rename("druid.indexer.fork.java", "druid.indexer.runner.javaCommand"),
new Rename("druid.indexer.fork.opts", "druid.indexer.runner.javaOpts"),
new Rename("druid.indexer.fork.classpath", "druid.indexer.runner.classpath"),
new Rename("druid.indexer.fork.hostpattern", "druid.indexer.runner.hostPattern"),
new Rename("druid.indexer.fork.startport", "druid.indexer.runner.startPort"),
new Rename("druid.indexer.properties.prefixes", "druid.indexer.runner.allowedPrefixes"),
new Rename("druid.indexer.taskAssignmentTimeoutDuration", "druid.indexer.runner.taskAssignmentTimeout"),
new Rename("druid.indexer.worker.version", "druid.indexer.runner.minWorkerVersion"),
new Rename("druid.zk.maxNumBytes", "druid.indexer.runner.maxZnodeBytes"),
new Rename("druid.indexer.provisionResources.duration", "druid.indexer.autoscale.provisionPeriod"),
new Rename("druid.indexer.terminateResources.duration", "druid.indexer.autoscale.terminatePeriod"),
new Rename("druid.indexer.terminateResources.originDateTime", "druid.indexer.autoscale.originTime"),
new Rename("druid.indexer.autoscaling.strategy", "druid.indexer.autoscale.strategy"),
new Rename("druid.indexer.logs.s3bucket", "druid.indexer.logs.s3Bucket"),
new Rename("druid.indexer.logs.s3prefix", "druid.indexer.logs.s3Prefix"),
new Rename("druid.indexer.maxWorkerIdleTimeMillisBeforeDeletion", "druid.indexer.autoscale.workerIdleTimeout"),
new Rename("druid.indexer.maxScalingDuration", "druid.indexer.autoscale.scalingTimeout"),
new Rename("druid.indexer.numEventsToTrack", "druid.indexer.autoscale.numEventsToTrack"),
new Rename("druid.indexer.maxPendingTaskDuration", "druid.indexer.autoscale.pendingTaskTimeout"),
new Rename("druid.indexer.worker.version", "druid.indexer.autoscale.workerVersion"),
new Rename("druid.indexer.worker.port", "druid.indexer.autoscale.workerPort"),
new Rename("druid.worker.masterService", "druid.selectors.indexing.serviceName"),
new ChatHandlerConverter(),
new Rename("druid.indexer.baseDir", "druid.indexer.task.baseDir"),
new Rename("druid.indexer.taskDir", "druid.indexer.task.taskDir"),
new Rename("druid.indexer.hadoopWorkingPath", "druid.indexer.task.hadoopWorkingPath"),
new Rename("druid.indexer.rowFlushBoundary", "druid.indexer.task.rowFlushBoundary"),
new Rename("druid.worker.taskActionClient.retry.minWaitMillis", "druid.worker.taskActionClient.retry.minWait"),
new Rename("druid.worker.taskActionClient.retry.maxWaitMillis", "druid.worker.taskActionClient.retry.maxWait"),
new Rename("druid.master.merger.service", "druid.selectors.indexing.serviceName"),
new Rename("druid.master.period.segmentMerger", "druid.coordinator.period.indexingPeriod"),
new Rename("druid.master.merger.on", "druid.coordinator.merge.on"),
new Rename("druid.master.period", "druid.coordinator.period"),
new PrefixRename("druid.master", "druid.coordinator"),
new PrefixRename("druid.pusher", "druid.storage"),
new DataSegmentPusherDefaultConverter(),
new Rename("druid.pusher.hdfs.storageDirectory", "druid.storage.storageDirectory"),
new Rename("druid.pusher.cassandra.host", "druid.storage.host"),
new Rename("druid.pusher.cassandra.keySpace", "druid.storage.keySpace")
private static String parseJdbcUrl(String jdbcUrl){
final String subString = jdbcUrl.substring("jdbc:".length());
final URI uri = URI.create(subString);
return uri.getScheme();
}
private static final List<PropertyConverter> converters6to7 = ImmutableList.<PropertyConverter>of(
new Rename("druid.db.connector.connectURI", "druid.metadata.storage.connector.connectURI"),
new Rename("druid.db.connector.user", "druid.metadata.storage.connector.user"),
new Rename("druid.db.connector.password", "druid.metadata.storage.connector.password"),
new Rename("druid.db.tables.base", "druid.metadata.storage.tables.base"),
new PropertyConverter()
{
// Add a new config for metadata storage type, and update property name
private static final String PROPERTY = "druid.db.connector.connectURI";
@Override
public boolean canHandle(String property)
{
return PROPERTY.equals(property);
}
@Override
public Map<String, String> convert(Properties properties)
{
if (properties.containsKey(PROPERTY)) {
String jdbcConnectorType = parseJdbcUrl(properties.getProperty(PROPERTY));
return ImmutableMap.of(
"druid.metadata.storage.connector.connectURI", properties.getProperty(PROPERTY),
"druid.metadata.storage.type", jdbcConnectorType
);
} else {
return ImmutableMap.of();
}
}
},
new PropertyConverter()
{
// Add a new coordinate for the metadata storage
private static final String PROPERTY = "druid.extensions.coordinates";
private final ObjectMapper defaultObjectMapper = new DefaultObjectMapper();
@Override
public boolean canHandle(String property)
{
return PROPERTY.equals(property);
}
private static final String uriPropertyKey = "druid.db.connector.connectURI";
@Override
public Map<String, String> convert(Properties properties)
{
final String jdbcUrl = properties.getProperty(uriPropertyKey);
if(null == jdbcUrl){
log.warn("No entry for [%s] found in properties! cannot add ????-metadata-storage to [%s]", uriPropertyKey, PROPERTY);
return ImmutableMap.of();
}
final String value = properties.getProperty(PROPERTY);
final Set<String> coordinates = Sets.newHashSet();
final List<String> oldCoordinates;
try {
oldCoordinates = defaultObjectMapper.readValue(
value, new TypeReference<List<String>>(){}
);
}
catch (IOException e) {
throw com.google.api.client.repackaged.com.google.common.base.Throwables.propagate(e);
}
coordinates.addAll(oldCoordinates);
coordinates.add(String.format("io.druid.extensions:%s-metadata-storage", parseJdbcUrl(jdbcUrl)));
try {
return ImmutableMap.of(PROPERTY, defaultObjectMapper.writeValueAsString(ImmutableList.copyOf(coordinates)));
}
catch (JsonProcessingException e) {
throw Throwables.propagate(e);
}
}
},
new ValueConverter("druid.indexer.storage.type", ImmutableMap.of("db", "metadata")),
new ValueConverter("druid.publish.type", ImmutableMap.of("db", "metadata"))
);
@Option(name = "-f", title = "file", description = "The properties file to convert", required = true)
@ -156,12 +173,12 @@ public class ConvertProperties implements Runnable
int count = 0;
for (String property : fromFile.stringPropertyNames()) {
boolean handled = false;
for (PropertyConverter converter : converters) {
for (PropertyConverter converter : converters6to7) {
if (converter.canHandle(property)) {
for (Map.Entry<String, String> entry : converter.convert(fromFile).entrySet()) {
if (entry.getValue() != null) {
++count;
log.info("Converting [%s] to [%s]", property, entry.getKey());
log.info("Converting [%s] to [%s]:[%s]", property, entry.getKey(), entry.getValue());
updatedProps.setProperty(entry.getKey(), entry.getValue());
}
}
@ -175,14 +192,12 @@ public class ConvertProperties implements Runnable
}
}
updatedProps.setProperty(
"druid.monitoring.monitors", "[\"com.metamx.metrics.SysMonitor\"]"
);
BufferedWriter out = null;
try {
TreeMap<Object, Object> orderedUpdates = new TreeMap<>();
orderedUpdates.putAll(updatedProps);
out = new BufferedWriter(new OutputStreamWriter(new FileOutputStream(outFile), Charsets.UTF_8));
for (Map.Entry<Object, Object> prop : updatedProps.entrySet()) {
for (Map.Entry<Object, Object> prop : orderedUpdates.entrySet()) {
out.write((String) prop.getKey());
out.write("=");
out.write((String) prop.getValue());

View File

@ -0,0 +1,58 @@
/*
* Druid - a distributed column store.
* Copyright (C) 2012, 2013, 2014 Metamarkets Group Inc.
*
* This program is free software; you can redistribute it and/or
* modify it under the terms of the GNU General Public License
* as published by the Free Software Foundation; either version 2
* of the License, or (at your option) any later version.
*
* This program is distributed in the hope that it will be useful,
* but WITHOUT ANY WARRANTY; without even the implied warranty of
* MERCHANTABILITY or FITNESS FOR A PARTICULAR PURPOSE. See the
* GNU General Public License for more details.
*
* You should have received a copy of the GNU General Public License
* along with this program; if not, write to the Free Software
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
*/
package io.druid.cli.convert;
import com.google.common.collect.ImmutableMap;
import java.util.Map;
import java.util.Properties;
/**
*
*/
public class ValueConverter implements PropertyConverter
{
private final Map<String, String> valueMap;
private final String property;
public ValueConverter(String property, Map<String, String> valueMap){
this.property = property;
this.valueMap = valueMap;
}
@Override
public boolean canHandle(String property)
{
return this.property.equals(property);
}
@Override
public Map<String, String> convert(Properties properties)
{
final String oldValue = properties.getProperty(this.property);
if(null == oldValue){
return ImmutableMap.of();
}
final String newValue = valueMap.get(oldValue);
if(null == newValue){
return ImmutableMap.of();
}
return ImmutableMap.of(this.property, newValue);
}
}