mirror of https://github.com/apache/druid.git
working indexer with registererers
This commit is contained in:
parent
5698f640d7
commit
34cb352cf8
|
@ -73,6 +73,11 @@ public class IndexIO
|
|||
handler.storeLatest(index, file);
|
||||
}
|
||||
|
||||
public static boolean hasHandler()
|
||||
{
|
||||
return (IndexIO.handler != null);
|
||||
}
|
||||
|
||||
public static void registerHandler(IndexIOHandler handler)
|
||||
{
|
||||
if (IndexIO.handler == null) {
|
||||
|
|
|
@ -87,9 +87,9 @@ public class HadoopDruidIndexerConfig
|
|||
|
||||
public static HadoopDruidIndexerConfig fromMap(Map<String, Object> argSpec)
|
||||
{
|
||||
if (argSpec.containsKey("registerers")) {
|
||||
if (argSpec.containsKey("registererers")) {
|
||||
List<Registererer> registererers = Lists.transform(
|
||||
MapUtils.getList(argSpec, "registerers"),
|
||||
MapUtils.getList(argSpec, "registererers"),
|
||||
new Function<Object, Registererer>()
|
||||
{
|
||||
@Override
|
||||
|
@ -179,6 +179,7 @@ public class HadoopDruidIndexerConfig
|
|||
private volatile DataRollupSpec rollupSpec;
|
||||
private volatile UpdaterJobSpec updaterJobSpec;
|
||||
private volatile boolean ignoreInvalidRows = false;
|
||||
private volatile List<String> registererers = Lists.newArrayList();
|
||||
|
||||
public List<Interval> getIntervals()
|
||||
{
|
||||
|
@ -425,9 +426,20 @@ public class HadoopDruidIndexerConfig
|
|||
this.ignoreInvalidRows = ignoreInvalidRows;
|
||||
}
|
||||
|
||||
/********************************************
|
||||
Granularity/Bucket Helper Methods
|
||||
********************************************/
|
||||
@JsonProperty
|
||||
public List<String> getRegistererers()
|
||||
{
|
||||
return registererers;
|
||||
}
|
||||
|
||||
public void setRegistererers(List<String> registererers)
|
||||
{
|
||||
this.registererers = registererers;
|
||||
}
|
||||
|
||||
/********************************************
|
||||
Granularity/Bucket Helper Methods
|
||||
********************************************/
|
||||
|
||||
/**
|
||||
* Get the proper bucket for this "row"
|
||||
|
|
|
@ -19,9 +19,13 @@
|
|||
|
||||
package com.metamx.druid.indexer;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.lifecycle.Lifecycle;
|
||||
import com.metamx.common.logger.Logger;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidIndexerMain
|
||||
|
@ -30,15 +34,14 @@ public class HadoopDruidIndexerMain
|
|||
|
||||
public static void main(String[] args) throws Exception
|
||||
{
|
||||
if (args.length < 1 || args.length > 2) {
|
||||
HadoopDruidIndexerNode.printHelp();
|
||||
if (args.length != 1) {
|
||||
printHelp();
|
||||
System.exit(2);
|
||||
}
|
||||
|
||||
HadoopDruidIndexerNode node = HadoopDruidIndexerNode.builder().build();
|
||||
|
||||
node.setIntervalSpec(args.length == 1 ? null : args[0])
|
||||
.setArgumentSpec(args[args.length == 1 ? 0 : 1]);
|
||||
node.setArgumentSpec(args[0]);
|
||||
|
||||
Lifecycle lifecycle = new Lifecycle();
|
||||
lifecycle.addManagedInstance(node);
|
||||
|
@ -49,8 +52,72 @@ public class HadoopDruidIndexerMain
|
|||
catch (Throwable t) {
|
||||
log.info(t, "Throwable caught at startup, committing seppuku");
|
||||
Thread.sleep(500);
|
||||
HadoopDruidIndexerNode.printHelp();
|
||||
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();
|
||||
|
||||
public 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("}");
|
||||
}
|
||||
}
|
||||
|
|
|
@ -1,88 +1,16 @@
|
|||
package com.metamx.druid.indexer;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.Lists;
|
||||
import com.metamx.common.Pair;
|
||||
import com.metamx.common.lifecycle.LifecycleStart;
|
||||
import com.metamx.common.lifecycle.LifecycleStop;
|
||||
import org.codehaus.jackson.map.jsontype.NamedType;
|
||||
import org.joda.time.DateTime;
|
||||
import org.joda.time.Interval;
|
||||
|
||||
import java.io.File;
|
||||
import java.util.Arrays;
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
*/
|
||||
public class HadoopDruidIndexerNode
|
||||
{
|
||||
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("segmentGranularity", "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();
|
||||
|
||||
public static void printHelp()
|
||||
{
|
||||
System.out.println("Usage: <java invocation> <time_interval> <config_spec>");
|
||||
System.out.println("<time_interval> is the ISO8601 interval of data to run over.");
|
||||
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("}");
|
||||
}
|
||||
|
||||
public static Builder builder()
|
||||
{
|
||||
return new Builder();
|
||||
|
@ -139,17 +67,6 @@ public class HadoopDruidIndexerNode
|
|||
config = HadoopDruidIndexerConfig.fromFile(new File(argumentSpec));
|
||||
}
|
||||
|
||||
final List<Interval> dataInterval;
|
||||
if (intervalSpec != null) {
|
||||
dataInterval = Lists.transform(
|
||||
Arrays.asList(intervalSpec.split(",")),
|
||||
new StringIntervalFunction()
|
||||
);
|
||||
|
||||
config.setIntervals(dataInterval);
|
||||
}
|
||||
config.setVersion(new DateTime());
|
||||
|
||||
new HadoopDruidIndexerJob(config).run();
|
||||
}
|
||||
|
||||
|
|
|
@ -42,9 +42,8 @@ import com.metamx.druid.index.v1.IncrementalIndex;
|
|||
import com.metamx.druid.index.v1.IndexIO;
|
||||
import com.metamx.druid.index.v1.IndexMerger;
|
||||
import com.metamx.druid.index.v1.MMappedIndex;
|
||||
import com.metamx.druid.index.v1.serde.Registererer;
|
||||
import com.metamx.druid.input.MapBasedInputRow;
|
||||
import com.metamx.druid.indexer.rollup.DataRollupSpec;
|
||||
import com.metamx.druid.input.MapBasedInputRow;
|
||||
import com.metamx.druid.jackson.DefaultObjectMapper;
|
||||
import org.apache.commons.io.FileUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
|
Loading…
Reference in New Issue