mirror of https://github.com/apache/druid.git
more fixes
This commit is contained in:
parent
f77fbfcfe7
commit
6b573c76f1
2
build.sh
2
build.sh
|
@ -30,4 +30,4 @@ echo "For examples, see: "
|
|||
echo " "
|
||||
ls -1 examples/*/*sh
|
||||
echo " "
|
||||
echo "See also http://druid.io/docs/0.6.1/Home.html"
|
||||
echo "See also http://druid.io/docs/0.6.2"
|
||||
|
|
|
@ -3,7 +3,7 @@ layout: doc_page
|
|||
---
|
||||
# Booting a Single Node Cluster #
|
||||
|
||||
[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.1-bin.tar.gz).
|
||||
[Loading Your Data](Tutorial%3A-Loading-Your-Data-Part-2.html) and [All About Queries](Tutorial%3A-All-About-Queries.html) contain recipes to boot a small druid cluster on localhost. Here we will boot a small cluster on EC2. You can checkout the code, or download a tarball from [here](http://static.druid.io/artifacts/druid-services-0.6.2-bin.tar.gz).
|
||||
|
||||
The [ec2 run script](https://github.com/metamx/druid/blob/master/examples/bin/run_ec2.sh), run_ec2.sh, is located at 'examples/bin' if you have checked out the code, or at the root of the project if you've downloaded a tarball. The scripts rely on the [Amazon EC2 API Tools](http://aws.amazon.com/developertools/351), and you will need to set three environment variables:
|
||||
|
||||
|
|
|
@ -15,7 +15,7 @@ There are three JVM parameters that we set on all of our processes:
|
|||
Modules
|
||||
=======
|
||||
|
||||
As of Druid v0.6.1, most core Druid functionality has been compartmentalized into modules. There are a set of default modules that may apply to any node type, and there are specific modules for the different node types. Default modules are __lazily instantiated__. Each module has its own set of configuration. This page will describe the configuration of the default modules.
|
||||
As of Druid v0.6.2, most core Druid functionality has been compartmentalized into modules. There are a set of default modules that may apply to any node type, and there are specific modules for the different node types. Default modules are __lazily instantiated__. Each module has its own set of configuration. This page will describe the configuration of the default modules.
|
||||
|
||||
Configuration of the various modules is done via Java properties. These can either be provided as `-D` system properties on the java command line or they can be passed in via a file called `runtime.properties` that exists on the classpath.
|
||||
|
||||
|
@ -27,7 +27,7 @@ The Druid servers emit various metrics and alerts via something we call an Emitt
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.emitter`|Appending "logging" or "http" to this config will instantialize one of the emitter modules.|logging|
|
||||
|`druid.emitter`|Setting this value to either "logging" or "http" will instantialize one of the emitter modules.|logging|
|
||||
|
||||
|
||||
#### Logging Emitter Module
|
||||
|
|
|
@ -48,7 +48,6 @@ The coordinator module uses several of the default modules in [Configuration](Co
|
|||
|--------|-----------|-------|
|
||||
|`druid.coordinator.period`|The run period for the coordinator. The coordinator’s operates by maintaining the current state of the world in memory and periodically looking at the set of segments available and segments being served to make decisions about whether any changes need to be made to the data topology. This property sets the delay between each of these runs.|PT60S|
|
||||
|`druid.coordinator.period.indexingPeriod`|How often to send indexing tasks to the indexing service. Only applies if merge or conversion is turned on.|PT1800S (30 mins)|
|
||||
|`druid.coordinator.removedSegmentLifetime`|When a node disappears, the coordinator can provide a grace period for how long it waits before deciding that the node really isn’t going to come back and it really should declare that all segments from that node are no longer available. This sets that grace period in number of runs of the coordinator.|1|
|
||||
|`druid.coordinator.startDelay`|The operation of the Coordinator works on the assumption that it has an up-to-date view of the state of the world when it runs, the current ZK interaction code, however, is written in a way that doesn’t allow the Coordinator to know for a fact that it’s done loading the current state of the world. This delay is a hack to give it enough time to believe that it has all the data.|PT300S|
|
||||
|`druid.coordinator.merge.on`|Boolean flag for whether or not the coordinator should try and merge small segments into a more optimal segment size.|PT300S|
|
||||
|`druid.coordinator.conversion.on`|Boolean flag for converting old segment indexing versions to the latest segment indexing version.|false|
|
||||
|
|
|
@ -6,7 +6,7 @@ A version may be declared as a release candidate if it has been deployed to a si
|
|||
Release Candidate
|
||||
-----------------
|
||||
|
||||
The current release candidate is tagged at version [0.6.1](https://github.com/metamx/druid/tree/druid-0.6.1).
|
||||
The current release candidate is tagged at version [0.6.2](https://github.com/metamx/druid/tree/druid-0.6.2).
|
||||
|
||||
Stable Release
|
||||
--------------
|
||||
|
|
|
@ -19,13 +19,13 @@ Clone Druid and build it:
|
|||
git clone https://github.com/metamx/druid.git druid
|
||||
cd druid
|
||||
git fetch --tags
|
||||
git checkout druid-0.6.1
|
||||
git checkout druid-0.6.2
|
||||
./build.sh
|
||||
```
|
||||
|
||||
### Downloading the DSK (Druid Standalone Kit)
|
||||
|
||||
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.1-bin.tar.gz) a stand-alone tarball and run it:
|
||||
[Download](http://static.druid.io/artifacts/releases/druid-services-0.6.2-bin.tar.gz) a stand-alone tarball and run it:
|
||||
|
||||
``` bash
|
||||
tar -xzf druid-services-0.X.X-bin.tar.gz
|
||||
|
|
|
@ -123,11 +123,11 @@ There are additional configs for autoscaling (if it is enabled):
|
|||
|
||||
|Property|Description|Default|
|
||||
|--------|-----------|-------|
|
||||
|`druid.indexer.autoscale.strategy`|Choices are "noop" or "ec2". Sets the strategy to run when autoscaling is required.|noop|
|
||||
|`druid.indexer.autoscale.doAutoscale`|If set to "true" autoscaling will be enabled.|false|
|
||||
|`druid.indexer.autoscale.provisionPeriod`|How often to check whether or not new middle managers should be added.|PT1M|
|
||||
|`druid.indexer.autoscale.terminatePeriod`|How often to check when middle managers should be removed.|PT1H|
|
||||
|`druid.indexer.autoscale.originTime`|The starting reference timestamp that the terminate period increments upon.|2012-01-01T00:55:00.000Z|
|
||||
|`druid.indexer.autoscale.strategy`|Choices are "noop" or "ec2". Sets the strategy to run when autoscaling is required.|noop|
|
||||
|`druid.indexer.autoscale.workerIdleTimeout`|How long can a worker be idle (not a run task) before it can be considered for termination.|PT10M|
|
||||
|`druid.indexer.autoscale.maxScalingDuration`|How long the overlord will wait around for a middle manager to show up before giving up.|PT15M|
|
||||
|`druid.indexer.autoscale.numEventsToTrack`|The number of autoscaling related events (node creation and termination) to track.|10|
|
||||
|
|
|
@ -187,8 +187,8 @@ Extending the code
|
|||
|
||||
Realtime integration is intended to be extended in two ways:
|
||||
|
||||
1. Connect to data streams from varied systems ([Firehose](https://github.com/metamx/druid/blob/druid-0.6.1/realtime/src/main/java/com/metamx/druid/realtime/firehose/FirehoseFactory.java))
|
||||
2. Adjust the publishing strategy to match your needs ([Plumber](https://github.com/metamx/druid/blob/druid-0.6.1/realtime/src/main/java/com/metamx/druid/realtime/plumber/PlumberSchool.java))
|
||||
1. Connect to data streams from varied systems ([Firehose](https://github.com/metamx/druid/blob/druid-0.6.2/realtime/src/main/java/com/metamx/druid/realtime/firehose/FirehoseFactory.java))
|
||||
2. Adjust the publishing strategy to match your needs ([Plumber](https://github.com/metamx/druid/blob/druid-0.6.2/realtime/src/main/java/com/metamx/druid/realtime/plumber/PlumberSchool.java))
|
||||
|
||||
The expectations are that the former will be very common and something that users of Druid will do on a fairly regular basis. Most users will probably never have to deal with the latter form of customization. Indeed, we hope that all potential use cases can be packaged up as part of Druid proper without requiring proprietary customization.
|
||||
|
||||
|
|
|
@ -47,7 +47,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu
|
|||
|
||||
### Download a Tarball
|
||||
|
||||
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.1-bin.tar.gz). Download this file to a directory of your choosing.
|
||||
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.2-bin.tar.gz). Download this file to a directory of your choosing.
|
||||
|
||||
You can extract the awesomeness within by issuing:
|
||||
|
||||
|
@ -58,7 +58,7 @@ tar -zxvf druid-services-*-bin.tar.gz
|
|||
Not too lost so far right? That's great! If you cd into the directory:
|
||||
|
||||
```
|
||||
cd druid-services-0.6.1
|
||||
cd druid-services-0.6.2
|
||||
```
|
||||
|
||||
You should see a bunch of files:
|
||||
|
|
|
@ -42,7 +42,7 @@ With real-world data, we recommend having a message bus such as [Apache Kafka](h
|
|||
|
||||
#### Setting up Kafka
|
||||
|
||||
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.1/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
|
||||
[KafkaFirehoseFactory](https://github.com/metamx/druid/blob/druid-0.6.2/realtime/src/main/java/com/metamx/druid/realtime/firehose/KafkaFirehoseFactory.java) is how druid communicates with Kafka. Using this [Firehose](Firehose.html) with the right configuration, we can import data into Druid in real-time without writing any code. To load data to a real-time node via Kafka, we'll first need to initialize Zookeeper and Kafka, and then configure and initialize a [Realtime](Realtime.html) node.
|
||||
|
||||
Instructions for booting a Zookeeper and then Kafka cluster are available [here](http://kafka.apache.org/07/quickstart.html).
|
||||
|
||||
|
|
|
@ -11,7 +11,7 @@ In this tutorial, we will set up other types of Druid nodes as well as and exter
|
|||
|
||||
If you followed the first tutorial, you should already have Druid downloaded. If not, let's go back and do that first.
|
||||
|
||||
You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.1-bin.tar.gz)
|
||||
You can download the latest version of druid [here](http://static.druid.io/artifacts/releases/druid-services-0.6.2-bin.tar.gz)
|
||||
|
||||
and untar the contents within by issuing:
|
||||
|
||||
|
@ -147,6 +147,8 @@ druid.port=8081
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.2"]
|
||||
|
||||
# Dummy read only AWS account (used to download example data)
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
|
|
|
@ -37,7 +37,7 @@ There are two ways to setup Druid: download a tarball, or [Build From Source](Bu
|
|||
|
||||
h3. Download a Tarball
|
||||
|
||||
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.1-bin.tar.gz)
|
||||
We've built a tarball that contains everything you'll need. You'll find it [here](http://static.druid.io/artifacts/releases/druid-services-0.6.2-bin.tar.gz)
|
||||
Download this file to a directory of your choosing.
|
||||
You can extract the awesomeness within by issuing:
|
||||
|
||||
|
@ -48,7 +48,7 @@ tar zxvf druid-services-*-bin.tar.gz
|
|||
Not too lost so far right? That's great! If you cd into the directory:
|
||||
|
||||
```
|
||||
cd druid-services-0.6.1
|
||||
cd druid-services-0.6.2
|
||||
```
|
||||
|
||||
You should see a bunch of files:
|
||||
|
|
|
@ -9,7 +9,7 @@ There are two ways to setup Druid: download a tarball, or build it from source.
|
|||
|
||||
h3. Download a Tarball
|
||||
|
||||
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.1-bin.tar.gz.
|
||||
We've built a tarball that contains everything you'll need. You'll find it "here":http://static.druid.io/artifacts/releases/druid-services-0.6.2-bin.tar.gz.
|
||||
Download this bad boy to a directory of your choosing.
|
||||
|
||||
You can extract the awesomeness within by issuing:
|
||||
|
|
|
@ -4,6 +4,8 @@ druid.port=8081
|
|||
|
||||
druid.zk.service.host=localhost
|
||||
|
||||
druid.extensions.coordinates=["io.druid.extensions:druid-s3-extensions:0.6.2"]
|
||||
|
||||
# Dummy read only AWS account (used to download example data)
|
||||
druid.s3.secretKey=QyyfVZ7llSiRg6Qcrql1eEUG7buFpAK6T6engr1b
|
||||
druid.s3.accessKey=AKIAIMKECRUYKDQGR6YQ
|
||||
|
|
|
@ -24,6 +24,7 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
|
|||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import io.druid.indexing.common.config.EventReceiverFirehoseFactoryConfig;
|
||||
import io.druid.indexing.common.index.EventReceiverFirehoseFactory;
|
||||
import io.druid.initialization.DruidModule;
|
||||
|
||||
|
@ -45,5 +46,7 @@ public class IndexingServiceFirehoseModule implements DruidModule
|
|||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
// backwards compatibility
|
||||
ConfigProvider.bind(binder, EventReceiverFirehoseFactoryConfig.class);
|
||||
}
|
||||
}
|
||||
|
|
|
@ -0,0 +1,31 @@
|
|||
/*
|
||||
* 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.indexing.common.config;
|
||||
|
||||
import org.skife.config.Config;
|
||||
|
||||
/**
|
||||
*/
|
||||
@Deprecated
|
||||
public abstract class EventReceiverFirehoseFactoryConfig
|
||||
{
|
||||
@Config("druid.indexer.firehoseId.prefix")
|
||||
public abstract String getFirehoseIdPrefix();
|
||||
}
|
|
@ -33,6 +33,7 @@ import io.druid.data.input.Firehose;
|
|||
import io.druid.data.input.FirehoseFactory;
|
||||
import io.druid.data.input.InputRow;
|
||||
import io.druid.data.input.impl.MapInputRowParser;
|
||||
import io.druid.indexing.common.config.EventReceiverFirehoseFactoryConfig;
|
||||
|
||||
import javax.ws.rs.POST;
|
||||
import javax.ws.rs.Path;
|
||||
|
@ -62,15 +63,31 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
|
|||
private final MapInputRowParser parser;
|
||||
private final Optional<ChatHandlerProvider> chatHandlerProvider;
|
||||
|
||||
@Deprecated
|
||||
private final String oldServiceName;
|
||||
|
||||
@JsonCreator
|
||||
public EventReceiverFirehoseFactory(
|
||||
@JsonProperty("serviceName") String serviceName,
|
||||
@JsonProperty("firehoseId") String firehoseId,
|
||||
@JsonProperty("bufferSize") Integer bufferSize,
|
||||
@JsonProperty("parser") MapInputRowParser parser,
|
||||
@JacksonInject ChatHandlerProvider chatHandlerProvider
|
||||
@JacksonInject ChatHandlerProvider chatHandlerProvider,
|
||||
@JacksonInject EventReceiverFirehoseFactoryConfig config
|
||||
)
|
||||
{
|
||||
this.serviceName = Preconditions.checkNotNull(serviceName, "serviceName");
|
||||
this.serviceName = serviceName;
|
||||
|
||||
// This code is here for backwards compatibility
|
||||
if (serviceName == null) {
|
||||
this.oldServiceName = String.format(
|
||||
config.getFirehoseIdPrefix(),
|
||||
Preconditions.checkNotNull(firehoseId, "firehoseId")
|
||||
);
|
||||
} else {
|
||||
this.oldServiceName = null;
|
||||
}
|
||||
|
||||
this.bufferSize = bufferSize == null || bufferSize <= 0 ? DEFAULT_BUFFER_SIZE : bufferSize;
|
||||
this.parser = Preconditions.checkNotNull(parser, "parser");
|
||||
this.chatHandlerProvider = Optional.fromNullable(chatHandlerProvider);
|
||||
|
@ -86,7 +103,12 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
|
|||
if (chatHandlerProvider.isPresent()) {
|
||||
log.info("Found chathandler of class[%s]", chatHandlerProvider.get().getClass().getName());
|
||||
chatHandlerProvider.get().register(serviceName, firehose);
|
||||
chatHandlerProvider.get().register(serviceName.replaceAll(".*:", ""), firehose); // rolf
|
||||
chatHandlerProvider.get().register(serviceName.replaceAll(".*:", ""), firehose); // rofl
|
||||
|
||||
// backwards compatibility
|
||||
if (oldServiceName != null) {
|
||||
chatHandlerProvider.get().register(oldServiceName, firehose);
|
||||
}
|
||||
} else {
|
||||
log.info("No chathandler detected");
|
||||
}
|
||||
|
@ -174,7 +196,7 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
|
|||
}
|
||||
|
||||
return nextRow != null;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
|
@ -213,6 +235,11 @@ public class EventReceiverFirehoseFactory implements FirehoseFactory
|
|||
|
||||
if (chatHandlerProvider.isPresent()) {
|
||||
chatHandlerProvider.get().unregister(serviceName);
|
||||
|
||||
// backwards compatibility
|
||||
if (oldServiceName != null) {
|
||||
chatHandlerProvider.get().unregister(oldServiceName);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,77 +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.indexing.overlord.config;
|
||||
|
||||
import com.google.common.base.Splitter;
|
||||
import com.google.common.collect.ImmutableSet;
|
||||
import io.druid.server.initialization.ZkPathsConfig;
|
||||
import org.skife.config.Config;
|
||||
import org.skife.config.Default;
|
||||
|
||||
import java.util.Set;
|
||||
|
||||
/**
|
||||
*/
|
||||
public abstract class OverlordConfig extends ZkPathsConfig
|
||||
{
|
||||
private volatile Set<String> whitelistDatasources = null;
|
||||
|
||||
@Config("druid.host")
|
||||
public abstract String getServerName();
|
||||
|
||||
@Config("druid.indexer.runner")
|
||||
@Default("local")
|
||||
public abstract String getRunnerImpl();
|
||||
|
||||
@Config("druid.indexer.storage")
|
||||
@Default("local")
|
||||
public abstract String getStorageImpl();
|
||||
|
||||
@Config("druid.indexer.whitelist.enabled")
|
||||
@Default("false")
|
||||
public abstract boolean isWhitelistEnabled();
|
||||
|
||||
@Config("druid.indexer.whitelist.datasources")
|
||||
@Default("")
|
||||
public abstract String getWhitelistDatasourcesString();
|
||||
|
||||
public Set<String> getWhitelistDatasources()
|
||||
{
|
||||
if (whitelistDatasources == null) {
|
||||
synchronized (this) {
|
||||
if (whitelistDatasources == null) {
|
||||
whitelistDatasources = ImmutableSet.copyOf(Splitter.on(",").split(getWhitelistDatasourcesString()));
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
return whitelistDatasources;
|
||||
}
|
||||
|
||||
@Config("druid.indexer.autoscaling.enabled")
|
||||
public boolean isAutoScalingEnabled()
|
||||
{
|
||||
return false;
|
||||
}
|
||||
|
||||
@Config("druid.indexer.autoscaling.strategy")
|
||||
@Default("noop")
|
||||
public abstract String getAutoScalingImpl();
|
||||
}
|
|
@ -63,7 +63,7 @@ import java.util.List;
|
|||
*/
|
||||
@Command(
|
||||
name = "coordinator",
|
||||
description = "Runs the Coordinator, see http://druid.io/docs/0.6.1/Coordinator.html for a description."
|
||||
description = "Runs the Coordinator, see http://druid.io/docs/0.6.2/Coordinator.html for a description."
|
||||
)
|
||||
public class CliCoordinator extends ServerRunnable
|
||||
{
|
||||
|
|
|
@ -41,7 +41,7 @@ import java.util.List;
|
|||
*/
|
||||
@Command(
|
||||
name = "hadoop",
|
||||
description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.1/Batch-ingestion.html for a description."
|
||||
description = "Runs the batch Hadoop Druid Indexer, see http://druid.io/docs/0.6.2/Batch-ingestion.html for a description."
|
||||
)
|
||||
public class CliHadoopIndexer implements Runnable
|
||||
{
|
||||
|
|
|
@ -42,7 +42,7 @@ import java.util.List;
|
|||
*/
|
||||
@Command(
|
||||
name = "historical",
|
||||
description = "Runs a Historical node, see http://druid.io/docs/0.6.1/Historical.html for a description"
|
||||
description = "Runs a Historical node, see http://druid.io/docs/0.6.2/Historical.html for a description"
|
||||
)
|
||||
public class CliHistorical extends ServerRunnable
|
||||
{
|
||||
|
|
|
@ -65,9 +65,9 @@ public class ConvertProperties implements Runnable
|
|||
new Rename("com.metamx.emitter.logging.level", "druid.emitter.logging.logLevel"),
|
||||
new Rename("com.metamx.emitter.http", "druid.emitter.http"),
|
||||
new Rename("com.metamx.emitter.http.url", "druid.emitter.http.recipientBaseUrl"),
|
||||
new Rename("com.metamx.emitter.period", "druid.emitter.emissionPeriod"),
|
||||
new Rename("com.metamx.druid.emitter.period", "druid.emitter.emissionPeriod"),
|
||||
new Rename("com.metamx.metrics.emitter.period", "druid.emitter.emissionPeriod"),
|
||||
new Rename("com.metamx.emitter.period", "druid.monitoring.emissionPeriod"),
|
||||
new Rename("com.metamx.druid.emitter.period", "druid.monitoring.emissionPeriod"),
|
||||
new Rename("com.metamx.metrics.emitter.period", "druid.monitoring.emissionPeriod"),
|
||||
new PrefixRename("com.metamx.emitter", "druid.emitter"),
|
||||
new PrefixRename("com.metamx.druid.emitter", "druid.emitter"),
|
||||
new IndexCacheConverter(),
|
||||
|
@ -79,6 +79,7 @@ public class ConvertProperties implements Runnable
|
|||
new Rename("druid.client.http.connections", "druid.broker.http.numConnections"),
|
||||
new Rename("com.metamx.query.groupBy.maxResults", "druid.query.groupBy.maxResults"),
|
||||
new Rename("com.metamx.query.search.maxSearchLimit", "druid.query.search.maxSearchLimit"),
|
||||
new Rename("druid.indexer.runner", "druid.indexer.runner.type"),
|
||||
new Rename("druid.indexer.storage", "druid.indexer.storage.type"),
|
||||
new Rename("druid.indexer.threads", "druid.indexer.runner.forks"),
|
||||
new Rename("druid.indexer.taskDir", "druid.indexer.runner.taskDir"),
|
||||
|
|
Loading…
Reference in New Issue