mirror of https://github.com/apache/druid.git
Merge branch 'master' into docs
Conflicts: docs/content/Loading-Your-Data.md
This commit is contained in:
commit
154b58defd
|
@ -57,7 +57,7 @@ public class ConfigProvider<T> implements Provider<T>
|
||||||
private final Class<T> clazz;
|
private final Class<T> clazz;
|
||||||
private final Map<String, String> replacements;
|
private final Map<String, String> replacements;
|
||||||
|
|
||||||
private T object = null;
|
private ConfigurationObjectFactory factory = null;
|
||||||
|
|
||||||
public ConfigProvider(
|
public ConfigProvider(
|
||||||
Class<T> clazz,
|
Class<T> clazz,
|
||||||
|
@ -70,20 +70,21 @@ public class ConfigProvider<T> implements Provider<T>
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public void inject(ConfigurationObjectFactory factory)
|
public void inject(ConfigurationObjectFactory factory)
|
||||||
|
{
|
||||||
|
this.factory = factory;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public T get()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
// ConfigMagic handles a null replacements
|
// ConfigMagic handles a null replacements
|
||||||
object = factory.buildWithReplacements(clazz, replacements);
|
Preconditions.checkNotNull(factory, "WTF!? Code misconfigured, inject() didn't get called.");
|
||||||
|
return factory.buildWithReplacements(clazz, replacements);
|
||||||
}
|
}
|
||||||
catch (IllegalArgumentException e) {
|
catch (IllegalArgumentException e) {
|
||||||
log.info("Unable to build instance of class[%s]", clazz);
|
log.info("Unable to build instance of class[%s]", clazz);
|
||||||
throw e;
|
throw e;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public T get()
|
|
||||||
{
|
|
||||||
return Preconditions.checkNotNull(object, "WTF!? Code misconfigured, inject() didn't get called.");
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -1,130 +0,0 @@
|
||||||
/*
|
|
||||||
* Druid - a distributed column store.
|
|
||||||
* Copyright (C) 2012, 2013 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.indexer;
|
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
|
||||||
import com.metamx.common.Pair;
|
|
||||||
import com.metamx.common.lifecycle.Lifecycle;
|
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
|
||||||
*/
|
|
||||||
@Deprecated
|
|
||||||
public class HadoopDruidIndexer
|
|
||||||
{
|
|
||||||
public static void main(String[] args) throws Exception
|
|
||||||
{
|
|
||||||
if (args.length < 1 || args.length > 2) {
|
|
||||||
printHelp();
|
|
||||||
System.exit(2);
|
|
||||||
}
|
|
||||||
|
|
||||||
HadoopDruidIndexerNode node = HadoopDruidIndexerNode.builder().build();
|
|
||||||
|
|
||||||
if (args.length == 2) {
|
|
||||||
node.setIntervalSpec(args[0]);
|
|
||||||
}
|
|
||||||
node.setArgumentSpec(args[args.length == 1 ? 0 : 1]);
|
|
||||||
|
|
||||||
Lifecycle lifecycle = new Lifecycle();
|
|
||||||
lifecycle.addManagedInstance(node);
|
|
||||||
|
|
||||||
try {
|
|
||||||
lifecycle.start();
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
e.printStackTrace();
|
|
||||||
Thread.sleep(500);
|
|
||||||
printHelp();
|
|
||||||
System.exit(1);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static final List<Pair<String, String>> expectedFields =
|
|
||||||
ImmutableList.<Pair<String, String>>builder()
|
|
||||||
.add(Pair.of("dataSource", "Name of dataSource"))
|
|
||||||
.add(Pair.of("timestampColumn", "Column name of the timestamp column"))
|
|
||||||
.add(Pair.of("timestampFormat", "Format name of the timestamp column (posix or iso)"))
|
|
||||||
.add(
|
|
||||||
Pair.of(
|
|
||||||
"dataSpec",
|
|
||||||
"A JSON object with fields "
|
|
||||||
+
|
|
||||||
"format=(json, csv, tsv), "
|
|
||||||
+
|
|
||||||
"columns=JSON array of column names for the delimited text input file (only for csv and tsv formats),"
|
|
||||||
+
|
|
||||||
"dimensions=JSON array of dimensionn names (must match names in columns),"
|
|
||||||
+
|
|
||||||
"delimiter=delimiter of the data (only for tsv format)"
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.add(
|
|
||||||
Pair.of(
|
|
||||||
"granularitySpec",
|
|
||||||
"A JSON object indicating the Granularity that segments should be created at."
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.add(
|
|
||||||
Pair.of(
|
|
||||||
"pathSpec",
|
|
||||||
"A JSON object with fields type=granularity, inputPath, filePattern, dataGranularity"
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.add(
|
|
||||||
Pair.of(
|
|
||||||
"rollupSpec",
|
|
||||||
"JSON object with fields rollupGranularity, aggs=JSON Array of Aggregator specs"
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.add(Pair.of("workingPath", "Path to store intermediate output data. Deleted when finished."))
|
|
||||||
.add(Pair.of("segmentOutputPath", "Path to store output segments."))
|
|
||||||
.add(
|
|
||||||
Pair.of(
|
|
||||||
"updaterJobSpec",
|
|
||||||
"JSON object with fields type=db, connectURI of the database, username, password, and segment table name"
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.add(Pair.of("cleanupOnFailure", "Clean up intermediate files on failure? (default: true)"))
|
|
||||||
.add(Pair.of("leaveIntermediate", "Leave intermediate files. (default: false)"))
|
|
||||||
.add(Pair.of("partitionDimension", "Dimension to partition by (optional)"))
|
|
||||||
.add(
|
|
||||||
Pair.of(
|
|
||||||
"targetPartitionSize",
|
|
||||||
"Integer representing the target number of rows in a partition (required if partitionDimension != null)"
|
|
||||||
)
|
|
||||||
)
|
|
||||||
.build();
|
|
||||||
|
|
||||||
private static void printHelp()
|
|
||||||
{
|
|
||||||
System.out.println("Usage: <java invocation> <config_spec>");
|
|
||||||
System.out.println("<config_spec> is either a JSON object or the path to a file that contains a JSON object.");
|
|
||||||
System.out.println();
|
|
||||||
System.out.println("JSON object description:");
|
|
||||||
System.out.println("{");
|
|
||||||
for (Pair<String, String> expectedField : expectedFields) {
|
|
||||||
System.out.printf(" \"%s\": %s%n", expectedField.lhs, expectedField.rhs);
|
|
||||||
}
|
|
||||||
System.out.println("}");
|
|
||||||
}
|
|
||||||
|
|
||||||
}
|
|
|
@ -97,27 +97,6 @@ public class HadoopDruidIndexerConfig
|
||||||
|
|
||||||
public static HadoopDruidIndexerConfig fromMap(Map<String, Object> argSpec)
|
public static HadoopDruidIndexerConfig fromMap(Map<String, Object> argSpec)
|
||||||
{
|
{
|
||||||
List<Registererer> registererers = Lists.transform(
|
|
||||||
MapUtils.getList(argSpec, "registererers", ImmutableList.of()),
|
|
||||||
new Function<Object, Registererer>()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public Registererer apply(@Nullable Object input)
|
|
||||||
{
|
|
||||||
try {
|
|
||||||
return (Registererer) Class.forName((String) input).newInstance();
|
|
||||||
}
|
|
||||||
catch (Exception e) {
|
|
||||||
throw Throwables.propagate(e);
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
|
||||||
);
|
|
||||||
|
|
||||||
if (!registererers.isEmpty()) {
|
|
||||||
Registererers.registerHandlers(registererers, Arrays.asList(jsonMapper));
|
|
||||||
}
|
|
||||||
|
|
||||||
return jsonMapper.convertValue(argSpec, HadoopDruidIndexerConfig.class);
|
return jsonMapper.convertValue(argSpec, HadoopDruidIndexerConfig.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -179,7 +158,6 @@ public class HadoopDruidIndexerConfig
|
||||||
private volatile DataRollupSpec rollupSpec;
|
private volatile DataRollupSpec rollupSpec;
|
||||||
private volatile DbUpdaterJobSpec updaterJobSpec;
|
private volatile DbUpdaterJobSpec updaterJobSpec;
|
||||||
private volatile boolean ignoreInvalidRows = false;
|
private volatile boolean ignoreInvalidRows = false;
|
||||||
private volatile List<String> registererers = Lists.newArrayList();
|
|
||||||
|
|
||||||
@JsonCreator
|
@JsonCreator
|
||||||
public HadoopDruidIndexerConfig(
|
public HadoopDruidIndexerConfig(
|
||||||
|
@ -203,8 +181,7 @@ public class HadoopDruidIndexerConfig
|
||||||
final @JsonProperty("overwriteFiles") boolean overwriteFiles,
|
final @JsonProperty("overwriteFiles") boolean overwriteFiles,
|
||||||
final @JsonProperty("rollupSpec") DataRollupSpec rollupSpec,
|
final @JsonProperty("rollupSpec") DataRollupSpec rollupSpec,
|
||||||
final @JsonProperty("updaterJobSpec") DbUpdaterJobSpec updaterJobSpec,
|
final @JsonProperty("updaterJobSpec") DbUpdaterJobSpec updaterJobSpec,
|
||||||
final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows,
|
final @JsonProperty("ignoreInvalidRows") boolean ignoreInvalidRows
|
||||||
final @JsonProperty("registererers") List<String> registererers
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.dataSource = dataSource;
|
this.dataSource = dataSource;
|
||||||
|
@ -224,7 +201,6 @@ public class HadoopDruidIndexerConfig
|
||||||
this.rollupSpec = rollupSpec;
|
this.rollupSpec = rollupSpec;
|
||||||
this.updaterJobSpec = updaterJobSpec;
|
this.updaterJobSpec = updaterJobSpec;
|
||||||
this.ignoreInvalidRows = ignoreInvalidRows;
|
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||||
this.registererers = registererers;
|
|
||||||
|
|
||||||
if(partitionsSpec != null) {
|
if(partitionsSpec != null) {
|
||||||
Preconditions.checkArgument(
|
Preconditions.checkArgument(
|
||||||
|
@ -517,17 +493,6 @@ public class HadoopDruidIndexerConfig
|
||||||
this.ignoreInvalidRows = ignoreInvalidRows;
|
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
|
||||||
public List<String> getRegistererers()
|
|
||||||
{
|
|
||||||
return registererers;
|
|
||||||
}
|
|
||||||
|
|
||||||
public void setRegistererers(List<String> registererers)
|
|
||||||
{
|
|
||||||
this.registererers = registererers;
|
|
||||||
}
|
|
||||||
|
|
||||||
/********************************************
|
/********************************************
|
||||||
Granularity/Bucket Helper Methods
|
Granularity/Bucket Helper Methods
|
||||||
********************************************/
|
********************************************/
|
||||||
|
|
|
@ -22,6 +22,7 @@ package io.druid.indexer;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.common.collect.Maps;
|
import com.google.common.collect.Maps;
|
||||||
|
import com.google.inject.Inject;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
|
@ -48,6 +49,7 @@ public class HadoopDruidIndexerJob implements Jobby
|
||||||
private IndexGeneratorJob indexJob;
|
private IndexGeneratorJob indexJob;
|
||||||
private volatile List<DataSegment> publishedSegments = null;
|
private volatile List<DataSegment> publishedSegments = null;
|
||||||
|
|
||||||
|
@Inject
|
||||||
public HadoopDruidIndexerJob(
|
public HadoopDruidIndexerJob(
|
||||||
HadoopDruidIndexerConfig config
|
HadoopDruidIndexerConfig config
|
||||||
)
|
)
|
||||||
|
|
|
@ -24,6 +24,7 @@ import com.google.inject.Inject;
|
||||||
import com.metamx.emitter.service.ServiceEmitter;
|
import com.metamx.emitter.service.ServiceEmitter;
|
||||||
import com.metamx.metrics.MonitorScheduler;
|
import com.metamx.metrics.MonitorScheduler;
|
||||||
import io.druid.client.ServerView;
|
import io.druid.client.ServerView;
|
||||||
|
import io.druid.guice.annotations.Json;
|
||||||
import io.druid.guice.annotations.Processing;
|
import io.druid.guice.annotations.Processing;
|
||||||
import io.druid.indexing.common.actions.TaskActionClientFactory;
|
import io.druid.indexing.common.actions.TaskActionClientFactory;
|
||||||
import io.druid.indexing.common.config.TaskConfig;
|
import io.druid.indexing.common.config.TaskConfig;
|
||||||
|
|
|
@ -27,8 +27,8 @@ import com.metamx.common.ISE;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.metamx.http.client.HttpClient;
|
import com.metamx.http.client.HttpClient;
|
||||||
import com.metamx.http.client.response.ToStringResponseHandler;
|
import com.metamx.http.client.response.ToStringResponseHandler;
|
||||||
import io.druid.client.indexing.IndexingServiceSelector;
|
|
||||||
import io.druid.client.selector.Server;
|
import io.druid.client.selector.Server;
|
||||||
|
import io.druid.curator.discovery.ServerDiscoverySelector;
|
||||||
import io.druid.indexing.common.RetryPolicy;
|
import io.druid.indexing.common.RetryPolicy;
|
||||||
import io.druid.indexing.common.RetryPolicyFactory;
|
import io.druid.indexing.common.RetryPolicyFactory;
|
||||||
import io.druid.indexing.common.task.Task;
|
import io.druid.indexing.common.task.Task;
|
||||||
|
@ -42,7 +42,7 @@ public class RemoteTaskActionClient implements TaskActionClient
|
||||||
{
|
{
|
||||||
private final Task task;
|
private final Task task;
|
||||||
private final HttpClient httpClient;
|
private final HttpClient httpClient;
|
||||||
private final IndexingServiceSelector serviceProvider;
|
private final ServerDiscoverySelector selector;
|
||||||
private final RetryPolicyFactory retryPolicyFactory;
|
private final RetryPolicyFactory retryPolicyFactory;
|
||||||
private final ObjectMapper jsonMapper;
|
private final ObjectMapper jsonMapper;
|
||||||
|
|
||||||
|
@ -51,14 +51,14 @@ public class RemoteTaskActionClient implements TaskActionClient
|
||||||
public RemoteTaskActionClient(
|
public RemoteTaskActionClient(
|
||||||
Task task,
|
Task task,
|
||||||
HttpClient httpClient,
|
HttpClient httpClient,
|
||||||
IndexingServiceSelector serviceProvider,
|
ServerDiscoverySelector selector,
|
||||||
RetryPolicyFactory retryPolicyFactory,
|
RetryPolicyFactory retryPolicyFactory,
|
||||||
ObjectMapper jsonMapper
|
ObjectMapper jsonMapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.task = task;
|
this.task = task;
|
||||||
this.httpClient = httpClient;
|
this.httpClient = httpClient;
|
||||||
this.serviceProvider = serviceProvider;
|
this.selector = selector;
|
||||||
this.retryPolicyFactory = retryPolicyFactory;
|
this.retryPolicyFactory = retryPolicyFactory;
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
}
|
}
|
||||||
|
@ -127,19 +127,11 @@ public class RemoteTaskActionClient implements TaskActionClient
|
||||||
|
|
||||||
private URI getServiceUri() throws Exception
|
private URI getServiceUri() throws Exception
|
||||||
{
|
{
|
||||||
final Server instance = serviceProvider.pick();
|
final Server instance = selector.pick();
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
throw new ISE("Cannot find instance of indexer to talk to!");
|
throw new ISE("Cannot find instance of indexer to talk to!");
|
||||||
}
|
}
|
||||||
|
|
||||||
return new URI(
|
return new URI(String.format("%s://%s%s", instance.getScheme(), instance.getHost(), "/druid/indexer/v1/action"));
|
||||||
instance.getScheme(),
|
|
||||||
null,
|
|
||||||
instance.getHost(),
|
|
||||||
instance.getPort(),
|
|
||||||
"/druid/indexer/v1/action",
|
|
||||||
null,
|
|
||||||
null
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,8 @@ package io.druid.indexing.common.actions;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.http.client.HttpClient;
|
import com.metamx.http.client.HttpClient;
|
||||||
import io.druid.client.indexing.IndexingServiceSelector;
|
import io.druid.client.indexing.IndexingService;
|
||||||
|
import io.druid.curator.discovery.ServerDiscoverySelector;
|
||||||
import io.druid.guice.annotations.Global;
|
import io.druid.guice.annotations.Global;
|
||||||
import io.druid.indexing.common.RetryPolicyFactory;
|
import io.druid.indexing.common.RetryPolicyFactory;
|
||||||
import io.druid.indexing.common.task.Task;
|
import io.druid.indexing.common.task.Task;
|
||||||
|
@ -32,20 +33,20 @@ import io.druid.indexing.common.task.Task;
|
||||||
public class RemoteTaskActionClientFactory implements TaskActionClientFactory
|
public class RemoteTaskActionClientFactory implements TaskActionClientFactory
|
||||||
{
|
{
|
||||||
private final HttpClient httpClient;
|
private final HttpClient httpClient;
|
||||||
private final IndexingServiceSelector serviceProvider;
|
private final ServerDiscoverySelector selector;
|
||||||
private final RetryPolicyFactory retryPolicyFactory;
|
private final RetryPolicyFactory retryPolicyFactory;
|
||||||
private final ObjectMapper jsonMapper;
|
private final ObjectMapper jsonMapper;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public RemoteTaskActionClientFactory(
|
public RemoteTaskActionClientFactory(
|
||||||
@Global HttpClient httpClient,
|
@Global HttpClient httpClient,
|
||||||
IndexingServiceSelector serviceProvider,
|
@IndexingService ServerDiscoverySelector selector,
|
||||||
RetryPolicyFactory retryPolicyFactory,
|
RetryPolicyFactory retryPolicyFactory,
|
||||||
ObjectMapper jsonMapper
|
ObjectMapper jsonMapper
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.httpClient = httpClient;
|
this.httpClient = httpClient;
|
||||||
this.serviceProvider = serviceProvider;
|
this.selector = selector;
|
||||||
this.retryPolicyFactory = retryPolicyFactory;
|
this.retryPolicyFactory = retryPolicyFactory;
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
}
|
}
|
||||||
|
@ -53,6 +54,6 @@ public class RemoteTaskActionClientFactory implements TaskActionClientFactory
|
||||||
@Override
|
@Override
|
||||||
public TaskActionClient create(Task task)
|
public TaskActionClient create(Task task)
|
||||||
{
|
{
|
||||||
return new RemoteTaskActionClient(task, httpClient, serviceProvider, retryPolicyFactory, jsonMapper);
|
return new RemoteTaskActionClient(task, httpClient, selector, retryPolicyFactory, jsonMapper);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -125,7 +125,6 @@ public abstract class MergeTaskBase extends AbstractTask
|
||||||
final File taskDir = toolbox.getTaskWorkDir();
|
final File taskDir = toolbox.getTaskWorkDir();
|
||||||
|
|
||||||
try {
|
try {
|
||||||
|
|
||||||
final long startTime = System.currentTimeMillis();
|
final long startTime = System.currentTimeMillis();
|
||||||
|
|
||||||
log.info(
|
log.info(
|
||||||
|
|
|
@ -55,12 +55,19 @@ public class ForkingTaskRunnerConfig
|
||||||
private String classpath = System.getProperty("java.class.path");
|
private String classpath = System.getProperty("java.class.path");
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@Min(1024) @Max(65535)
|
@Min(1024)
|
||||||
|
@Max(65535)
|
||||||
private int startPort = 8080;
|
private int startPort = 8080;
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@NotNull
|
@NotNull
|
||||||
List<String> allowedPrefixes = Lists.newArrayList("com.metamx", "druid", "io.druid");
|
List<String> allowedPrefixes = Lists.newArrayList(
|
||||||
|
"com.metamx",
|
||||||
|
"druid",
|
||||||
|
"io.druid",
|
||||||
|
"user.timezone",
|
||||||
|
"file.encoding"
|
||||||
|
);
|
||||||
|
|
||||||
public int maxForks()
|
public int maxForks()
|
||||||
{
|
{
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package io.druid.indexing.coordinator.http;
|
package io.druid.indexing.coordinator.http;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
import com.google.common.base.Function;
|
import com.google.common.base.Function;
|
||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.collect.Collections2;
|
import com.google.common.collect.Collections2;
|
||||||
|
@ -89,7 +88,6 @@ public class IndexerCoordinatorResource
|
||||||
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
private final TaskStorageQueryAdapter taskStorageQueryAdapter;
|
||||||
private final TaskLogStreamer taskLogStreamer;
|
private final TaskLogStreamer taskLogStreamer;
|
||||||
private final JacksonConfigManager configManager;
|
private final JacksonConfigManager configManager;
|
||||||
private final ObjectMapper jsonMapper;
|
|
||||||
|
|
||||||
private AtomicReference<WorkerSetupData> workerSetupDataRef = null;
|
private AtomicReference<WorkerSetupData> workerSetupDataRef = null;
|
||||||
|
|
||||||
|
@ -98,21 +96,20 @@ public class IndexerCoordinatorResource
|
||||||
TaskMaster taskMaster,
|
TaskMaster taskMaster,
|
||||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||||
TaskLogStreamer taskLogStreamer,
|
TaskLogStreamer taskLogStreamer,
|
||||||
JacksonConfigManager configManager,
|
JacksonConfigManager configManager
|
||||||
ObjectMapper jsonMapper
|
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
this.taskMaster = taskMaster;
|
this.taskMaster = taskMaster;
|
||||||
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
this.taskStorageQueryAdapter = taskStorageQueryAdapter;
|
||||||
this.taskLogStreamer = taskLogStreamer;
|
this.taskLogStreamer = taskLogStreamer;
|
||||||
this.configManager = configManager;
|
this.configManager = configManager;
|
||||||
this.jsonMapper = jsonMapper;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@POST
|
@POST
|
||||||
@Path("/merge")
|
@Path("/merge")
|
||||||
@Consumes("application/json")
|
@Consumes("application/json")
|
||||||
@Produces("application/json")
|
@Produces("application/json")
|
||||||
|
@Deprecated
|
||||||
public Response doMerge(final Task task)
|
public Response doMerge(final Task task)
|
||||||
{
|
{
|
||||||
// legacy endpoint
|
// legacy endpoint
|
||||||
|
@ -123,6 +120,7 @@ public class IndexerCoordinatorResource
|
||||||
@Path("/index")
|
@Path("/index")
|
||||||
@Consumes("application/json")
|
@Consumes("application/json")
|
||||||
@Produces("application/json")
|
@Produces("application/json")
|
||||||
|
@Deprecated
|
||||||
public Response doIndex(final Task task)
|
public Response doIndex(final Task task)
|
||||||
{
|
{
|
||||||
return taskPost(task);
|
return taskPost(task);
|
||||||
|
|
|
@ -19,7 +19,6 @@
|
||||||
|
|
||||||
package io.druid.indexing.coordinator.http;
|
package io.druid.indexing.coordinator.http;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import io.druid.common.config.JacksonConfigManager;
|
import io.druid.common.config.JacksonConfigManager;
|
||||||
import io.druid.indexing.common.tasklogs.TaskLogStreamer;
|
import io.druid.indexing.common.tasklogs.TaskLogStreamer;
|
||||||
|
@ -39,10 +38,9 @@ public class OldIndexerCoordinatorResource extends IndexerCoordinatorResource
|
||||||
TaskMaster taskMaster,
|
TaskMaster taskMaster,
|
||||||
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
TaskStorageQueryAdapter taskStorageQueryAdapter,
|
||||||
TaskLogStreamer taskLogStreamer,
|
TaskLogStreamer taskLogStreamer,
|
||||||
JacksonConfigManager configManager,
|
JacksonConfigManager configManager
|
||||||
ObjectMapper jsonMapper
|
|
||||||
) throws Exception
|
) throws Exception
|
||||||
{
|
{
|
||||||
super(taskMaster, taskStorageQueryAdapter, taskLogStreamer, configManager, jsonMapper);
|
super(taskMaster, taskStorageQueryAdapter, taskLogStreamer, configManager);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -34,6 +34,7 @@ import com.google.common.base.Supplier;
|
||||||
import com.google.common.collect.Lists;
|
import com.google.common.collect.Lists;
|
||||||
import com.google.inject.Inject;
|
import com.google.inject.Inject;
|
||||||
import com.metamx.emitter.EmittingLogger;
|
import com.metamx.emitter.EmittingLogger;
|
||||||
|
import io.druid.guice.annotations.Json;
|
||||||
import io.druid.indexing.coordinator.setup.EC2NodeData;
|
import io.druid.indexing.coordinator.setup.EC2NodeData;
|
||||||
import io.druid.indexing.coordinator.setup.GalaxyUserData;
|
import io.druid.indexing.coordinator.setup.GalaxyUserData;
|
||||||
import io.druid.indexing.coordinator.setup.WorkerSetupData;
|
import io.druid.indexing.coordinator.setup.WorkerSetupData;
|
||||||
|
@ -55,7 +56,7 @@ public class EC2AutoScalingStrategy implements AutoScalingStrategy
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public EC2AutoScalingStrategy(
|
public EC2AutoScalingStrategy(
|
||||||
ObjectMapper jsonMapper,
|
@Json ObjectMapper jsonMapper,
|
||||||
AmazonEC2 amazonEC2Client,
|
AmazonEC2 amazonEC2Client,
|
||||||
SimpleResourceManagementConfig config,
|
SimpleResourceManagementConfig config,
|
||||||
Supplier<WorkerSetupData> workerSetupDataRef
|
Supplier<WorkerSetupData> workerSetupDataRef
|
||||||
|
|
|
@ -54,10 +54,14 @@ public class ScalingStats
|
||||||
|
|
||||||
public ScalingStats(int capacity)
|
public ScalingStats(int capacity)
|
||||||
{
|
{
|
||||||
this.recentEvents = MinMaxPriorityQueue
|
if (capacity == 0) {
|
||||||
.orderedBy(comparator)
|
this.recentEvents = MinMaxPriorityQueue.orderedBy(comparator).create();
|
||||||
.maximumSize(capacity)
|
} else {
|
||||||
.create();
|
this.recentEvents = MinMaxPriorityQueue
|
||||||
|
.orderedBy(comparator)
|
||||||
|
.maximumSize(capacity)
|
||||||
|
.create();
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public void addProvisionEvent(AutoScalingData data)
|
public void addProvisionEvent(AutoScalingData data)
|
||||||
|
|
|
@ -36,10 +36,6 @@ public class WorkerConfig
|
||||||
@NotNull
|
@NotNull
|
||||||
private String version = null;
|
private String version = null;
|
||||||
|
|
||||||
@JsonProperty
|
|
||||||
@NotNull
|
|
||||||
private String overlordService = null;
|
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@Min(1)
|
@Min(1)
|
||||||
private int capacity = Runtime.getRuntime().availableProcessors() - 1;
|
private int capacity = Runtime.getRuntime().availableProcessors() - 1;
|
||||||
|
@ -54,11 +50,6 @@ public class WorkerConfig
|
||||||
return version;
|
return version;
|
||||||
}
|
}
|
||||||
|
|
||||||
public String getOverlordService()
|
|
||||||
{
|
|
||||||
return overlordService;
|
|
||||||
}
|
|
||||||
|
|
||||||
public int getCapacity()
|
public int getCapacity()
|
||||||
{
|
{
|
||||||
return capacity;
|
return capacity;
|
||||||
|
|
|
@ -123,6 +123,7 @@ public class ExecutorLifecycle
|
||||||
jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(taskStatus)
|
jsonMapper.writerWithDefaultPrettyPrinter().writeValueAsString(taskStatus)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
statusFile.getParentFile().mkdirs();
|
||||||
jsonMapper.writeValue(statusFile, taskStatus);
|
jsonMapper.writeValue(statusFile, taskStatus);
|
||||||
|
|
||||||
return taskStatus;
|
return taskStatus;
|
||||||
|
|
|
@ -3,22 +3,22 @@
|
||||||
var oTable = [];
|
var oTable = [];
|
||||||
|
|
||||||
$(document).ready(function() {
|
$(document).ready(function() {
|
||||||
$.get('/mmx/merger/v1/runningTasks', function(data) {
|
$.get('/druid/indexer/v1/runningTasks', function(data) {
|
||||||
$('.running_loading').hide();
|
$('.running_loading').hide();
|
||||||
buildTable(data, $('#runningTable'), ["segments"]);
|
buildTable(data, $('#runningTable'), ["segments"]);
|
||||||
});
|
});
|
||||||
|
|
||||||
$.get('/mmx/merger/v1/pendingTasks', function(data) {
|
$.get('/druid/indexer/v1/pendingTasks', function(data) {
|
||||||
$('.pending_loading').hide();
|
$('.pending_loading').hide();
|
||||||
buildTable(data, $('#pendingTable'), ["segments"]);
|
buildTable(data, $('#pendingTable'), ["segments"]);
|
||||||
});
|
});
|
||||||
|
|
||||||
$.get('/mmx/merger/v1/workers', function(data) {
|
$.get('/druid/indexer/v1/workers', function(data) {
|
||||||
$('.workers_loading').hide();
|
$('.workers_loading').hide();
|
||||||
buildTable(data, $('#workerTable'));
|
buildTable(data, $('#workerTable'));
|
||||||
});
|
});
|
||||||
|
|
||||||
$.get('/mmx/merger/v1/scaling', function(data) {
|
$.get('/druid/indexer/v1/scaling', function(data) {
|
||||||
$('.events_loading').hide();
|
$('.events_loading').hide();
|
||||||
buildTable(data, $('#eventTable'));
|
buildTable(data, $('#eventTable'));
|
||||||
});
|
});
|
||||||
|
|
|
@ -359,8 +359,7 @@ public class TaskSerdeTest
|
||||||
false,
|
false,
|
||||||
new DataRollupSpec(ImmutableList.<AggregatorFactory>of(), QueryGranularity.NONE),
|
new DataRollupSpec(ImmutableList.<AggregatorFactory>of(), QueryGranularity.NONE),
|
||||||
null,
|
null,
|
||||||
false,
|
false
|
||||||
ImmutableList.<String>of()
|
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
|
|
||||||
|
|
|
@ -20,6 +20,7 @@
|
||||||
package io.druid.segment.realtime;
|
package io.druid.segment.realtime;
|
||||||
|
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.inject.Inject;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import io.druid.db.DbTablesConfig;
|
import io.druid.db.DbTablesConfig;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
|
@ -40,6 +41,7 @@ public class DbSegmentPublisher implements SegmentPublisher
|
||||||
private final DbTablesConfig config;
|
private final DbTablesConfig config;
|
||||||
private final IDBI dbi;
|
private final IDBI dbi;
|
||||||
|
|
||||||
|
@Inject
|
||||||
public DbSegmentPublisher(
|
public DbSegmentPublisher(
|
||||||
ObjectMapper jsonMapper,
|
ObjectMapper jsonMapper,
|
||||||
DbTablesConfig config,
|
DbTablesConfig config,
|
||||||
|
|
|
@ -29,7 +29,7 @@ public class DruidServerConfig
|
||||||
{
|
{
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@Min(0)
|
@Min(0)
|
||||||
private long maxSize = -1;
|
private long maxSize = 0;
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
private String tier = "_default_tier";
|
private String tier = "_default_tier";
|
||||||
|
|
|
@ -26,8 +26,8 @@ import com.metamx.common.IAE;
|
||||||
import com.metamx.common.ISE;
|
import com.metamx.common.ISE;
|
||||||
import com.metamx.http.client.HttpClient;
|
import com.metamx.http.client.HttpClient;
|
||||||
import com.metamx.http.client.response.InputStreamResponseHandler;
|
import com.metamx.http.client.response.InputStreamResponseHandler;
|
||||||
import io.druid.client.selector.DiscoverySelector;
|
|
||||||
import io.druid.client.selector.Server;
|
import io.druid.client.selector.Server;
|
||||||
|
import io.druid.curator.discovery.ServerDiscoverySelector;
|
||||||
import io.druid.guice.annotations.Global;
|
import io.druid.guice.annotations.Global;
|
||||||
import io.druid.timeline.DataSegment;
|
import io.druid.timeline.DataSegment;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -43,18 +43,18 @@ public class IndexingServiceClient
|
||||||
|
|
||||||
private final HttpClient client;
|
private final HttpClient client;
|
||||||
private final ObjectMapper jsonMapper;
|
private final ObjectMapper jsonMapper;
|
||||||
private final DiscoverySelector<Server> serviceProvider;
|
private final ServerDiscoverySelector selector;
|
||||||
|
|
||||||
@Inject
|
@Inject
|
||||||
public IndexingServiceClient(
|
public IndexingServiceClient(
|
||||||
@Global HttpClient client,
|
@Global HttpClient client,
|
||||||
ObjectMapper jsonMapper,
|
ObjectMapper jsonMapper,
|
||||||
@IndexingService DiscoverySelector<Server> serviceProvider
|
@IndexingService ServerDiscoverySelector selector
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.client = client;
|
this.client = client;
|
||||||
this.jsonMapper = jsonMapper;
|
this.jsonMapper = jsonMapper;
|
||||||
this.serviceProvider = serviceProvider;
|
this.selector = selector;
|
||||||
}
|
}
|
||||||
|
|
||||||
public void mergeSegments(List<DataSegment> segments)
|
public void mergeSegments(List<DataSegment> segments)
|
||||||
|
@ -72,28 +72,28 @@ public class IndexingServiceClient
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
runQuery("merge", new ClientAppendQuery(dataSource, segments));
|
runQuery(new ClientAppendQuery(dataSource, segments));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void killSegments(String dataSource, Interval interval)
|
public void killSegments(String dataSource, Interval interval)
|
||||||
{
|
{
|
||||||
runQuery("index", new ClientKillQuery(dataSource, interval));
|
runQuery(new ClientKillQuery(dataSource, interval));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void upgradeSegment(DataSegment dataSegment)
|
public void upgradeSegment(DataSegment dataSegment)
|
||||||
{
|
{
|
||||||
runQuery("task", new ClientConversionQuery(dataSegment));
|
runQuery(new ClientConversionQuery(dataSegment));
|
||||||
}
|
}
|
||||||
|
|
||||||
public void upgradeSegments(String dataSource, Interval interval)
|
public void upgradeSegments(String dataSource, Interval interval)
|
||||||
{
|
{
|
||||||
runQuery("task", new ClientConversionQuery(dataSource, interval));
|
runQuery(new ClientConversionQuery(dataSource, interval));
|
||||||
}
|
}
|
||||||
|
|
||||||
private InputStream runQuery(String endpoint, Object queryObject)
|
private InputStream runQuery(Object queryObject)
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
return client.post(new URL(String.format("%s/%s", baseUrl(), endpoint)))
|
return client.post(new URL(String.format("%s/task", baseUrl())))
|
||||||
.setContent("application/json", jsonMapper.writeValueAsBytes(queryObject))
|
.setContent("application/json", jsonMapper.writeValueAsBytes(queryObject))
|
||||||
.go(RESPONSE_HANDLER)
|
.go(RESPONSE_HANDLER)
|
||||||
.get();
|
.get();
|
||||||
|
@ -106,12 +106,12 @@ public class IndexingServiceClient
|
||||||
private String baseUrl()
|
private String baseUrl()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
final Server instance = serviceProvider.pick();
|
final Server instance = selector.pick();
|
||||||
if (instance == null) {
|
if (instance == null) {
|
||||||
throw new ISE("Cannot find instance of indexingService");
|
throw new ISE("Cannot find instance of indexingService");
|
||||||
}
|
}
|
||||||
|
|
||||||
return String.format("http://%s:%s/druid/indexer/v1", instance.getHost(), instance.getPort());
|
return String.format("http://%s/druid/indexer/v1", instance.getHost());
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
|
|
|
@ -35,6 +35,8 @@ import io.druid.guice.DruidBinders;
|
||||||
import io.druid.guice.JsonConfigProvider;
|
import io.druid.guice.JsonConfigProvider;
|
||||||
import io.druid.guice.KeyHolder;
|
import io.druid.guice.KeyHolder;
|
||||||
import io.druid.guice.LazySingleton;
|
import io.druid.guice.LazySingleton;
|
||||||
|
import io.druid.guice.LifecycleModule;
|
||||||
|
import io.druid.guice.annotations.Self;
|
||||||
import io.druid.server.DruidNode;
|
import io.druid.server.DruidNode;
|
||||||
import io.druid.server.initialization.CuratorDiscoveryConfig;
|
import io.druid.server.initialization.CuratorDiscoveryConfig;
|
||||||
import org.apache.curator.framework.CuratorFramework;
|
import org.apache.curator.framework.CuratorFramework;
|
||||||
|
@ -59,7 +61,7 @@ import java.util.concurrent.ThreadFactory;
|
||||||
/**
|
/**
|
||||||
* The DiscoveryModule allows for the registration of Keys of DruidNode objects, which it intends to be
|
* The DiscoveryModule allows for the registration of Keys of DruidNode objects, which it intends to be
|
||||||
* automatically announced at the end of the lifecycle start.
|
* automatically announced at the end of the lifecycle start.
|
||||||
*
|
* <p/>
|
||||||
* In order for this to work a ServiceAnnouncer instance *must* be injected and instantiated first.
|
* In order for this to work a ServiceAnnouncer instance *must* be injected and instantiated first.
|
||||||
* This can often be achieved by registering ServiceAnnouncer.class with the LifecycleModule.
|
* This can often be achieved by registering ServiceAnnouncer.class with the LifecycleModule.
|
||||||
*/
|
*/
|
||||||
|
@ -69,7 +71,7 @@ public class DiscoveryModule implements Module
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Requests that the un-annotated DruidNode instance be injected and published as part of the lifecycle.
|
* Requests that the un-annotated DruidNode instance be injected and published as part of the lifecycle.
|
||||||
*
|
* <p/>
|
||||||
* That is, this module will announce the DruidNode instance returned by
|
* That is, this module will announce the DruidNode instance returned by
|
||||||
* injector.getInstance(Key.get(DruidNode.class)) automatically.
|
* injector.getInstance(Key.get(DruidNode.class)) automatically.
|
||||||
* Announcement will happen in the LAST stage of the Lifecycle
|
* Announcement will happen in the LAST stage of the Lifecycle
|
||||||
|
@ -81,7 +83,7 @@ public class DiscoveryModule implements Module
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Requests that the annotated DruidNode instance be injected and published as part of the lifecycle.
|
* Requests that the annotated DruidNode instance be injected and published as part of the lifecycle.
|
||||||
*
|
* <p/>
|
||||||
* That is, this module will announce the DruidNode instance returned by
|
* That is, this module will announce the DruidNode instance returned by
|
||||||
* injector.getInstance(Key.get(DruidNode.class, annotation)) automatically.
|
* injector.getInstance(Key.get(DruidNode.class, annotation)) automatically.
|
||||||
* Announcement will happen in the LAST stage of the Lifecycle
|
* Announcement will happen in the LAST stage of the Lifecycle
|
||||||
|
@ -95,7 +97,7 @@ public class DiscoveryModule implements Module
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Requests that the annotated DruidNode instance be injected and published as part of the lifecycle.
|
* Requests that the annotated DruidNode instance be injected and published as part of the lifecycle.
|
||||||
*
|
* <p/>
|
||||||
* That is, this module will announce the DruidNode instance returned by
|
* That is, this module will announce the DruidNode instance returned by
|
||||||
* injector.getInstance(Key.get(DruidNode.class, annotation)) automatically.
|
* injector.getInstance(Key.get(DruidNode.class, annotation)) automatically.
|
||||||
* Announcement will happen in the LAST stage of the Lifecycle
|
* Announcement will happen in the LAST stage of the Lifecycle
|
||||||
|
@ -109,7 +111,7 @@ public class DiscoveryModule implements Module
|
||||||
|
|
||||||
/**
|
/**
|
||||||
* Requests that the keyed DruidNode instance be injected and published as part of the lifecycle.
|
* Requests that the keyed DruidNode instance be injected and published as part of the lifecycle.
|
||||||
*
|
* <p/>
|
||||||
* That is, this module will announce the DruidNode instance returned by
|
* That is, this module will announce the DruidNode instance returned by
|
||||||
* injector.getInstance(Key.get(DruidNode.class, annotation)) automatically.
|
* injector.getInstance(Key.get(DruidNode.class, annotation)) automatically.
|
||||||
* Announcement will happen in the LAST stage of the Lifecycle
|
* Announcement will happen in the LAST stage of the Lifecycle
|
||||||
|
@ -119,6 +121,7 @@ public class DiscoveryModule implements Module
|
||||||
public static void registerKey(Binder binder, Key<DruidNode> key)
|
public static void registerKey(Binder binder, Key<DruidNode> key)
|
||||||
{
|
{
|
||||||
DruidBinders.discoveryAnnouncementBinder(binder).addBinding().toInstance(new KeyHolder<>(key));
|
DruidBinders.discoveryAnnouncementBinder(binder).addBinding().toInstance(new KeyHolder<>(key));
|
||||||
|
LifecycleModule.register(binder, ServiceAnnouncer.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -131,13 +134,14 @@ public class DiscoveryModule implements Module
|
||||||
// Build the binder so that it will at a minimum inject an empty set.
|
// Build the binder so that it will at a minimum inject an empty set.
|
||||||
DruidBinders.discoveryAnnouncementBinder(binder);
|
DruidBinders.discoveryAnnouncementBinder(binder);
|
||||||
|
|
||||||
// We bind this eagerly so that it gets instantiated and registers stuff with Lifecycle as a side-effect
|
|
||||||
binder.bind(ServiceAnnouncer.class)
|
binder.bind(ServiceAnnouncer.class)
|
||||||
.to(Key.get(CuratorServiceAnnouncer.class, Names.named(NAME)))
|
.to(Key.get(CuratorServiceAnnouncer.class, Names.named(NAME)))
|
||||||
.asEagerSingleton();
|
.in(LazySingleton.class);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Provides @LazySingleton @Named(NAME)
|
@Provides
|
||||||
|
@LazySingleton
|
||||||
|
@Named(NAME)
|
||||||
public CuratorServiceAnnouncer getServiceAnnouncer(
|
public CuratorServiceAnnouncer getServiceAnnouncer(
|
||||||
final CuratorServiceAnnouncer announcer,
|
final CuratorServiceAnnouncer announcer,
|
||||||
final Injector injector,
|
final Injector injector,
|
||||||
|
@ -181,7 +185,8 @@ public class DiscoveryModule implements Module
|
||||||
return announcer;
|
return announcer;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Provides @LazySingleton
|
@Provides
|
||||||
|
@LazySingleton
|
||||||
public ServiceDiscovery<Void> getServiceDiscovery(
|
public ServiceDiscovery<Void> getServiceDiscovery(
|
||||||
CuratorFramework curator,
|
CuratorFramework curator,
|
||||||
CuratorDiscoveryConfig config,
|
CuratorDiscoveryConfig config,
|
||||||
|
@ -217,13 +222,21 @@ public class DiscoveryModule implements Module
|
||||||
throw Throwables.propagate(e);
|
throw Throwables.propagate(e);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
},
|
}
|
||||||
Lifecycle.Stage.LAST
|
|
||||||
);
|
);
|
||||||
|
|
||||||
return serviceDiscovery;
|
return serviceDiscovery;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Provides
|
||||||
|
@LazySingleton
|
||||||
|
public ServerDiscoveryFactory getServerDiscoveryFactory(
|
||||||
|
ServiceDiscovery<Void> serviceDiscovery
|
||||||
|
)
|
||||||
|
{
|
||||||
|
return new ServerDiscoveryFactory(serviceDiscovery);
|
||||||
|
}
|
||||||
|
|
||||||
private static class NoopServiceDiscovery<T> implements ServiceDiscovery<T>
|
private static class NoopServiceDiscovery<T> implements ServiceDiscovery<T>
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -0,0 +1,72 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012, 2013 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.curator.discovery;
|
||||||
|
|
||||||
|
import com.google.inject.Inject;
|
||||||
|
import org.apache.curator.x.discovery.ServiceDiscovery;
|
||||||
|
import org.apache.curator.x.discovery.ServiceInstance;
|
||||||
|
import org.apache.curator.x.discovery.ServiceProvider;
|
||||||
|
|
||||||
|
import java.io.IOException;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class ServerDiscoveryFactory
|
||||||
|
{
|
||||||
|
private final ServiceDiscovery<Void> serviceDiscovery;
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public ServerDiscoveryFactory(ServiceDiscovery<Void> serviceDiscovery)
|
||||||
|
{
|
||||||
|
this.serviceDiscovery = serviceDiscovery;
|
||||||
|
}
|
||||||
|
|
||||||
|
public ServerDiscoverySelector createSelector(String serviceName)
|
||||||
|
{
|
||||||
|
if (serviceName == null) {
|
||||||
|
return new ServerDiscoverySelector(new NoopServiceProvider());
|
||||||
|
}
|
||||||
|
|
||||||
|
final ServiceProvider serviceProvider = serviceDiscovery.serviceProviderBuilder().serviceName(serviceName).build();
|
||||||
|
return new ServerDiscoverySelector(serviceProvider);
|
||||||
|
}
|
||||||
|
|
||||||
|
private static class NoopServiceProvider<T> implements ServiceProvider<T>
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void start() throws Exception
|
||||||
|
{
|
||||||
|
// do nothing
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public ServiceInstance<T> getInstance() throws Exception
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void close() throws IOException
|
||||||
|
{
|
||||||
|
// do nothing
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -17,9 +17,8 @@
|
||||||
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
* Foundation, Inc., 51 Franklin Street, Fifth Floor, Boston, MA 02110-1301, USA.
|
||||||
*/
|
*/
|
||||||
|
|
||||||
package io.druid.client.indexing;
|
package io.druid.curator.discovery;
|
||||||
|
|
||||||
import com.google.inject.Inject;
|
|
||||||
import com.metamx.common.lifecycle.LifecycleStart;
|
import com.metamx.common.lifecycle.LifecycleStart;
|
||||||
import com.metamx.common.lifecycle.LifecycleStop;
|
import com.metamx.common.lifecycle.LifecycleStop;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
|
@ -28,21 +27,18 @@ import io.druid.client.selector.Server;
|
||||||
import org.apache.curator.x.discovery.ServiceInstance;
|
import org.apache.curator.x.discovery.ServiceInstance;
|
||||||
import org.apache.curator.x.discovery.ServiceProvider;
|
import org.apache.curator.x.discovery.ServiceProvider;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class IndexingServiceSelector implements DiscoverySelector<Server>
|
public class ServerDiscoverySelector implements DiscoverySelector<Server>
|
||||||
{
|
{
|
||||||
private static final Logger log = new Logger(IndexingServiceSelector.class);
|
private static final Logger log = new Logger(ServerDiscoverySelector.class);
|
||||||
|
|
||||||
private final ServiceProvider serviceProvider;
|
private final ServiceProvider serviceProvider;
|
||||||
|
|
||||||
@Inject
|
public ServerDiscoverySelector(ServiceProvider serviceProvider)
|
||||||
public IndexingServiceSelector(
|
{
|
||||||
@Nullable @IndexingService ServiceProvider serviceProvider
|
|
||||||
) {
|
|
||||||
this.serviceProvider = serviceProvider;
|
this.serviceProvider = serviceProvider;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -54,7 +50,12 @@ public class IndexingServiceSelector implements DiscoverySelector<Server>
|
||||||
instance = serviceProvider.getInstance();
|
instance = serviceProvider.getInstance();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
log.info(e, "");
|
log.info(e, "Exception getting instance");
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
if (instance == null) {
|
||||||
|
log.error("No server instance found");
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,17 +22,10 @@ package io.druid.guice;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
import com.google.inject.Module;
|
import com.google.inject.Module;
|
||||||
import com.google.inject.Provides;
|
import com.google.inject.Provides;
|
||||||
import com.google.inject.TypeLiteral;
|
|
||||||
import io.druid.client.indexing.IndexingService;
|
import io.druid.client.indexing.IndexingService;
|
||||||
import io.druid.client.indexing.IndexingServiceSelector;
|
|
||||||
import io.druid.client.indexing.IndexingServiceSelectorConfig;
|
import io.druid.client.indexing.IndexingServiceSelectorConfig;
|
||||||
import io.druid.client.selector.DiscoverySelector;
|
import io.druid.curator.discovery.ServerDiscoveryFactory;
|
||||||
import io.druid.client.selector.Server;
|
import io.druid.curator.discovery.ServerDiscoverySelector;
|
||||||
import org.apache.curator.x.discovery.ServiceDiscovery;
|
|
||||||
import org.apache.curator.x.discovery.ServiceInstance;
|
|
||||||
import org.apache.curator.x.discovery.ServiceProvider;
|
|
||||||
|
|
||||||
import java.io.IOException;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
|
@ -42,42 +35,16 @@ public class IndexingServiceDiscoveryModule implements Module
|
||||||
public void configure(Binder binder)
|
public void configure(Binder binder)
|
||||||
{
|
{
|
||||||
JsonConfigProvider.bind(binder, "druid.selectors.indexing", IndexingServiceSelectorConfig.class);
|
JsonConfigProvider.bind(binder, "druid.selectors.indexing", IndexingServiceSelectorConfig.class);
|
||||||
binder.bind(new TypeLiteral<DiscoverySelector<Server>>(){})
|
|
||||||
.annotatedWith(IndexingService.class)
|
|
||||||
.to(IndexingServiceSelector.class);
|
|
||||||
|
|
||||||
binder.bind(IndexingServiceSelector.class).in(ManageLifecycle.class);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Provides
|
@Provides
|
||||||
@LazySingleton @IndexingService
|
@IndexingService
|
||||||
public ServiceProvider getServiceProvider(
|
@ManageLifecycle
|
||||||
|
public ServerDiscoverySelector getServiceProvider(
|
||||||
IndexingServiceSelectorConfig config,
|
IndexingServiceSelectorConfig config,
|
||||||
ServiceDiscovery<Void> serviceDiscovery
|
ServerDiscoveryFactory serverDiscoveryFactory
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (config.getServiceName() == null) {
|
return serverDiscoveryFactory.createSelector(config.getServiceName());
|
||||||
return new ServiceProvider()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void start() throws Exception
|
|
||||||
{
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public ServiceInstance getInstance() throws Exception
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close() throws IOException
|
|
||||||
{
|
|
||||||
|
|
||||||
}
|
|
||||||
};
|
|
||||||
}
|
|
||||||
return serviceDiscovery.serviceProviderBuilder().serviceName(config.getServiceName()).build();
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -110,7 +110,6 @@ public class S3DataSegmentPuller implements DataSegmentPuller
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
catch (IOException e) {
|
catch (IOException e) {
|
||||||
FileUtils.deleteDirectory(outDir);
|
|
||||||
throw new IOException(String.format("Problem decompressing object[%s]", s3Obj), e);
|
throw new IOException(String.format("Problem decompressing object[%s]", s3Obj), e);
|
||||||
}
|
}
|
||||||
finally {
|
finally {
|
||||||
|
@ -125,6 +124,16 @@ public class S3DataSegmentPuller implements DataSegmentPuller
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
try {
|
||||||
|
FileUtils.deleteDirectory(outDir);
|
||||||
|
} catch (IOException ioe) {
|
||||||
|
log.warn(
|
||||||
|
ioe,
|
||||||
|
"Failed to remove output directory for segment[%s] after exception: %s",
|
||||||
|
segment.getIdentifier(),
|
||||||
|
outDir
|
||||||
|
);
|
||||||
|
}
|
||||||
throw new SegmentLoadingException(e, e.getMessage());
|
throw new SegmentLoadingException(e, e.getMessage());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -116,11 +116,9 @@ public class QueryServlet extends HttpServlet
|
||||||
emitter.emit(
|
emitter.emit(
|
||||||
new ServiceMetricEvent.Builder()
|
new ServiceMetricEvent.Builder()
|
||||||
.setUser2(query.getDataSource())
|
.setUser2(query.getDataSource())
|
||||||
//.setUser3(originatorType)
|
|
||||||
.setUser4(query.getType())
|
.setUser4(query.getType())
|
||||||
.setUser5(query.getIntervals().get(0).toString())
|
.setUser5(query.getIntervals().get(0).toString())
|
||||||
.setUser6(String.valueOf(query.hasFilters()))
|
.setUser6(String.valueOf(query.hasFilters()))
|
||||||
//.setUser8(originatorId)
|
|
||||||
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString())
|
.setUser9(query.getDuration().toPeriod().toStandardMinutes().toString())
|
||||||
.build("request/time", requestTime)
|
.build("request/time", requestTime)
|
||||||
);
|
);
|
||||||
|
|
|
@ -29,6 +29,10 @@ import java.util.List;
|
||||||
*/
|
*/
|
||||||
public class ExtensionsConfig
|
public class ExtensionsConfig
|
||||||
{
|
{
|
||||||
|
@JsonProperty
|
||||||
|
@NotNull
|
||||||
|
private boolean searchCurrentClassloader = true;
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
@NotNull
|
@NotNull
|
||||||
private List<String> coordinates = ImmutableList.of();
|
private List<String> coordinates = ImmutableList.of();
|
||||||
|
@ -44,6 +48,11 @@ public class ExtensionsConfig
|
||||||
"https://metamx.artifactoryonline.com/metamx/pub-libs-releases-local"
|
"https://metamx.artifactoryonline.com/metamx/pub-libs-releases-local"
|
||||||
);
|
);
|
||||||
|
|
||||||
|
public boolean searchCurrentClassloader()
|
||||||
|
{
|
||||||
|
return searchCurrentClassloader;
|
||||||
|
}
|
||||||
|
|
||||||
public List<String> getCoordinates()
|
public List<String> getCoordinates()
|
||||||
{
|
{
|
||||||
return coordinates;
|
return coordinates;
|
||||||
|
|
|
@ -19,6 +19,8 @@
|
||||||
|
|
||||||
package io.druid.server.initialization;
|
package io.druid.server.initialization;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.fasterxml.jackson.jaxrs.json.JacksonJsonProvider;
|
||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import com.google.common.primitives.Ints;
|
import com.google.common.primitives.Ints;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
|
@ -28,6 +30,7 @@ import com.google.inject.Injector;
|
||||||
import com.google.inject.Provides;
|
import com.google.inject.Provides;
|
||||||
import com.google.inject.ProvisionException;
|
import com.google.inject.ProvisionException;
|
||||||
import com.google.inject.Scopes;
|
import com.google.inject.Scopes;
|
||||||
|
import com.google.inject.Singleton;
|
||||||
import com.metamx.common.lifecycle.Lifecycle;
|
import com.metamx.common.lifecycle.Lifecycle;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import com.sun.jersey.api.core.DefaultResourceConfig;
|
import com.sun.jersey.api.core.DefaultResourceConfig;
|
||||||
|
@ -39,7 +42,9 @@ import io.druid.guice.Jerseys;
|
||||||
import io.druid.guice.JsonConfigProvider;
|
import io.druid.guice.JsonConfigProvider;
|
||||||
import io.druid.guice.LazySingleton;
|
import io.druid.guice.LazySingleton;
|
||||||
import io.druid.guice.annotations.JSR311Resource;
|
import io.druid.guice.annotations.JSR311Resource;
|
||||||
|
import io.druid.guice.annotations.Json;
|
||||||
import io.druid.guice.annotations.Self;
|
import io.druid.guice.annotations.Self;
|
||||||
|
import io.druid.jackson.DefaultObjectMapper;
|
||||||
import io.druid.server.DruidNode;
|
import io.druid.server.DruidNode;
|
||||||
import io.druid.server.StatusResource;
|
import io.druid.server.StatusResource;
|
||||||
import org.eclipse.jetty.server.Connector;
|
import org.eclipse.jetty.server.Connector;
|
||||||
|
@ -95,7 +100,8 @@ public class JettyServerModule extends JerseyServletModule
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
@Provides @LazySingleton
|
@Provides
|
||||||
|
@LazySingleton
|
||||||
public Server getServer(Injector injector, Lifecycle lifecycle, @Self DruidNode node, ServerConfig config)
|
public Server getServer(Injector injector, Lifecycle lifecycle, @Self DruidNode node, ServerConfig config)
|
||||||
{
|
{
|
||||||
JettyServerInitializer initializer = injector.getInstance(JettyServerInitializer.class);
|
JettyServerInitializer initializer = injector.getInstance(JettyServerInitializer.class);
|
||||||
|
@ -133,6 +139,15 @@ public class JettyServerModule extends JerseyServletModule
|
||||||
return server;
|
return server;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Provides
|
||||||
|
@Singleton
|
||||||
|
public JacksonJsonProvider getJacksonJsonProvider(@Json ObjectMapper objectMapper)
|
||||||
|
{
|
||||||
|
final JacksonJsonProvider provider = new JacksonJsonProvider();
|
||||||
|
provider.setMapper(objectMapper);
|
||||||
|
return provider;
|
||||||
|
}
|
||||||
|
|
||||||
private static Server makeJettyServer(@Self DruidNode node, ServerConfig config)
|
private static Server makeJettyServer(@Self DruidNode node, ServerConfig config)
|
||||||
{
|
{
|
||||||
final QueuedThreadPool threadPool = new QueuedThreadPool();
|
final QueuedThreadPool threadPool = new QueuedThreadPool();
|
||||||
|
|
|
@ -161,7 +161,7 @@ public class DruidMaster
|
||||||
this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d");
|
this.exec = scheduledExecutorFactory.create(1, "Master-Exec--%d");
|
||||||
|
|
||||||
this.leaderLatch = new AtomicReference<>(null);
|
this.leaderLatch = new AtomicReference<>(null);
|
||||||
this.segmentSettingsAtomicReference= new AtomicReference<>(null);
|
this.segmentSettingsAtomicReference = new AtomicReference<>(null);
|
||||||
this.loadManagementPeons = loadQueuePeonMap;
|
this.loadManagementPeons = loadQueuePeonMap;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -471,10 +471,13 @@ public class DruidMaster
|
||||||
serverInventoryView.start();
|
serverInventoryView.start();
|
||||||
|
|
||||||
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
|
final List<Pair<? extends MasterRunnable, Duration>> masterRunnables = Lists.newArrayList();
|
||||||
segmentSettingsAtomicReference = configManager.watch(MasterSegmentSettings.CONFIG_KEY, MasterSegmentSettings.class,new MasterSegmentSettings.Builder().build());
|
segmentSettingsAtomicReference = configManager.watch(
|
||||||
|
MasterSegmentSettings.CONFIG_KEY,
|
||||||
|
MasterSegmentSettings.class,
|
||||||
|
new MasterSegmentSettings.Builder().build()
|
||||||
|
);
|
||||||
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
|
masterRunnables.add(Pair.of(new MasterComputeManagerRunnable(), config.getMasterPeriod()));
|
||||||
if (indexingServiceClient != null) {
|
if (indexingServiceClient != null) {
|
||||||
|
|
||||||
masterRunnables.add(
|
masterRunnables.add(
|
||||||
Pair.of(
|
Pair.of(
|
||||||
new MasterIndexingServiceRunnable(
|
new MasterIndexingServiceRunnable(
|
||||||
|
|
|
@ -42,7 +42,7 @@ public abstract class DruidMasterConfig
|
||||||
@Default("PT1800s")
|
@Default("PT1800s")
|
||||||
public abstract Duration getMasterSegmentMergerPeriod();
|
public abstract Duration getMasterSegmentMergerPeriod();
|
||||||
|
|
||||||
@Config("druid.master.merger.on")
|
@Config("druid.master.merge.on")
|
||||||
public boolean isMergeSegments()
|
public boolean isMergeSegments()
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
|
|
|
@ -24,8 +24,8 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
public class MasterSegmentSettings
|
public class MasterSegmentSettings
|
||||||
{
|
{
|
||||||
public static final String CONFIG_KEY = "master.dynamicConfigs";
|
public static final String CONFIG_KEY = "master.dynamicConfigs";
|
||||||
private long millisToWaitBeforeDeleting=15 * 60 * 1000L;
|
private long millisToWaitBeforeDeleting = 15 * 60 * 1000L;
|
||||||
private long mergeBytesLimit= 100000000L;
|
private long mergeBytesLimit = 100000000L;
|
||||||
private int mergeSegmentsLimit = Integer.MAX_VALUE;
|
private int mergeSegmentsLimit = Integer.MAX_VALUE;
|
||||||
private int maxSegmentsToMove = 5;
|
private int maxSegmentsToMove = 5;
|
||||||
private boolean emitBalancingStats = false;
|
private boolean emitBalancingStats = false;
|
||||||
|
@ -39,11 +39,11 @@ public class MasterSegmentSettings
|
||||||
@JsonProperty("emitBalancingStats") Boolean emitBalancingStats
|
@JsonProperty("emitBalancingStats") Boolean emitBalancingStats
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
this.maxSegmentsToMove=maxSegmentsToMove;
|
this.maxSegmentsToMove = maxSegmentsToMove;
|
||||||
this.millisToWaitBeforeDeleting=millisToWaitBeforeDeleting;
|
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting;
|
||||||
this.mergeSegmentsLimit=mergeSegmentsLimit;
|
this.mergeSegmentsLimit = mergeSegmentsLimit;
|
||||||
this.mergeBytesLimit=mergeBytesLimit;
|
this.mergeBytesLimit = mergeBytesLimit;
|
||||||
this.emitBalancingStats = emitBalancingStats;
|
this.emitBalancingStats = emitBalancingStats;
|
||||||
}
|
}
|
||||||
|
|
||||||
public static String getConfigKey()
|
public static String getConfigKey()
|
||||||
|
@ -81,7 +81,6 @@ public class MasterSegmentSettings
|
||||||
return maxSegmentsToMove;
|
return maxSegmentsToMove;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
|
||||||
public static class Builder
|
public static class Builder
|
||||||
{
|
{
|
||||||
public static final String CONFIG_KEY = "master.dynamicConfigs";
|
public static final String CONFIG_KEY = "master.dynamicConfigs";
|
||||||
|
@ -93,14 +92,16 @@ public class MasterSegmentSettings
|
||||||
|
|
||||||
public Builder()
|
public Builder()
|
||||||
{
|
{
|
||||||
this.millisToWaitBeforeDeleting=15 * 60 * 1000L;
|
this(15 * 60 * 1000L, 100000000L, Integer.MAX_VALUE, 5, false);
|
||||||
this.mergeBytesLimit= 100000000L;
|
|
||||||
this.mergeSegmentsLimit= Integer.MAX_VALUE;
|
|
||||||
this.maxSegmentsToMove = 5;
|
|
||||||
this.emitBalancingStats = false;
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder(long millisToWaitBeforeDeleting, long mergeBytesLimit, int mergeSegmentsLimit, int maxSegmentsToMove, boolean emitBalancingStats)
|
public Builder(
|
||||||
|
long millisToWaitBeforeDeleting,
|
||||||
|
long mergeBytesLimit,
|
||||||
|
int mergeSegmentsLimit,
|
||||||
|
int maxSegmentsToMove,
|
||||||
|
boolean emitBalancingStats
|
||||||
|
)
|
||||||
{
|
{
|
||||||
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting;
|
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting;
|
||||||
this.mergeBytesLimit = mergeBytesLimit;
|
this.mergeBytesLimit = mergeBytesLimit;
|
||||||
|
@ -111,31 +112,37 @@ public class MasterSegmentSettings
|
||||||
|
|
||||||
public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting)
|
public Builder withMillisToWaitBeforeDeleting(long millisToWaitBeforeDeleting)
|
||||||
{
|
{
|
||||||
this.millisToWaitBeforeDeleting=millisToWaitBeforeDeleting;
|
this.millisToWaitBeforeDeleting = millisToWaitBeforeDeleting;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder withMergeBytesLimit(long mergeBytesLimit)
|
public Builder withMergeBytesLimit(long mergeBytesLimit)
|
||||||
{
|
{
|
||||||
this.mergeBytesLimit=mergeBytesLimit;
|
this.mergeBytesLimit = mergeBytesLimit;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder withMergeSegmentsLimit(int mergeSegmentsLimit)
|
public Builder withMergeSegmentsLimit(int mergeSegmentsLimit)
|
||||||
{
|
{
|
||||||
this.mergeSegmentsLimit=mergeSegmentsLimit;
|
this.mergeSegmentsLimit = mergeSegmentsLimit;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public Builder withMaxSegmentsToMove(int maxSegmentsToMove)
|
public Builder withMaxSegmentsToMove(int maxSegmentsToMove)
|
||||||
{
|
{
|
||||||
this.maxSegmentsToMove=maxSegmentsToMove;
|
this.maxSegmentsToMove = maxSegmentsToMove;
|
||||||
return this;
|
return this;
|
||||||
}
|
}
|
||||||
|
|
||||||
public MasterSegmentSettings build()
|
public MasterSegmentSettings build()
|
||||||
{
|
{
|
||||||
return new MasterSegmentSettings(millisToWaitBeforeDeleting,mergeBytesLimit,mergeSegmentsLimit,maxSegmentsToMove, emitBalancingStats);
|
return new MasterSegmentSettings(
|
||||||
|
millisToWaitBeforeDeleting,
|
||||||
|
mergeBytesLimit,
|
||||||
|
mergeSegmentsLimit,
|
||||||
|
maxSegmentsToMove,
|
||||||
|
emitBalancingStats
|
||||||
|
);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,59 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012, 2013 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;
|
||||||
|
|
||||||
|
import com.google.inject.Injector;
|
||||||
|
import com.google.inject.servlet.GuiceFilter;
|
||||||
|
import io.druid.server.QueryServlet;
|
||||||
|
import io.druid.server.initialization.JettyServerInitializer;
|
||||||
|
import org.eclipse.jetty.server.Handler;
|
||||||
|
import org.eclipse.jetty.server.Server;
|
||||||
|
import org.eclipse.jetty.server.handler.DefaultHandler;
|
||||||
|
import org.eclipse.jetty.server.handler.HandlerList;
|
||||||
|
import org.eclipse.jetty.servlet.DefaultServlet;
|
||||||
|
import org.eclipse.jetty.servlet.ServletContextHandler;
|
||||||
|
import org.eclipse.jetty.servlet.ServletHolder;
|
||||||
|
import org.eclipse.jetty.servlets.GzipFilter;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public class BrokerJettyServerInitializer implements JettyServerInitializer
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void initialize(Server server, Injector injector)
|
||||||
|
{
|
||||||
|
final ServletContextHandler resources = new ServletContextHandler(ServletContextHandler.SESSIONS);
|
||||||
|
resources.addServlet(new ServletHolder(new DefaultServlet()), "/druid/v2/datasources/*");
|
||||||
|
resources.addFilter(GuiceFilter.class, "/druid/v2/datasources/*", null);
|
||||||
|
|
||||||
|
final ServletContextHandler queries = new ServletContextHandler(ServletContextHandler.SESSIONS);
|
||||||
|
queries.setResourceBase("/");
|
||||||
|
queries.addServlet(new ServletHolder(injector.getInstance(QueryServlet.class)), "/druid/v2/*");
|
||||||
|
|
||||||
|
final ServletContextHandler root = new ServletContextHandler(ServletContextHandler.SESSIONS);
|
||||||
|
root.addServlet(new ServletHolder(new DefaultServlet()), "/*");
|
||||||
|
root.addFilter(GzipFilter.class, "/*", null);
|
||||||
|
root.addFilter(GuiceFilter.class, "/*", null);
|
||||||
|
|
||||||
|
final HandlerList handlerList = new HandlerList();
|
||||||
|
handlerList.setHandlers(new Handler[]{resources, queries, root, new DefaultHandler()});
|
||||||
|
server.setHandler(handlerList);
|
||||||
|
}
|
||||||
|
}
|
|
@ -81,7 +81,7 @@ public class CliBroker extends ServerRunnable
|
||||||
JsonConfigProvider.bind(binder, "druid.broker.cache", CacheProvider.class);
|
JsonConfigProvider.bind(binder, "druid.broker.cache", CacheProvider.class);
|
||||||
|
|
||||||
binder.bind(QuerySegmentWalker.class).to(ClientQuerySegmentWalker.class).in(LazySingleton.class);
|
binder.bind(QuerySegmentWalker.class).to(ClientQuerySegmentWalker.class).in(LazySingleton.class);
|
||||||
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class).in(LazySingleton.class);
|
binder.bind(JettyServerInitializer.class).to(BrokerJettyServerInitializer.class).in(LazySingleton.class);
|
||||||
Jerseys.addResource(binder, ClientInfoResource.class);
|
Jerseys.addResource(binder, ClientInfoResource.class);
|
||||||
|
|
||||||
DiscoveryModule.register(binder, Self.class);
|
DiscoveryModule.register(binder, Self.class);
|
||||||
|
|
|
@ -0,0 +1,137 @@
|
||||||
|
/*
|
||||||
|
* Druid - a distributed column store.
|
||||||
|
* Copyright (C) 2012, 2013 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;
|
||||||
|
|
||||||
|
import com.google.api.client.repackaged.com.google.common.base.Throwables;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.io.CharStreams;
|
||||||
|
import com.google.common.io.InputSupplier;
|
||||||
|
import com.google.inject.Binder;
|
||||||
|
import com.google.inject.Injector;
|
||||||
|
import com.google.inject.Module;
|
||||||
|
import com.google.inject.Provides;
|
||||||
|
import com.metamx.common.lifecycle.Lifecycle;
|
||||||
|
import com.metamx.common.logger.Logger;
|
||||||
|
import io.airlift.command.Arguments;
|
||||||
|
import io.airlift.command.Command;
|
||||||
|
import io.druid.guice.LazySingleton;
|
||||||
|
import io.druid.indexer.HadoopDruidIndexerConfig;
|
||||||
|
import io.druid.indexer.HadoopDruidIndexerJob;
|
||||||
|
import org.apache.hadoop.conf.Configuration;
|
||||||
|
import org.apache.hadoop.fs.FileSystem;
|
||||||
|
import org.apache.hadoop.fs.Path;
|
||||||
|
|
||||||
|
import java.io.File;
|
||||||
|
import java.io.IOException;
|
||||||
|
import java.io.InputStreamReader;
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
@Command(
|
||||||
|
name = "hadoop",
|
||||||
|
description = "Runs the batch Hadoop Druid Indexer, see https://github.com/metamx/druid/wiki/Batch-ingestion for a description."
|
||||||
|
)
|
||||||
|
public class CliHadoopIndexer extends GuiceRunnable
|
||||||
|
{
|
||||||
|
@Arguments(description = "A JSON object or the path to a file that contains a JSON object", required = true)
|
||||||
|
private String argumentSpec;
|
||||||
|
|
||||||
|
private static final Logger log = new Logger(CliHadoopIndexer.class);
|
||||||
|
|
||||||
|
public CliHadoopIndexer()
|
||||||
|
{
|
||||||
|
super(log);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
protected List<Object> getModules()
|
||||||
|
{
|
||||||
|
return ImmutableList.<Object>of(
|
||||||
|
new Module()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void configure(Binder binder)
|
||||||
|
{
|
||||||
|
binder.bind(HadoopDruidIndexerJob.class).in(LazySingleton.class);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Provides
|
||||||
|
@LazySingleton
|
||||||
|
public HadoopDruidIndexerConfig getHadoopDruidIndexerConfig()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
if (argumentSpec.startsWith("{")) {
|
||||||
|
return HadoopDruidIndexerConfig.fromString(argumentSpec);
|
||||||
|
} else if (argumentSpec.startsWith("s3://")) {
|
||||||
|
final Path s3nPath = new Path(String.format("s3n://%s", argumentSpec.substring("s3://".length())));
|
||||||
|
final FileSystem fs = s3nPath.getFileSystem(new Configuration());
|
||||||
|
|
||||||
|
String configString = CharStreams.toString(
|
||||||
|
new InputSupplier<InputStreamReader>()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public InputStreamReader getInput() throws IOException
|
||||||
|
{
|
||||||
|
return new InputStreamReader(fs.open(s3nPath));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
|
||||||
|
return HadoopDruidIndexerConfig.fromString(configString);
|
||||||
|
} else {
|
||||||
|
return HadoopDruidIndexerConfig.fromFile(new File(argumentSpec));
|
||||||
|
}
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public void run()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
Injector injector = makeInjector();
|
||||||
|
final HadoopDruidIndexerJob job = injector.getInstance(HadoopDruidIndexerJob.class);
|
||||||
|
|
||||||
|
Lifecycle lifecycle = initLifecycle(injector);
|
||||||
|
|
||||||
|
job.run();
|
||||||
|
|
||||||
|
try {
|
||||||
|
lifecycle.stop();
|
||||||
|
}
|
||||||
|
catch (Throwable t) {
|
||||||
|
log.error(t, "Error when stopping. Failing.");
|
||||||
|
System.exit(1);
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
}
|
|
@ -57,6 +57,7 @@ import io.druid.indexing.coordinator.TaskRunnerFactory;
|
||||||
import io.druid.indexing.coordinator.TaskStorage;
|
import io.druid.indexing.coordinator.TaskStorage;
|
||||||
import io.druid.indexing.coordinator.TaskStorageQueryAdapter;
|
import io.druid.indexing.coordinator.TaskStorageQueryAdapter;
|
||||||
import io.druid.indexing.coordinator.http.IndexerCoordinatorResource;
|
import io.druid.indexing.coordinator.http.IndexerCoordinatorResource;
|
||||||
|
import io.druid.indexing.coordinator.http.OldIndexerCoordinatorResource;
|
||||||
import io.druid.indexing.coordinator.http.OverlordRedirectInfo;
|
import io.druid.indexing.coordinator.http.OverlordRedirectInfo;
|
||||||
import io.druid.indexing.coordinator.scaling.AutoScalingStrategy;
|
import io.druid.indexing.coordinator.scaling.AutoScalingStrategy;
|
||||||
import io.druid.indexing.coordinator.scaling.EC2AutoScalingStrategy;
|
import io.druid.indexing.coordinator.scaling.EC2AutoScalingStrategy;
|
||||||
|
@ -139,6 +140,7 @@ public class CliOverlord extends ServerRunnable
|
||||||
|
|
||||||
binder.bind(JettyServerInitializer.class).toInstance(new OverlordJettyServerInitializer());
|
binder.bind(JettyServerInitializer.class).toInstance(new OverlordJettyServerInitializer());
|
||||||
Jerseys.addResource(binder, IndexerCoordinatorResource.class);
|
Jerseys.addResource(binder, IndexerCoordinatorResource.class);
|
||||||
|
Jerseys.addResource(binder, OldIndexerCoordinatorResource.class);
|
||||||
|
|
||||||
LifecycleModule.register(binder, Server.class);
|
LifecycleModule.register(binder, Server.class);
|
||||||
}
|
}
|
||||||
|
|
|
@ -22,7 +22,6 @@ package io.druid.cli;
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
import com.google.inject.Inject;
|
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
import com.google.inject.Key;
|
import com.google.inject.Key;
|
||||||
import com.google.inject.Module;
|
import com.google.inject.Module;
|
||||||
|
@ -35,6 +34,7 @@ import io.airlift.command.Option;
|
||||||
import io.druid.guice.Jerseys;
|
import io.druid.guice.Jerseys;
|
||||||
import io.druid.guice.JsonConfigProvider;
|
import io.druid.guice.JsonConfigProvider;
|
||||||
import io.druid.guice.LazySingleton;
|
import io.druid.guice.LazySingleton;
|
||||||
|
import io.druid.guice.LifecycleModule;
|
||||||
import io.druid.guice.ManageLifecycle;
|
import io.druid.guice.ManageLifecycle;
|
||||||
import io.druid.guice.NodeTypeConfig;
|
import io.druid.guice.NodeTypeConfig;
|
||||||
import io.druid.guice.PolyBind;
|
import io.druid.guice.PolyBind;
|
||||||
|
@ -52,13 +52,13 @@ import io.druid.indexing.coordinator.ThreadPoolTaskRunner;
|
||||||
import io.druid.indexing.worker.executor.ChatHandlerResource;
|
import io.druid.indexing.worker.executor.ChatHandlerResource;
|
||||||
import io.druid.indexing.worker.executor.ExecutorLifecycle;
|
import io.druid.indexing.worker.executor.ExecutorLifecycle;
|
||||||
import io.druid.indexing.worker.executor.ExecutorLifecycleConfig;
|
import io.druid.indexing.worker.executor.ExecutorLifecycleConfig;
|
||||||
import io.druid.initialization.LogLevelAdjuster;
|
|
||||||
import io.druid.query.QuerySegmentWalker;
|
import io.druid.query.QuerySegmentWalker;
|
||||||
import io.druid.segment.loading.DataSegmentKiller;
|
import io.druid.segment.loading.DataSegmentKiller;
|
||||||
import io.druid.segment.loading.S3DataSegmentKiller;
|
import io.druid.segment.loading.S3DataSegmentKiller;
|
||||||
import io.druid.segment.loading.SegmentLoaderConfig;
|
import io.druid.segment.loading.SegmentLoaderConfig;
|
||||||
import io.druid.segment.loading.StorageLocationConfig;
|
import io.druid.segment.loading.StorageLocationConfig;
|
||||||
import io.druid.server.initialization.JettyServerInitializer;
|
import io.druid.server.initialization.JettyServerInitializer;
|
||||||
|
import org.eclipse.jetty.server.Server;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
|
@ -71,7 +71,7 @@ import java.util.List;
|
||||||
description = "Runs a Peon, this is an individual forked \"task\" used as part of the indexing service. "
|
description = "Runs a Peon, this is an individual forked \"task\" used as part of the indexing service. "
|
||||||
+ "This should rarely, if ever, be used directly."
|
+ "This should rarely, if ever, be used directly."
|
||||||
)
|
)
|
||||||
public class CliPeon implements Runnable
|
public class CliPeon extends GuiceRunnable
|
||||||
{
|
{
|
||||||
@Arguments(description = "task.json status.json", required = true)
|
@Arguments(description = "task.json status.json", required = true)
|
||||||
public List<String> taskAndStatusFile;
|
public List<String> taskAndStatusFile;
|
||||||
|
@ -79,74 +79,71 @@ public class CliPeon implements Runnable
|
||||||
@Option(name = "--nodeType", title = "nodeType", description = "Set the node type to expose on ZK")
|
@Option(name = "--nodeType", title = "nodeType", description = "Set the node type to expose on ZK")
|
||||||
public String nodeType = "indexer-executor";
|
public String nodeType = "indexer-executor";
|
||||||
|
|
||||||
private Injector injector;
|
|
||||||
|
|
||||||
@Inject
|
|
||||||
public void configure(Injector injector)
|
|
||||||
{
|
|
||||||
this.injector = injector;
|
|
||||||
}
|
|
||||||
|
|
||||||
private static final Logger log = new Logger(CliPeon.class);
|
private static final Logger log = new Logger(CliPeon.class);
|
||||||
|
|
||||||
protected Injector getInjector()
|
public CliPeon()
|
||||||
{
|
{
|
||||||
return Initialization.makeInjectorWithModules(
|
super(log);
|
||||||
injector,
|
}
|
||||||
ImmutableList.<Object>of(
|
|
||||||
new Module()
|
|
||||||
{
|
|
||||||
@Override
|
|
||||||
public void configure(Binder binder)
|
|
||||||
{
|
|
||||||
PolyBind.createChoice(
|
|
||||||
binder,
|
|
||||||
"druid.indexer.task.chathandler.type",
|
|
||||||
Key.get(ChatHandlerProvider.class),
|
|
||||||
Key.get(NoopChatHandlerProvider.class)
|
|
||||||
);
|
|
||||||
final MapBinder<String, ChatHandlerProvider> handlerProviderBinder = PolyBind.optionBinder(
|
|
||||||
binder, Key.get(ChatHandlerProvider.class)
|
|
||||||
);
|
|
||||||
handlerProviderBinder.addBinding("curator").to(EventReceivingChatHandlerProvider.class);
|
|
||||||
handlerProviderBinder.addBinding("noop").to(NoopChatHandlerProvider.class);
|
|
||||||
|
|
||||||
binder.bind(TaskToolboxFactory.class).in(LazySingleton.class);
|
@Override
|
||||||
|
protected List<Object> getModules()
|
||||||
|
{
|
||||||
|
return ImmutableList.<Object>of(
|
||||||
|
new Module()
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public void configure(Binder binder)
|
||||||
|
{
|
||||||
|
PolyBind.createChoice(
|
||||||
|
binder,
|
||||||
|
"druid.indexer.task.chathandler.type",
|
||||||
|
Key.get(ChatHandlerProvider.class),
|
||||||
|
Key.get(NoopChatHandlerProvider.class)
|
||||||
|
);
|
||||||
|
final MapBinder<String, ChatHandlerProvider> handlerProviderBinder = PolyBind.optionBinder(
|
||||||
|
binder, Key.get(ChatHandlerProvider.class)
|
||||||
|
);
|
||||||
|
handlerProviderBinder.addBinding("curator").to(EventReceivingChatHandlerProvider.class);
|
||||||
|
handlerProviderBinder.addBinding("noop").to(NoopChatHandlerProvider.class);
|
||||||
|
|
||||||
JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class);
|
binder.bind(TaskToolboxFactory.class).in(LazySingleton.class);
|
||||||
JsonConfigProvider.bind(binder, "druid.worker.taskActionClient.retry", RetryPolicyConfig.class);
|
|
||||||
|
|
||||||
binder.bind(TaskActionClientFactory.class)
|
JsonConfigProvider.bind(binder, "druid.indexer.task", TaskConfig.class);
|
||||||
.to(RemoteTaskActionClientFactory.class)
|
JsonConfigProvider.bind(binder, "druid.worker.taskActionClient.retry", RetryPolicyConfig.class);
|
||||||
.in(LazySingleton.class);
|
|
||||||
binder.bind(RetryPolicyFactory.class).in(LazySingleton.class);
|
|
||||||
|
|
||||||
binder.bind(DataSegmentKiller.class).to(S3DataSegmentKiller.class).in(LazySingleton.class);
|
binder.bind(TaskActionClientFactory.class)
|
||||||
|
.to(RemoteTaskActionClientFactory.class)
|
||||||
|
.in(LazySingleton.class);
|
||||||
|
binder.bind(RetryPolicyFactory.class).in(LazySingleton.class);
|
||||||
|
|
||||||
binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class);
|
binder.bind(DataSegmentKiller.class).to(S3DataSegmentKiller.class).in(LazySingleton.class);
|
||||||
binder.bind(ExecutorLifecycleConfig.class).toInstance(
|
|
||||||
new ExecutorLifecycleConfig()
|
|
||||||
.setTaskFile(new File(taskAndStatusFile.get(0)))
|
|
||||||
.setStatusFile(new File(taskAndStatusFile.get(1)))
|
|
||||||
);
|
|
||||||
|
|
||||||
binder.bind(TaskRunner.class).to(ThreadPoolTaskRunner.class);
|
binder.bind(ExecutorLifecycle.class).in(ManageLifecycle.class);
|
||||||
binder.bind(QuerySegmentWalker.class).to(ThreadPoolTaskRunner.class);
|
binder.bind(ExecutorLifecycleConfig.class).toInstance(
|
||||||
binder.bind(ThreadPoolTaskRunner.class).in(ManageLifecycle.class);
|
new ExecutorLifecycleConfig()
|
||||||
|
.setTaskFile(new File(taskAndStatusFile.get(0)))
|
||||||
|
.setStatusFile(new File(taskAndStatusFile.get(1)))
|
||||||
|
);
|
||||||
|
|
||||||
// Override the default SegmentLoaderConfig because we don't actually care about the
|
binder.bind(TaskRunner.class).to(ThreadPoolTaskRunner.class);
|
||||||
// configuration based locations. This will override them anyway. This is also stopping
|
binder.bind(QuerySegmentWalker.class).to(ThreadPoolTaskRunner.class);
|
||||||
// configuration of other parameters, but I don't think that's actually a problem.
|
binder.bind(ThreadPoolTaskRunner.class).in(ManageLifecycle.class);
|
||||||
// Note, if that is actually not a problem, then that probably means we have the wrong abstraction.
|
|
||||||
binder.bind(SegmentLoaderConfig.class)
|
|
||||||
.toInstance(new SegmentLoaderConfig().withLocations(Arrays.<StorageLocationConfig>asList()));
|
|
||||||
|
|
||||||
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class);
|
// Override the default SegmentLoaderConfig because we don't actually care about the
|
||||||
Jerseys.addResource(binder, ChatHandlerResource.class);
|
// configuration based locations. This will override them anyway. This is also stopping
|
||||||
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(nodeType));
|
// configuration of other parameters, but I don't think that's actually a problem.
|
||||||
}
|
// Note, if that is actually not a problem, then that probably means we have the wrong abstraction.
|
||||||
}
|
binder.bind(SegmentLoaderConfig.class)
|
||||||
)
|
.toInstance(new SegmentLoaderConfig().withLocations(Arrays.<StorageLocationConfig>asList()));
|
||||||
|
|
||||||
|
binder.bind(JettyServerInitializer.class).to(QueryJettyServerInitializer.class);
|
||||||
|
Jerseys.addResource(binder, ChatHandlerResource.class);
|
||||||
|
binder.bind(NodeTypeConfig.class).toInstance(new NodeTypeConfig(nodeType));
|
||||||
|
|
||||||
|
LifecycleModule.register(binder, Server.class);
|
||||||
|
}
|
||||||
|
}
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -154,13 +151,11 @@ public class CliPeon implements Runnable
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
try {
|
try {
|
||||||
LogLevelAdjuster.register();
|
Injector injector = makeInjector();
|
||||||
|
|
||||||
final Injector injector = getInjector();
|
|
||||||
final Lifecycle lifecycle = injector.getInstance(Lifecycle.class);
|
|
||||||
|
|
||||||
try {
|
try {
|
||||||
lifecycle.start();
|
Lifecycle lifecycle = initLifecycle(injector);
|
||||||
|
|
||||||
injector.getInstance(ExecutorLifecycle.class).join();
|
injector.getInstance(ExecutorLifecycle.class).join();
|
||||||
lifecycle.stop();
|
lifecycle.stop();
|
||||||
}
|
}
|
||||||
|
|
|
@ -0,0 +1,65 @@
|
||||||
|
package io.druid.cli;
|
||||||
|
|
||||||
|
import com.google.common.base.Throwables;
|
||||||
|
import com.google.inject.Inject;
|
||||||
|
import com.google.inject.Injector;
|
||||||
|
import com.metamx.common.lifecycle.Lifecycle;
|
||||||
|
import com.metamx.common.logger.Logger;
|
||||||
|
import io.druid.initialization.LogLevelAdjuster;
|
||||||
|
|
||||||
|
import java.util.List;
|
||||||
|
|
||||||
|
/**
|
||||||
|
*/
|
||||||
|
public abstract class GuiceRunnable implements Runnable
|
||||||
|
{
|
||||||
|
private final Logger log;
|
||||||
|
|
||||||
|
private Injector baseInjector;
|
||||||
|
|
||||||
|
public GuiceRunnable(Logger log)
|
||||||
|
{
|
||||||
|
this.log = log;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Inject
|
||||||
|
public void configure(Injector injector)
|
||||||
|
{
|
||||||
|
this.baseInjector = injector;
|
||||||
|
}
|
||||||
|
|
||||||
|
protected abstract List<Object> getModules();
|
||||||
|
|
||||||
|
public Injector makeInjector()
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
return Initialization.makeInjectorWithModules(
|
||||||
|
baseInjector, getModules()
|
||||||
|
);
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
|
public Lifecycle initLifecycle(Injector injector)
|
||||||
|
{
|
||||||
|
try {
|
||||||
|
LogLevelAdjuster.register();
|
||||||
|
final Lifecycle lifecycle = injector.getInstance(Lifecycle.class);
|
||||||
|
|
||||||
|
try {
|
||||||
|
lifecycle.start();
|
||||||
|
}
|
||||||
|
catch (Throwable t) {
|
||||||
|
log.error(t, "Error when starting up. Failing.");
|
||||||
|
System.exit(1);
|
||||||
|
}
|
||||||
|
|
||||||
|
return lifecycle;
|
||||||
|
}
|
||||||
|
catch (Exception e) {
|
||||||
|
throw Throwables.propagate(e);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
}
|
|
@ -97,6 +97,13 @@ public class
|
||||||
final TeslaAether aether = getAetherClient(config);
|
final TeslaAether aether = getAetherClient(config);
|
||||||
List<T> retVal = Lists.newArrayList();
|
List<T> retVal = Lists.newArrayList();
|
||||||
|
|
||||||
|
if (config.searchCurrentClassloader()) {
|
||||||
|
for (T module : ServiceLoader.load(clazz, Initialization.class.getClassLoader())) {
|
||||||
|
log.info("Adding local module[%s]", module.getClass());
|
||||||
|
retVal.add(module);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
for (String coordinate : config.getCoordinates()) {
|
for (String coordinate : config.getCoordinates()) {
|
||||||
log.info("Loading extension[%s]", coordinate);
|
log.info("Loading extension[%s]", coordinate);
|
||||||
try {
|
try {
|
||||||
|
@ -139,14 +146,13 @@ public class
|
||||||
for (Artifact artifact : artifacts) {
|
for (Artifact artifact : artifacts) {
|
||||||
if (!exclusions.contains(artifact.getGroupId())) {
|
if (!exclusions.contains(artifact.getGroupId())) {
|
||||||
urls.add(artifact.getFile().toURI().toURL());
|
urls.add(artifact.getFile().toURI().toURL());
|
||||||
}
|
} else {
|
||||||
else {
|
|
||||||
log.error("Skipped Artifact[%s]", artifact);
|
log.error("Skipped Artifact[%s]", artifact);
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
for (URL url : urls) {
|
for (URL url : urls) {
|
||||||
log.error("Added URL[%s]", url);
|
log.info("Added URL[%s]", url);
|
||||||
}
|
}
|
||||||
|
|
||||||
loader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader());
|
loader = new URLClassLoader(urls.toArray(new URL[urls.size()]), Initialization.class.getClassLoader());
|
||||||
|
@ -243,7 +249,8 @@ public class
|
||||||
private final ObjectMapper smileMapper;
|
private final ObjectMapper smileMapper;
|
||||||
private final List<Module> modules;
|
private final List<Module> modules;
|
||||||
|
|
||||||
public ModuleList(Injector baseInjector) {
|
public ModuleList(Injector baseInjector)
|
||||||
|
{
|
||||||
this.baseInjector = baseInjector;
|
this.baseInjector = baseInjector;
|
||||||
this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class));
|
this.jsonMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Json.class));
|
||||||
this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class));
|
this.smileMapper = baseInjector.getInstance(Key.get(ObjectMapper.class, Smile.class));
|
||||||
|
@ -260,24 +267,19 @@ public class
|
||||||
if (input instanceof DruidModule) {
|
if (input instanceof DruidModule) {
|
||||||
baseInjector.injectMembers(input);
|
baseInjector.injectMembers(input);
|
||||||
modules.add(registerJacksonModules(((DruidModule) input)));
|
modules.add(registerJacksonModules(((DruidModule) input)));
|
||||||
}
|
} else if (input instanceof Module) {
|
||||||
else if (input instanceof Module) {
|
|
||||||
baseInjector.injectMembers(input);
|
baseInjector.injectMembers(input);
|
||||||
modules.add((Module) input);
|
modules.add((Module) input);
|
||||||
}
|
} else if (input instanceof Class) {
|
||||||
else if (input instanceof Class) {
|
|
||||||
if (DruidModule.class.isAssignableFrom((Class) input)) {
|
if (DruidModule.class.isAssignableFrom((Class) input)) {
|
||||||
modules.add(registerJacksonModules(baseInjector.getInstance((Class<? extends DruidModule>) input)));
|
modules.add(registerJacksonModules(baseInjector.getInstance((Class<? extends DruidModule>) input)));
|
||||||
}
|
} else if (Module.class.isAssignableFrom((Class) input)) {
|
||||||
else if (Module.class.isAssignableFrom((Class) input)) {
|
|
||||||
modules.add(baseInjector.getInstance((Class<? extends Module>) input));
|
modules.add(baseInjector.getInstance((Class<? extends Module>) input));
|
||||||
return;
|
return;
|
||||||
}
|
} else {
|
||||||
else {
|
|
||||||
throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class);
|
throw new ISE("Class[%s] does not implement %s", input.getClass(), Module.class);
|
||||||
}
|
}
|
||||||
}
|
} else {
|
||||||
else {
|
|
||||||
throw new ISE("Unknown module type[%s]", input.getClass());
|
throw new ISE("Unknown module type[%s]", input.getClass());
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -68,6 +68,11 @@ public class Main
|
||||||
.withDefaultCommand(Help.class)
|
.withDefaultCommand(Help.class)
|
||||||
.withCommands(ConvertProperties.class);
|
.withCommands(ConvertProperties.class);
|
||||||
|
|
||||||
|
builder.withGroup("index")
|
||||||
|
.withDescription("Run indexing for druid")
|
||||||
|
.withDefaultCommand(Help.class)
|
||||||
|
.withCommands(CliHadoopIndexer.class);
|
||||||
|
|
||||||
builder.withGroup("internal")
|
builder.withGroup("internal")
|
||||||
.withDescription("Processes that Druid runs \"internally\", you should rarely use these directly")
|
.withDescription("Processes that Druid runs \"internally\", you should rarely use these directly")
|
||||||
.withDefaultCommand(Help.class)
|
.withDefaultCommand(Help.class)
|
||||||
|
|
|
@ -20,54 +20,26 @@
|
||||||
package io.druid.cli;
|
package io.druid.cli;
|
||||||
|
|
||||||
import com.google.common.base.Throwables;
|
import com.google.common.base.Throwables;
|
||||||
import com.google.inject.Inject;
|
|
||||||
import com.google.inject.Injector;
|
import com.google.inject.Injector;
|
||||||
import com.metamx.common.lifecycle.Lifecycle;
|
import com.metamx.common.lifecycle.Lifecycle;
|
||||||
import com.metamx.common.logger.Logger;
|
import com.metamx.common.logger.Logger;
|
||||||
import io.druid.initialization.LogLevelAdjuster;
|
|
||||||
|
|
||||||
import java.util.List;
|
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public abstract class ServerRunnable implements Runnable
|
public abstract class ServerRunnable extends GuiceRunnable
|
||||||
{
|
{
|
||||||
private final Logger log;
|
|
||||||
|
|
||||||
private Injector baseInjector;
|
|
||||||
|
|
||||||
public ServerRunnable(Logger log)
|
public ServerRunnable(Logger log)
|
||||||
{
|
{
|
||||||
this.log = log;
|
super(log);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Inject
|
|
||||||
public void configure(Injector injector)
|
|
||||||
{
|
|
||||||
this.baseInjector = injector;
|
|
||||||
}
|
|
||||||
|
|
||||||
protected abstract List<Object> getModules();
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public void run()
|
public void run()
|
||||||
{
|
{
|
||||||
|
final Injector injector = makeInjector();
|
||||||
|
final Lifecycle lifecycle = initLifecycle(injector);
|
||||||
|
|
||||||
try {
|
try {
|
||||||
LogLevelAdjuster.register();
|
|
||||||
|
|
||||||
final Injector injector = Initialization.makeInjectorWithModules(
|
|
||||||
baseInjector, getModules()
|
|
||||||
);
|
|
||||||
final Lifecycle lifecycle = injector.getInstance(Lifecycle.class);
|
|
||||||
|
|
||||||
try {
|
|
||||||
lifecycle.start();
|
|
||||||
}
|
|
||||||
catch (Throwable t) {
|
|
||||||
log.error(t, "Error when starting up. Failing.");
|
|
||||||
System.exit(1);
|
|
||||||
}
|
|
||||||
|
|
||||||
lifecycle.join();
|
lifecycle.join();
|
||||||
}
|
}
|
||||||
catch (Exception e) {
|
catch (Exception e) {
|
||||||
|
|
|
@ -91,7 +91,7 @@ public class ConvertProperties implements Runnable
|
||||||
new Rename("druid.indexer.maxPendingTaskDuration", "druid.indexer.autoscale.pendingTaskTimeout"),
|
new Rename("druid.indexer.maxPendingTaskDuration", "druid.indexer.autoscale.pendingTaskTimeout"),
|
||||||
new Rename("druid.indexer.worker.version", "druid.indexer.autoscale.workerVersion"),
|
new Rename("druid.indexer.worker.version", "druid.indexer.autoscale.workerVersion"),
|
||||||
new Rename("druid.indexer.worker.port", "druid.indexer.autoscale.workerPort"),
|
new Rename("druid.indexer.worker.port", "druid.indexer.autoscale.workerPort"),
|
||||||
new Rename("druid.worker.masterService", "druid.worker.overlordService"),
|
new Rename("druid.worker.masterService", "druid.selectors.indexing.serviceName"),
|
||||||
new ChatHandlerConverter(),
|
new ChatHandlerConverter(),
|
||||||
new Rename("druid.indexer.baseDir", "druid.indexer.task.baseDir"),
|
new Rename("druid.indexer.baseDir", "druid.indexer.task.baseDir"),
|
||||||
new Rename("druid.indexer.taskDir", "druid.indexer.task.taskDir"),
|
new Rename("druid.indexer.taskDir", "druid.indexer.task.taskDir"),
|
||||||
|
@ -100,6 +100,7 @@ public class ConvertProperties implements Runnable
|
||||||
new Rename("druid.worker.taskActionClient.retry.minWaitMillis", "druid.worker.taskActionClient.retry.minWait"),
|
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.worker.taskActionClient.retry.maxWaitMillis", "druid.worker.taskActionClient.retry.maxWait"),
|
||||||
new Rename("druid.master.merger.service", "druid.selectors.indexing.serviceName"),
|
new Rename("druid.master.merger.service", "druid.selectors.indexing.serviceName"),
|
||||||
|
new Rename("druid.master.merger.on", "druid.master.merge.on"),
|
||||||
new DataSegmentPusherDefaultConverter(),
|
new DataSegmentPusherDefaultConverter(),
|
||||||
new Rename("druid.pusher.hdfs.storageDirectory", "druid.pusher.storageDirectory"),
|
new Rename("druid.pusher.hdfs.storageDirectory", "druid.pusher.storageDirectory"),
|
||||||
new Rename("druid.pusher.cassandra.host", "druid.pusher.host"),
|
new Rename("druid.pusher.cassandra.host", "druid.pusher.host"),
|
||||||
|
|
|
@ -23,11 +23,15 @@ import com.fasterxml.jackson.databind.Module;
|
||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||||
import com.google.inject.Binder;
|
import com.google.inject.Binder;
|
||||||
|
import com.google.inject.Key;
|
||||||
import com.google.inject.TypeLiteral;
|
import com.google.inject.TypeLiteral;
|
||||||
|
import com.google.inject.multibindings.MapBinder;
|
||||||
import io.druid.cli.QueryJettyServerInitializer;
|
import io.druid.cli.QueryJettyServerInitializer;
|
||||||
import io.druid.initialization.DruidModule;
|
import io.druid.initialization.DruidModule;
|
||||||
import io.druid.query.QuerySegmentWalker;
|
import io.druid.query.QuerySegmentWalker;
|
||||||
|
import io.druid.segment.realtime.DbSegmentPublisher;
|
||||||
import io.druid.segment.realtime.FireDepartment;
|
import io.druid.segment.realtime.FireDepartment;
|
||||||
|
import io.druid.segment.realtime.NoopSegmentPublisher;
|
||||||
import io.druid.segment.realtime.RealtimeManager;
|
import io.druid.segment.realtime.RealtimeManager;
|
||||||
import io.druid.segment.realtime.SegmentPublisher;
|
import io.druid.segment.realtime.SegmentPublisher;
|
||||||
import io.druid.segment.realtime.firehose.KafkaFirehoseFactory;
|
import io.druid.segment.realtime.firehose.KafkaFirehoseFactory;
|
||||||
|
@ -38,17 +42,28 @@ import java.util.Arrays;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*/
|
*/
|
||||||
public class RealtimeModule implements DruidModule
|
public class RealtimeModule implements DruidModule
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public void configure(Binder binder)
|
public void configure(Binder binder)
|
||||||
{
|
{
|
||||||
JsonConfigProvider.bind(binder, "druid.publish", SegmentPublisherProvider.class);
|
PolyBind.createChoice(
|
||||||
binder.bind(SegmentPublisher.class).toProvider(SegmentPublisherProvider.class);
|
binder,
|
||||||
|
"druid.publish.type",
|
||||||
|
Key.get(SegmentPublisher.class),
|
||||||
|
Key.get(NoopSegmentPublisher.class)
|
||||||
|
);
|
||||||
|
final MapBinder<String, SegmentPublisher> publisherBinder = PolyBind.optionBinder(binder, Key.get(SegmentPublisher.class));
|
||||||
|
publisherBinder.addBinding("db").to(DbSegmentPublisher.class);
|
||||||
|
binder.bind(DbSegmentPublisher.class).in(LazySingleton.class);
|
||||||
|
|
||||||
JsonConfigProvider.bind(binder, "druid.realtime", RealtimeManagerConfig.class);
|
JsonConfigProvider.bind(binder, "druid.realtime", RealtimeManagerConfig.class);
|
||||||
binder.bind(new TypeLiteral<List<FireDepartment>>(){})
|
binder.bind(
|
||||||
|
new TypeLiteral<List<FireDepartment>>()
|
||||||
|
{
|
||||||
|
}
|
||||||
|
)
|
||||||
.toProvider(FireDepartmentsProvider.class)
|
.toProvider(FireDepartmentsProvider.class)
|
||||||
.in(LazySingleton.class);
|
.in(LazySingleton.class);
|
||||||
|
|
||||||
|
|
Loading…
Reference in New Issue