mirror of
https://github.com/apache/druid.git
synced 2025-02-09 03:24:55 +00:00
Remove ParseSpec.toInputFormat() (#9815)
* Remove toInputFormat() from ParseSpec * fix test
This commit is contained in:
parent
c6caae9a24
commit
964a1fc9df
@ -22,7 +22,6 @@ package org.apache.druid.data.input.impl;
|
|||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
|
||||||
import org.apache.druid.java.util.common.parsers.CSVParser;
|
import org.apache.druid.java.util.common.parsers.CSVParser;
|
||||||
import org.apache.druid.java.util.common.parsers.Parser;
|
import org.apache.druid.java.util.common.parsers.Parser;
|
||||||
|
|
||||||
@ -97,12 +96,6 @@ public class CSVParseSpec extends ParseSpec
|
|||||||
return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows);
|
return new CSVParser(listDelimiter, columns, hasHeaderRow, skipHeaderRows);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public InputFormat toInputFormat()
|
|
||||||
{
|
|
||||||
return new CsvInputFormat(columns, listDelimiter, null, hasHeaderRow, skipHeaderRows);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
||||||
{
|
{
|
||||||
|
@ -22,7 +22,6 @@ package org.apache.druid.data.input.impl;
|
|||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
|
||||||
import org.apache.druid.java.util.common.parsers.DelimitedParser;
|
import org.apache.druid.java.util.common.parsers.DelimitedParser;
|
||||||
import org.apache.druid.java.util.common.parsers.Parser;
|
import org.apache.druid.java.util.common.parsers.Parser;
|
||||||
|
|
||||||
@ -124,12 +123,6 @@ public class DelimitedParseSpec extends ParseSpec
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public InputFormat toInputFormat()
|
|
||||||
{
|
|
||||||
return new DelimitedInputFormat(columns, listDelimiter, delimiter, hasHeaderRow, null, skipHeaderRows);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
||||||
{
|
{
|
||||||
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonCreator;
|
|||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||||
import com.fasterxml.jackson.core.JsonParser.Feature;
|
import com.fasterxml.jackson.core.JsonParser.Feature;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathParser;
|
import org.apache.druid.java.util.common.parsers.JSONPathParser;
|
||||||
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
|
||||||
import org.apache.druid.java.util.common.parsers.Parser;
|
import org.apache.druid.java.util.common.parsers.Parser;
|
||||||
@ -68,12 +67,6 @@ public class JSONParseSpec extends NestedDataParseSpec<JSONPathSpec>
|
|||||||
return new JSONPathParser(getFlattenSpec(), objectMapper);
|
return new JSONPathParser(getFlattenSpec(), objectMapper);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public InputFormat toInputFormat()
|
|
||||||
{
|
|
||||||
return new JsonInputFormat(getFlattenSpec(), getFeatureSpec());
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
||||||
{
|
{
|
||||||
|
@ -23,13 +23,10 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||||||
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
import com.fasterxml.jackson.annotation.JsonSubTypes;
|
||||||
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
import com.fasterxml.jackson.annotation.JsonTypeInfo;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
|
||||||
import org.apache.druid.guice.annotations.ExtensionPoint;
|
import org.apache.druid.guice.annotations.ExtensionPoint;
|
||||||
import org.apache.druid.guice.annotations.PublicApi;
|
import org.apache.druid.guice.annotations.PublicApi;
|
||||||
import org.apache.druid.java.util.common.parsers.Parser;
|
import org.apache.druid.java.util.common.parsers.Parser;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
|
|
||||||
@Deprecated
|
@Deprecated
|
||||||
@ExtensionPoint
|
@ExtensionPoint
|
||||||
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format")
|
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "format")
|
||||||
@ -71,16 +68,6 @@ public abstract class ParseSpec
|
|||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
|
||||||
* Returns null if it's not implemented yet.
|
|
||||||
* This method (and maybe this class) will be removed in favor of {@link InputFormat} in the future.
|
|
||||||
*/
|
|
||||||
@Nullable
|
|
||||||
public InputFormat toInputFormat()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@PublicApi
|
@PublicApi
|
||||||
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
public ParseSpec withTimestampSpec(TimestampSpec spec)
|
||||||
{
|
{
|
||||||
|
@ -52,7 +52,7 @@ public class FirehoseFactoryToInputSourceAdaptorTest
|
|||||||
}
|
}
|
||||||
final TestFirehoseFactory firehoseFactory = new TestFirehoseFactory(lines);
|
final TestFirehoseFactory firehoseFactory = new TestFirehoseFactory(lines);
|
||||||
final StringInputRowParser inputRowParser = new StringInputRowParser(
|
final StringInputRowParser inputRowParser = new StringInputRowParser(
|
||||||
new UnimplementedInputFormatCsvParseSpec(
|
new CSVParseSpec(
|
||||||
new TimestampSpec(null, "yyyyMMdd", null),
|
new TimestampSpec(null, "yyyyMMdd", null),
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "name", "score"))),
|
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("timestamp", "name", "score"))),
|
||||||
",",
|
",",
|
||||||
@ -95,28 +95,6 @@ public class FirehoseFactoryToInputSourceAdaptorTest
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class UnimplementedInputFormatCsvParseSpec extends CSVParseSpec
|
|
||||||
{
|
|
||||||
private UnimplementedInputFormatCsvParseSpec(
|
|
||||||
TimestampSpec timestampSpec,
|
|
||||||
DimensionsSpec dimensionsSpec,
|
|
||||||
String listDelimiter,
|
|
||||||
List<String> columns,
|
|
||||||
boolean hasHeaderRow,
|
|
||||||
int skipHeaderRows
|
|
||||||
)
|
|
||||||
{
|
|
||||||
super(timestampSpec, dimensionsSpec, listDelimiter, columns, hasHeaderRow, skipHeaderRows);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
@Override
|
|
||||||
public InputFormat toInputFormat()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class TestFirehoseFactory implements FiniteFirehoseFactory<StringInputRowParser, Object>
|
private static class TestFirehoseFactory implements FiniteFirehoseFactory<StringInputRowParser, Object>
|
||||||
{
|
{
|
||||||
private final List<String> lines;
|
private final List<String> lines;
|
||||||
|
@ -194,9 +194,7 @@ public class KafkaSupervisor extends SeekableStreamSupervisor<Integer, Long>
|
|||||||
true,
|
true,
|
||||||
minimumMessageTime,
|
minimumMessageTime,
|
||||||
maximumMessageTime,
|
maximumMessageTime,
|
||||||
ioConfig.getInputFormat(
|
ioConfig.getInputFormat()
|
||||||
spec.getDataSchema().getParser() == null ? null : spec.getDataSchema().getParser().getParseSpec()
|
|
||||||
)
|
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -140,9 +140,7 @@ public class KinesisSupervisor extends SeekableStreamSupervisor<String, String>
|
|||||||
true,
|
true,
|
||||||
minimumMessageTime,
|
minimumMessageTime,
|
||||||
maximumMessageTime,
|
maximumMessageTime,
|
||||||
ioConfig.getInputFormat(
|
ioConfig.getInputFormat(),
|
||||||
spec.getDataSchema().getParser() == null ? null : spec.getDataSchema().getParser().getParseSpec()
|
|
||||||
),
|
|
||||||
ioConfig.getEndpoint(),
|
ioConfig.getEndpoint(),
|
||||||
ioConfig.getRecordsPerFetch(),
|
ioConfig.getRecordsPerFetch(),
|
||||||
ioConfig.getFetchDelayMillis(),
|
ioConfig.getFetchDelayMillis(),
|
||||||
|
@ -44,7 +44,6 @@ import org.apache.druid.data.input.InputSource;
|
|||||||
import org.apache.druid.data.input.InputSourceReader;
|
import org.apache.druid.data.input.InputSourceReader;
|
||||||
import org.apache.druid.data.input.Rows;
|
import org.apache.druid.data.input.Rows;
|
||||||
import org.apache.druid.data.input.impl.InputRowParser;
|
import org.apache.druid.data.input.impl.InputRowParser;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
|
||||||
import org.apache.druid.hll.HyperLogLogCollector;
|
import org.apache.druid.hll.HyperLogLogCollector;
|
||||||
import org.apache.druid.indexer.Checks;
|
import org.apache.druid.indexer.Checks;
|
||||||
import org.apache.druid.indexer.IngestionState;
|
import org.apache.druid.indexer.IngestionState;
|
||||||
@ -1046,10 +1045,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
|||||||
|
|
||||||
private static InputFormat getInputFormat(IndexIngestionSpec ingestionSchema)
|
private static InputFormat getInputFormat(IndexIngestionSpec ingestionSchema)
|
||||||
{
|
{
|
||||||
final InputRowParser parser = ingestionSchema.getDataSchema().getParser();
|
return ingestionSchema.getIOConfig().getNonNullInputFormat();
|
||||||
return ingestionSchema.getIOConfig().getNonNullInputFormat(
|
|
||||||
parser == null ? null : parser.getParseSpec()
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
public static class IndexIngestionSpec extends IngestionSpec<IndexIOConfig, IndexTuningConfig>
|
public static class IndexIngestionSpec extends IngestionSpec<IndexIOConfig, IndexTuningConfig>
|
||||||
@ -1184,13 +1180,9 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
|
|||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
public InputFormat getNonNullInputFormat(@Nullable ParseSpec parseSpec)
|
public InputFormat getNonNullInputFormat()
|
||||||
{
|
{
|
||||||
if (inputFormat == null) {
|
return Preconditions.checkNotNull(inputFormat, "inputFormat");
|
||||||
return Preconditions.checkNotNull(parseSpec, "parseSpec").toInputFormat();
|
|
||||||
} else {
|
|
||||||
return inputFormat;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -30,7 +30,6 @@ import org.apache.druid.data.input.InputFormat;
|
|||||||
import org.apache.druid.data.input.InputSource;
|
import org.apache.druid.data.input.InputSource;
|
||||||
import org.apache.druid.data.input.InputSplit;
|
import org.apache.druid.data.input.InputSplit;
|
||||||
import org.apache.druid.data.input.SplitHintSpec;
|
import org.apache.druid.data.input.SplitHintSpec;
|
||||||
import org.apache.druid.data.input.impl.InputRowParser;
|
|
||||||
import org.apache.druid.data.input.impl.SplittableInputSource;
|
import org.apache.druid.data.input.impl.SplittableInputSource;
|
||||||
import org.apache.druid.indexer.TaskState;
|
import org.apache.druid.indexer.TaskState;
|
||||||
import org.apache.druid.indexer.TaskStatusPlus;
|
import org.apache.druid.indexer.TaskStatusPlus;
|
||||||
@ -196,19 +195,8 @@ public abstract class ParallelIndexPhaseRunner<SubTaskType extends Task, SubTask
|
|||||||
if (lastStatus != null) {
|
if (lastStatus != null) {
|
||||||
LOG.error("Failed because of the failed sub task[%s]", lastStatus.getId());
|
LOG.error("Failed because of the failed sub task[%s]", lastStatus.getId());
|
||||||
} else {
|
} else {
|
||||||
final SinglePhaseSubTaskSpec spec =
|
final SinglePhaseSubTaskSpec spec = (SinglePhaseSubTaskSpec) taskCompleteEvent.getSpec();
|
||||||
(SinglePhaseSubTaskSpec) taskCompleteEvent.getSpec();
|
LOG.error("Failed to run sub tasks for inputSplits[%s]", spec.getInputSplit());
|
||||||
final InputRowParser inputRowParser = spec.getIngestionSpec().getDataSchema().getParser();
|
|
||||||
LOG.error(
|
|
||||||
"Failed to run sub tasks for inputSplits[%s]",
|
|
||||||
getSplitsIfSplittable(
|
|
||||||
spec.getIngestionSpec().getIOConfig().getNonNullInputSource(inputRowParser),
|
|
||||||
spec.getIngestionSpec().getIOConfig().getNonNullInputFormat(
|
|
||||||
inputRowParser == null ? null : inputRowParser.getParseSpec()
|
|
||||||
),
|
|
||||||
tuningConfig.getSplitHintSpec()
|
|
||||||
)
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
break;
|
break;
|
||||||
default:
|
default:
|
||||||
|
@ -33,7 +33,6 @@ import org.apache.druid.client.indexing.IndexingServiceClient;
|
|||||||
import org.apache.druid.data.input.FiniteFirehoseFactory;
|
import org.apache.druid.data.input.FiniteFirehoseFactory;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.InputSource;
|
import org.apache.druid.data.input.InputSource;
|
||||||
import org.apache.druid.data.input.impl.InputRowParser;
|
|
||||||
import org.apache.druid.indexer.TaskState;
|
import org.apache.druid.indexer.TaskState;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
import org.apache.druid.indexer.partitions.PartitionsSpec;
|
||||||
@ -1001,10 +1000,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
|
|||||||
|
|
||||||
static InputFormat getInputFormat(ParallelIndexIngestionSpec ingestionSchema)
|
static InputFormat getInputFormat(ParallelIndexIngestionSpec ingestionSchema)
|
||||||
{
|
{
|
||||||
final InputRowParser parser = ingestionSchema.getDataSchema().getParser();
|
return ingestionSchema.getIOConfig().getNonNullInputFormat();
|
||||||
return ingestionSchema.getIOConfig().getNonNullInputFormat(
|
|
||||||
parser == null ? null : parser.getParseSpec()
|
|
||||||
);
|
|
||||||
}
|
}
|
||||||
|
|
||||||
/**
|
/**
|
||||||
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
|
||||||
import org.apache.druid.segment.indexing.IOConfig;
|
import org.apache.druid.segment.indexing.IOConfig;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
|
|
||||||
@ -127,8 +126,8 @@ public abstract class SeekableStreamIndexTaskIOConfig<PartitionIdType, SequenceO
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
public InputFormat getInputFormat(ParseSpec parseSpec)
|
public InputFormat getInputFormat()
|
||||||
{
|
{
|
||||||
return inputFormat == null ? Preconditions.checkNotNull(parseSpec, "parseSpec").toInputFormat() : inputFormat;
|
return inputFormat;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
@ -255,7 +255,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
|
|||||||
.map(AggregatorFactory::getName)
|
.map(AggregatorFactory::getName)
|
||||||
.collect(Collectors.toList())
|
.collect(Collectors.toList())
|
||||||
);
|
);
|
||||||
this.inputFormat = ioConfig.getInputFormat(parser == null ? null : parser.getParseSpec());
|
this.inputFormat = ioConfig.getInputFormat();
|
||||||
this.parser = parser;
|
this.parser = parser;
|
||||||
this.authorizerMapper = authorizerMapper;
|
this.authorizerMapper = authorizerMapper;
|
||||||
this.chatHandlerProvider = chatHandlerProvider;
|
this.chatHandlerProvider = chatHandlerProvider;
|
||||||
|
@ -98,7 +98,7 @@ public abstract class SeekableStreamSamplerSpec<PartitionIdType, SequenceOffsetT
|
|||||||
ioConfig.isUseEarliestSequenceNumber()
|
ioConfig.isUseEarliestSequenceNumber()
|
||||||
);
|
);
|
||||||
inputFormat = Preconditions.checkNotNull(
|
inputFormat = Preconditions.checkNotNull(
|
||||||
ioConfig.getInputFormat(null),
|
ioConfig.getInputFormat(),
|
||||||
"[spec.ioConfig.inputFormat] is required"
|
"[spec.ioConfig.inputFormat] is required"
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JsonProperty;
|
|||||||
import com.google.common.base.Optional;
|
import com.google.common.base.Optional;
|
||||||
import com.google.common.base.Preconditions;
|
import com.google.common.base.Preconditions;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
|
||||||
import org.apache.druid.java.util.common.IAE;
|
import org.apache.druid.java.util.common.IAE;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Duration;
|
import org.joda.time.Duration;
|
||||||
@ -102,22 +101,12 @@ public abstract class SeekableStreamSupervisorIOConfig
|
|||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
@Nullable
|
||||||
@JsonProperty("inputFormat")
|
@JsonProperty()
|
||||||
private InputFormat getGivenInputFormat()
|
public InputFormat getInputFormat()
|
||||||
{
|
{
|
||||||
return inputFormat;
|
return inputFormat;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Nullable
|
|
||||||
public InputFormat getInputFormat(@Nullable ParseSpec parseSpec)
|
|
||||||
{
|
|
||||||
if (inputFormat == null) {
|
|
||||||
return Preconditions.checkNotNull(parseSpec, "parseSpec").toInputFormat();
|
|
||||||
} else {
|
|
||||||
return inputFormat;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public Integer getReplicas()
|
public Integer getReplicas()
|
||||||
{
|
{
|
||||||
|
@ -176,6 +176,7 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
|
|||||||
getObjectMapper(),
|
getObjectMapper(),
|
||||||
tmpDir,
|
tmpDir,
|
||||||
CompactionTaskRunTest.DEFAULT_PARSE_SPEC,
|
CompactionTaskRunTest.DEFAULT_PARSE_SPEC,
|
||||||
|
null,
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
|
@ -381,6 +381,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||||||
getObjectMapper(),
|
getObjectMapper(),
|
||||||
tmpDir,
|
tmpDir,
|
||||||
DEFAULT_PARSE_SPEC,
|
DEFAULT_PARSE_SPEC,
|
||||||
|
null,
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
@ -791,6 +792,7 @@ public class CompactionTaskRunTest extends IngestionTestBase
|
|||||||
getObjectMapper(),
|
getObjectMapper(),
|
||||||
tmpDir,
|
tmpDir,
|
||||||
DEFAULT_PARSE_SPEC,
|
DEFAULT_PARSE_SPEC,
|
||||||
|
null,
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
|
@ -22,15 +22,18 @@ package org.apache.druid.indexing.common.task;
|
|||||||
import com.fasterxml.jackson.core.JsonProcessingException;
|
import com.fasterxml.jackson.core.JsonProcessingException;
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
import com.google.common.collect.ImmutableMap;
|
import com.google.common.collect.ImmutableMap;
|
||||||
import com.google.common.collect.Sets;
|
import com.google.common.collect.Sets;
|
||||||
import com.google.common.io.Files;
|
import com.google.common.io.Files;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.CSVParseSpec;
|
import org.apache.druid.data.input.impl.CSVParseSpec;
|
||||||
|
import org.apache.druid.data.input.impl.CsvInputFormat;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||||
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
import org.apache.druid.data.input.impl.FloatDimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.JSONParseSpec;
|
import org.apache.druid.data.input.impl.JSONParseSpec;
|
||||||
|
import org.apache.druid.data.input.impl.JsonInputFormat;
|
||||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||||
import org.apache.druid.data.input.impl.LongDimensionSchema;
|
import org.apache.druid.data.input.impl.LongDimensionSchema;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
import org.apache.druid.data.input.impl.ParseSpec;
|
||||||
@ -134,7 +137,13 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
false,
|
false,
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
private static final InputFormat DEFAULT_INPUT_FORMAT = DEFAULT_PARSE_SPEC.toInputFormat();
|
private static final InputFormat DEFAULT_INPUT_FORMAT = new CsvInputFormat(
|
||||||
|
Arrays.asList("ts", "dim", "val"),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
0
|
||||||
|
);
|
||||||
|
|
||||||
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}")
|
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}")
|
||||||
public static Iterable<Object[]> constructorFeeder()
|
public static Iterable<Object[]> constructorFeeder()
|
||||||
@ -205,13 +214,11 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
|
||||||
createTuningConfigWithMaxRowsPerSegment(2, true),
|
createTuningConfigWithMaxRowsPerSegment(2, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -252,20 +259,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
writer.write("2014-01-01T02:00:30Z,c,and|another,0|1,1\n");
|
writer.write("2014-01-01T02:00:30Z,c,and|another,0|1,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
final DimensionsSpec dimensionsSpec = new DimensionsSpec(
|
||||||
null,
|
|
||||||
null,
|
|
||||||
createIngestionSpec(
|
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
|
||||||
tmpDir,
|
|
||||||
new CSVParseSpec(
|
|
||||||
new TimestampSpec(
|
|
||||||
"ts",
|
|
||||||
"auto",
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new DimensionsSpec(
|
|
||||||
DimensionsSpec.getDefaultSchemas(
|
DimensionsSpec.getDefaultSchemas(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"ts",
|
"ts",
|
||||||
@ -277,28 +271,53 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
"dimtarray2",
|
"dimtarray2",
|
||||||
"dimtnum_array"
|
"dimtnum_array"
|
||||||
)
|
)
|
||||||
),
|
)
|
||||||
new ArrayList<>(),
|
);
|
||||||
new ArrayList<>()
|
final List<String> columns = Arrays.asList("ts", "dim", "dim_array", "dim_num_array", "val");
|
||||||
),
|
final String listDelimiter = "|";
|
||||||
"|",
|
final TransformSpec transformSpec = new TransformSpec(
|
||||||
Arrays.asList("ts", "dim", "dim_array", "dim_num_array", "val"),
|
|
||||||
false,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
new TransformSpec(
|
|
||||||
new SelectorDimFilter("dim", "b", null),
|
new SelectorDimFilter("dim", "b", null),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new ExpressionTransform("dimt", "concat(dim,dim)", ExprMacroTable.nil()),
|
new ExpressionTransform("dimt", "concat(dim,dim)", ExprMacroTable.nil()),
|
||||||
new ExpressionTransform("dimtarray1", "array(dim, dim)", ExprMacroTable.nil()),
|
new ExpressionTransform("dimtarray1", "array(dim, dim)", ExprMacroTable.nil()),
|
||||||
new ExpressionTransform("dimtarray2", "map(d -> concat(d, 'foo'), dim_array)", ExprMacroTable.nil()),
|
new ExpressionTransform(
|
||||||
|
"dimtarray2",
|
||||||
|
"map(d -> concat(d, 'foo'), dim_array)",
|
||||||
|
ExprMacroTable.nil()
|
||||||
|
),
|
||||||
new ExpressionTransform("dimtnum_array", "map(d -> d + 3, dim_num_array)", ExprMacroTable.nil())
|
new ExpressionTransform("dimtnum_array", "map(d -> d + 3, dim_num_array)", ExprMacroTable.nil())
|
||||||
)
|
)
|
||||||
),
|
);
|
||||||
|
final IndexTuningConfig tuningConfig = createTuningConfigWithMaxRowsPerSegment(2, false);
|
||||||
|
final IndexIngestionSpec indexIngestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
indexIngestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
DEFAULT_TIMESTAMP_SPEC,
|
||||||
|
dimensionsSpec,
|
||||||
|
new CsvInputFormat(columns, listDelimiter, null, false, 0),
|
||||||
|
transformSpec,
|
||||||
null,
|
null,
|
||||||
createTuningConfigWithMaxRowsPerSegment(2, false),
|
tuningConfig,
|
||||||
false
|
false
|
||||||
),
|
);
|
||||||
|
} else {
|
||||||
|
indexIngestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(DEFAULT_TIMESTAMP_SPEC, dimensionsSpec, listDelimiter, columns, false, 0),
|
||||||
|
transformSpec,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
|
IndexTask indexTask = new IndexTask(
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
indexIngestionSpec,
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -331,11 +350,20 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
final DimensionSelector selector1 = cursor.getColumnSelectorFactory()
|
final DimensionSelector selector1 = cursor.getColumnSelectorFactory()
|
||||||
.makeDimensionSelector(new DefaultDimensionSpec("dimt", "dimt"));
|
.makeDimensionSelector(new DefaultDimensionSpec("dimt", "dimt"));
|
||||||
final DimensionSelector selector2 = cursor.getColumnSelectorFactory()
|
final DimensionSelector selector2 = cursor.getColumnSelectorFactory()
|
||||||
.makeDimensionSelector(new DefaultDimensionSpec("dimtarray1", "dimtarray1"));
|
.makeDimensionSelector(new DefaultDimensionSpec(
|
||||||
|
"dimtarray1",
|
||||||
|
"dimtarray1"
|
||||||
|
));
|
||||||
final DimensionSelector selector3 = cursor.getColumnSelectorFactory()
|
final DimensionSelector selector3 = cursor.getColumnSelectorFactory()
|
||||||
.makeDimensionSelector(new DefaultDimensionSpec("dimtarray2", "dimtarray2"));
|
.makeDimensionSelector(new DefaultDimensionSpec(
|
||||||
|
"dimtarray2",
|
||||||
|
"dimtarray2"
|
||||||
|
));
|
||||||
final DimensionSelector selector4 = cursor.getColumnSelectorFactory()
|
final DimensionSelector selector4 = cursor.getColumnSelectorFactory()
|
||||||
.makeDimensionSelector(new DefaultDimensionSpec("dimtnum_array", "dimtnum_array"));
|
.makeDimensionSelector(new DefaultDimensionSpec(
|
||||||
|
"dimtnum_array",
|
||||||
|
"dimtnum_array"
|
||||||
|
));
|
||||||
|
|
||||||
|
|
||||||
Map<String, Object> row = new HashMap<>();
|
Map<String, Object> row = new HashMap<>();
|
||||||
@ -375,15 +403,14 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new ArbitraryGranularitySpec(
|
new ArbitraryGranularitySpec(
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02"))
|
Collections.singletonList(Intervals.of("2014-01-01/2014-01-02"))
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfigWithMaxRowsPerSegment(10, true),
|
createTuningConfigWithMaxRowsPerSegment(10, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -414,16 +441,15 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
Collections.singletonList(Intervals.of("2014-01-01T08:00:00Z/2014-01-01T09:00:00Z"))
|
Collections.singletonList(Intervals.of("2014-01-01T08:00:00Z/2014-01-01T09:00:00Z"))
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfigWithMaxRowsPerSegment(50, true),
|
createTuningConfigWithMaxRowsPerSegment(50, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -454,13 +480,11 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
|
||||||
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 1, null), true),
|
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 1, null), true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -496,13 +520,11 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
final IndexTask indexTask = new IndexTask(
|
final IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
|
||||||
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")), true),
|
createTuningConfigWithPartitionsSpec(new HashedPartitionsSpec(null, 2, ImmutableList.of("dim")), true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -574,13 +596,11 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
|
||||||
createTuningConfigWithMaxRowsPerSegment(2, false),
|
createTuningConfigWithMaxRowsPerSegment(2, false),
|
||||||
true
|
true
|
||||||
),
|
),
|
||||||
@ -624,16 +644,15 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfigWithMaxRowsPerSegment(2, true),
|
createTuningConfigWithMaxRowsPerSegment(2, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -676,25 +695,37 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null);
|
||||||
|
final IndexTuningConfig tuningConfig = createTuningConfigWithMaxRowsPerSegment(2, true);
|
||||||
|
final IndexIngestionSpec ingestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, null, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
timestampSpec,
|
||||||
|
DimensionsSpec.EMPTY,
|
||||||
|
new CsvInputFormat(null, null, null, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
ingestionSpec,
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
|
||||||
tmpDir,
|
|
||||||
new CSVParseSpec(
|
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
DimensionsSpec.EMPTY,
|
|
||||||
null,
|
|
||||||
null,
|
|
||||||
true,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
|
||||||
createTuningConfigWithMaxRowsPerSegment(2, true),
|
|
||||||
false
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -723,25 +754,38 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null);
|
||||||
|
final List<String> columns = Arrays.asList("time", "dim", "val");
|
||||||
|
final IndexTuningConfig tuningConfig = createTuningConfigWithMaxRowsPerSegment(2, true);
|
||||||
|
final IndexIngestionSpec ingestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
timestampSpec,
|
||||||
|
DimensionsSpec.EMPTY,
|
||||||
|
new CsvInputFormat(columns, null, null, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
ingestionSpec,
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
|
||||||
tmpDir,
|
|
||||||
new CSVParseSpec(
|
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
DimensionsSpec.EMPTY,
|
|
||||||
null,
|
|
||||||
Arrays.asList("time", "dim", "val"),
|
|
||||||
true,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
|
||||||
createTuningConfigWithMaxRowsPerSegment(2, true),
|
|
||||||
false
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -779,16 +823,15 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.HOUR,
|
Granularities.HOUR,
|
||||||
Granularities.MINUTE,
|
Granularities.MINUTE,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfig(2, 2, null, 2L, null, false, true),
|
createTuningConfig(2, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -826,17 +869,16 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfig(3, 2, null, 2L, null, true, true),
|
createTuningConfig(3, 2, null, 2L, null, true, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -873,17 +915,16 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfig(3, 2, null, 2L, null, false, true),
|
createTuningConfig(3, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -937,24 +978,37 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
|
final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null);
|
||||||
|
final List<String> columns = Arrays.asList("time", "dim", "val");
|
||||||
|
// ignore parse exception
|
||||||
|
final IndexTuningConfig tuningConfig = createTuningConfig(2, null, null, null, null, false, false);
|
||||||
|
|
||||||
// GranularitySpec.intervals and numShards must be null to verify reportParseException=false is respected both in
|
// GranularitySpec.intervals and numShards must be null to verify reportParseException=false is respected both in
|
||||||
// IndexTask.determineShardSpecs() and IndexTask.generateAndPublishSegments()
|
// IndexTask.determineShardSpecs() and IndexTask.generateAndPublishSegments()
|
||||||
final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec(
|
final IndexIngestionSpec parseExceptionIgnoreSpec;
|
||||||
useInputFormatApi,
|
if (useInputFormatApi) {
|
||||||
|
parseExceptionIgnoreSpec = createIngestionSpec(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
new CSVParseSpec(
|
timestampSpec,
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
DimensionsSpec.EMPTY,
|
DimensionsSpec.EMPTY,
|
||||||
|
new CsvInputFormat(columns, null, null, true, 0),
|
||||||
null,
|
null,
|
||||||
Arrays.asList("time", "dim", "val"),
|
|
||||||
true,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
createTuningConfig(2, null, null, null, null, false, false), // ignore parse exception,
|
tuningConfig,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
} else {
|
||||||
|
parseExceptionIgnoreSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
@ -987,27 +1041,39 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec(
|
final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null);
|
||||||
useInputFormatApi,
|
final List<String> columns = Arrays.asList("time", "dim", "val");
|
||||||
|
// report parse exception
|
||||||
|
final IndexTuningConfig tuningConfig = createTuningConfig(2, null, null, null, null, false, true);
|
||||||
|
final IndexIngestionSpec indexIngestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
indexIngestionSpec = createIngestionSpec(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
new CSVParseSpec(
|
timestampSpec,
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
DimensionsSpec.EMPTY,
|
DimensionsSpec.EMPTY,
|
||||||
|
new CsvInputFormat(columns, null, null, true, 0),
|
||||||
null,
|
null,
|
||||||
Arrays.asList("time", "dim", "val"),
|
|
||||||
true,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
createTuningConfig(2, null, null, null, null, false, true), // report parse exception
|
tuningConfig,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
} else {
|
||||||
|
indexIngestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
parseExceptionIgnoreSpec,
|
indexIngestionSpec,
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -1075,31 +1141,43 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
7
|
7
|
||||||
);
|
);
|
||||||
|
|
||||||
final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec(
|
final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null);
|
||||||
useInputFormatApi,
|
final DimensionsSpec dimensionsSpec = new DimensionsSpec(
|
||||||
jsonMapper,
|
|
||||||
tmpDir,
|
|
||||||
new JSONParseSpec(
|
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
new DimensionsSpec(
|
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
new StringDimensionSchema("dim"),
|
new StringDimensionSchema("dim"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new FloatDimensionSchema("dimFloat")
|
||||||
)
|
)
|
||||||
),
|
);
|
||||||
|
final IndexIngestionSpec ingestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
timestampSpec,
|
||||||
|
dimensionsSpec,
|
||||||
|
new JsonInputFormat(null, null),
|
||||||
null,
|
null,
|
||||||
null
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
tuningConfig,
|
tuningConfig,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
} else {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new JSONParseSpec(timestampSpec, dimensionsSpec, null, null),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
parseExceptionIgnoreSpec,
|
ingestionSpec,
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -1195,33 +1273,44 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
5
|
5
|
||||||
);
|
);
|
||||||
|
|
||||||
final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec(
|
final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null);
|
||||||
useInputFormatApi,
|
final DimensionsSpec dimensionsSpec = new DimensionsSpec(
|
||||||
jsonMapper,
|
|
||||||
tmpDir,
|
|
||||||
new CSVParseSpec(
|
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
new DimensionsSpec(
|
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
new StringDimensionSchema("dim"),
|
new StringDimensionSchema("dim"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new FloatDimensionSchema("dimFloat")
|
||||||
)
|
)
|
||||||
),
|
);
|
||||||
|
final List<String> columns = Arrays.asList("time", "dim", "dimLong", "dimFloat", "val");
|
||||||
|
final IndexIngestionSpec ingestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
timestampSpec,
|
||||||
|
dimensionsSpec,
|
||||||
|
new CsvInputFormat(columns, null, null, true, 0),
|
||||||
null,
|
null,
|
||||||
Arrays.asList("time", "dim", "dimLong", "dimFloat", "val"),
|
|
||||||
true,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
tuningConfig,
|
tuningConfig,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
} else {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(timestampSpec, dimensionsSpec, null, columns, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
parseExceptionIgnoreSpec,
|
ingestionSpec,
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -1308,33 +1397,44 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
5
|
5
|
||||||
);
|
);
|
||||||
|
|
||||||
final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec(
|
final TimestampSpec timestampSpec = new TimestampSpec("time", "auto", null);
|
||||||
useInputFormatApi,
|
final DimensionsSpec dimensionsSpec = new DimensionsSpec(
|
||||||
jsonMapper,
|
|
||||||
tmpDir,
|
|
||||||
new CSVParseSpec(
|
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
new DimensionsSpec(
|
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
new StringDimensionSchema("dim"),
|
new StringDimensionSchema("dim"),
|
||||||
new LongDimensionSchema("dimLong"),
|
new LongDimensionSchema("dimLong"),
|
||||||
new FloatDimensionSchema("dimFloat")
|
new FloatDimensionSchema("dimFloat")
|
||||||
)
|
)
|
||||||
),
|
);
|
||||||
|
final List<String> columns = Arrays.asList("time", "dim", "dimLong", "dimFloat", "val");
|
||||||
|
final IndexIngestionSpec ingestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
timestampSpec,
|
||||||
|
dimensionsSpec,
|
||||||
|
new CsvInputFormat(columns, null, null, true, 0),
|
||||||
null,
|
null,
|
||||||
Arrays.asList("time", "dim", "dimLong", "dimFloat", "val"),
|
|
||||||
true,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
tuningConfig,
|
tuningConfig,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
} else {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(timestampSpec, dimensionsSpec, null, columns, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
parseExceptionIgnoreSpec,
|
ingestionSpec,
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -1389,45 +1489,55 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
File tmpFile = File.createTempFile("druid", "index", tmpDir);
|
File tmpFile = File.createTempFile("druid", "index", tmpDir);
|
||||||
|
|
||||||
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
|
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
|
||||||
writer.write("time,,\n");
|
writer.write("ts,,\n");
|
||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
tmpFile = File.createTempFile("druid", "index", tmpDir);
|
tmpFile = File.createTempFile("druid", "index", tmpDir);
|
||||||
|
|
||||||
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
|
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
|
||||||
writer.write("time,dim,\n");
|
writer.write("ts,dim,\n");
|
||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
tmpFile = File.createTempFile("druid", "index", tmpDir);
|
tmpFile = File.createTempFile("druid", "index", tmpDir);
|
||||||
|
|
||||||
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
|
try (BufferedWriter writer = Files.newWriter(tmpFile, StandardCharsets.UTF_8)) {
|
||||||
writer.write("time,,val\n");
|
writer.write("ts,,val\n");
|
||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec(
|
// report parse exception
|
||||||
useInputFormatApi,
|
final IndexTuningConfig tuningConfig = createTuningConfig(2, 1, null, null, null, true, true);
|
||||||
|
final IndexIngestionSpec ingestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
new CSVParseSpec(
|
DEFAULT_TIMESTAMP_SPEC,
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
DimensionsSpec.EMPTY,
|
DimensionsSpec.EMPTY,
|
||||||
|
new CsvInputFormat(null, null, null, true, 0),
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
true,
|
tuningConfig,
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
|
||||||
createTuningConfig(2, 1, null, null, null, true, true), // report parse exception
|
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
} else {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(DEFAULT_TIMESTAMP_SPEC, DimensionsSpec.EMPTY, null, null, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
parseExceptionIgnoreSpec,
|
ingestionSpec,
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -1472,27 +1582,38 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
writer.write("2014-01-01T00:00:10Z,a,1\n");
|
||||||
}
|
}
|
||||||
|
|
||||||
final IndexIngestionSpec parseExceptionIgnoreSpec = createIngestionSpec(
|
final List<String> columns = Arrays.asList("ts", "", "");
|
||||||
useInputFormatApi,
|
// report parse exception
|
||||||
|
final IndexTuningConfig tuningConfig = createTuningConfig(2, null, null, null, null, false, true);
|
||||||
|
final IndexIngestionSpec ingestionSpec;
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
new CSVParseSpec(
|
DEFAULT_TIMESTAMP_SPEC,
|
||||||
new TimestampSpec("time", "auto", null),
|
|
||||||
DimensionsSpec.EMPTY,
|
DimensionsSpec.EMPTY,
|
||||||
|
new CsvInputFormat(columns, null, null, true, 0),
|
||||||
null,
|
null,
|
||||||
Arrays.asList("time", "", ""),
|
|
||||||
true,
|
|
||||||
0
|
|
||||||
),
|
|
||||||
null,
|
null,
|
||||||
createTuningConfig(2, null, null, null, null, false, true), // report parse exception
|
tuningConfig,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
|
} else {
|
||||||
|
ingestionSpec = createIngestionSpec(
|
||||||
|
jsonMapper,
|
||||||
|
tmpDir,
|
||||||
|
new CSVParseSpec(DEFAULT_TIMESTAMP_SPEC, DimensionsSpec.EMPTY, null, columns, true, 0),
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
tuningConfig,
|
||||||
|
false
|
||||||
|
);
|
||||||
|
}
|
||||||
|
|
||||||
IndexTask indexTask = new IndexTask(
|
IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
parseExceptionIgnoreSpec,
|
ingestionSpec,
|
||||||
null,
|
null,
|
||||||
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
AuthTestUtils.TEST_AUTHORIZER_MAPPER,
|
||||||
null,
|
null,
|
||||||
@ -1529,17 +1650,16 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
final IndexTask indexTask = new IndexTask(
|
final IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfig(3, 2, null, 2L, null, false, true),
|
createTuningConfig(3, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -1598,17 +1718,16 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
final IndexTask indexTask = new IndexTask(
|
final IndexTask indexTask = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
|
||||||
new UniformGranularitySpec(
|
new UniformGranularitySpec(
|
||||||
segmentGranularity,
|
segmentGranularity,
|
||||||
Granularities.DAY,
|
Granularities.DAY,
|
||||||
true,
|
true,
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
|
null,
|
||||||
createTuningConfig(3, 2, null, 2L, null, false, true),
|
createTuningConfig(3, 2, null, 2L, null, false, true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -1643,13 +1762,11 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
final IndexTask task = new IndexTask(
|
final IndexTask task = new IndexTask(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
createIngestionSpec(
|
createDefaultIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
jsonMapper,
|
jsonMapper,
|
||||||
tmpDir,
|
tmpDir,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null,
|
|
||||||
createTuningConfigWithPartitionsSpec(new SingleDimensionPartitionsSpec(null, 1, null, false), true),
|
createTuningConfigWithPartitionsSpec(new SingleDimensionPartitionsSpec(null, 1, null, false), true),
|
||||||
false
|
false
|
||||||
),
|
),
|
||||||
@ -1762,20 +1879,58 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
private IndexIngestionSpec createDefaultIngestionSpec(
|
||||||
|
ObjectMapper objectMapper,
|
||||||
|
File baseDir,
|
||||||
|
@Nullable GranularitySpec granularitySpec,
|
||||||
|
@Nullable TransformSpec transformSpec,
|
||||||
|
IndexTuningConfig tuningConfig,
|
||||||
|
boolean appendToExisting
|
||||||
|
)
|
||||||
|
{
|
||||||
|
if (useInputFormatApi) {
|
||||||
|
return createIngestionSpec(
|
||||||
|
objectMapper,
|
||||||
|
baseDir,
|
||||||
|
DEFAULT_TIMESTAMP_SPEC,
|
||||||
|
DEFAULT_DIMENSIONS_SPEC,
|
||||||
|
DEFAULT_INPUT_FORMAT,
|
||||||
|
transformSpec,
|
||||||
|
granularitySpec,
|
||||||
|
tuningConfig,
|
||||||
|
appendToExisting
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
return createIngestionSpec(
|
||||||
|
objectMapper,
|
||||||
|
baseDir,
|
||||||
|
DEFAULT_PARSE_SPEC,
|
||||||
|
transformSpec,
|
||||||
|
granularitySpec,
|
||||||
|
tuningConfig,
|
||||||
|
appendToExisting
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
static IndexIngestionSpec createIngestionSpec(
|
static IndexIngestionSpec createIngestionSpec(
|
||||||
ObjectMapper objectMapper,
|
ObjectMapper objectMapper,
|
||||||
File baseDir,
|
File baseDir,
|
||||||
@Nullable ParseSpec parseSpec,
|
@Nullable ParseSpec parseSpec,
|
||||||
GranularitySpec granularitySpec,
|
@Nullable TransformSpec transformSpec,
|
||||||
|
@Nullable GranularitySpec granularitySpec,
|
||||||
IndexTuningConfig tuningConfig,
|
IndexTuningConfig tuningConfig,
|
||||||
boolean appendToExisting
|
boolean appendToExisting
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return createIngestionSpec(
|
return createIngestionSpec(
|
||||||
false,
|
|
||||||
objectMapper,
|
objectMapper,
|
||||||
baseDir,
|
baseDir,
|
||||||
parseSpec,
|
parseSpec,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
transformSpec,
|
||||||
granularitySpec,
|
granularitySpec,
|
||||||
tuningConfig,
|
tuningConfig,
|
||||||
appendToExisting
|
appendToExisting
|
||||||
@ -1783,21 +1938,25 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
}
|
}
|
||||||
|
|
||||||
static IndexIngestionSpec createIngestionSpec(
|
static IndexIngestionSpec createIngestionSpec(
|
||||||
boolean useInputFormatApi,
|
|
||||||
ObjectMapper objectMapper,
|
ObjectMapper objectMapper,
|
||||||
File baseDir,
|
File baseDir,
|
||||||
@Nullable ParseSpec parseSpec,
|
TimestampSpec timestampSpec,
|
||||||
GranularitySpec granularitySpec,
|
DimensionsSpec dimensionsSpec,
|
||||||
|
InputFormat inputFormat,
|
||||||
|
@Nullable TransformSpec transformSpec,
|
||||||
|
@Nullable GranularitySpec granularitySpec,
|
||||||
IndexTuningConfig tuningConfig,
|
IndexTuningConfig tuningConfig,
|
||||||
boolean appendToExisting
|
boolean appendToExisting
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
return createIngestionSpec(
|
return createIngestionSpec(
|
||||||
useInputFormatApi,
|
|
||||||
objectMapper,
|
objectMapper,
|
||||||
baseDir,
|
baseDir,
|
||||||
parseSpec,
|
null,
|
||||||
TransformSpec.NONE,
|
timestampSpec,
|
||||||
|
dimensionsSpec,
|
||||||
|
inputFormat,
|
||||||
|
transformSpec,
|
||||||
granularitySpec,
|
granularitySpec,
|
||||||
tuningConfig,
|
tuningConfig,
|
||||||
appendToExisting
|
appendToExisting
|
||||||
@ -1805,22 +1964,25 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
}
|
}
|
||||||
|
|
||||||
private static IndexIngestionSpec createIngestionSpec(
|
private static IndexIngestionSpec createIngestionSpec(
|
||||||
boolean useInputFormatApi,
|
|
||||||
ObjectMapper objectMapper,
|
ObjectMapper objectMapper,
|
||||||
File baseDir,
|
File baseDir,
|
||||||
@Nullable ParseSpec parseSpec,
|
@Nullable ParseSpec parseSpec,
|
||||||
TransformSpec transformSpec,
|
@Nullable TimestampSpec timestampSpec,
|
||||||
GranularitySpec granularitySpec,
|
@Nullable DimensionsSpec dimensionsSpec,
|
||||||
|
@Nullable InputFormat inputFormat,
|
||||||
|
@Nullable TransformSpec transformSpec,
|
||||||
|
@Nullable GranularitySpec granularitySpec,
|
||||||
IndexTuningConfig tuningConfig,
|
IndexTuningConfig tuningConfig,
|
||||||
boolean appendToExisting
|
boolean appendToExisting
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (useInputFormatApi) {
|
if (inputFormat != null) {
|
||||||
|
Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec");
|
||||||
return new IndexIngestionSpec(
|
return new IndexIngestionSpec(
|
||||||
new DataSchema(
|
new DataSchema(
|
||||||
"test",
|
"test",
|
||||||
parseSpec == null ? DEFAULT_TIMESTAMP_SPEC : parseSpec.getTimestampSpec(),
|
Preconditions.checkNotNull(timestampSpec, "timestampSpec"),
|
||||||
parseSpec == null ? DEFAULT_DIMENSIONS_SPEC : parseSpec.getDimensionsSpec(),
|
Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"),
|
||||||
new AggregatorFactory[]{
|
new AggregatorFactory[]{
|
||||||
new LongSumAggregatorFactory("val", "val")
|
new LongSumAggregatorFactory("val", "val")
|
||||||
},
|
},
|
||||||
@ -1834,7 +1996,7 @@ public class IndexTaskTest extends IngestionTestBase
|
|||||||
new IndexIOConfig(
|
new IndexIOConfig(
|
||||||
null,
|
null,
|
||||||
new LocalInputSource(baseDir, "druid*"),
|
new LocalInputSource(baseDir, "druid*"),
|
||||||
parseSpec == null ? DEFAULT_INPUT_FORMAT : parseSpec.toInputFormat(),
|
inputFormat,
|
||||||
appendToExisting
|
appendToExisting
|
||||||
),
|
),
|
||||||
tuningConfig
|
tuningConfig
|
||||||
|
@ -19,10 +19,14 @@
|
|||||||
|
|
||||||
package org.apache.druid.indexing.common.task.batch.parallel;
|
package org.apache.druid.indexing.common.task.batch.parallel;
|
||||||
|
|
||||||
|
import com.google.common.base.Preconditions;
|
||||||
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.MaxSizeSplitHintSpec;
|
import org.apache.druid.data.input.MaxSizeSplitHintSpec;
|
||||||
|
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||||
import org.apache.druid.data.input.impl.LocalInputSource;
|
import org.apache.druid.data.input.impl.LocalInputSource;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
import org.apache.druid.data.input.impl.ParseSpec;
|
||||||
import org.apache.druid.data.input.impl.StringInputRowParser;
|
import org.apache.druid.data.input.impl.StringInputRowParser;
|
||||||
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
import org.apache.druid.indexer.TaskState;
|
import org.apache.druid.indexer.TaskState;
|
||||||
import org.apache.druid.indexer.TaskStatus;
|
import org.apache.druid.indexer.TaskStatus;
|
||||||
import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec;
|
import org.apache.druid.indexer.partitions.DimensionBasedPartitionsSpec;
|
||||||
@ -55,6 +59,7 @@ import org.apache.druid.timeline.DataSegment;
|
|||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
@ -82,8 +87,16 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
|||||||
this.useInputFormatApi = useInputFormatApi;
|
this.useInputFormatApi = useInputFormatApi;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
boolean isUseInputFormatApi()
|
||||||
|
{
|
||||||
|
return useInputFormatApi;
|
||||||
|
}
|
||||||
|
|
||||||
Set<DataSegment> runTestTask(
|
Set<DataSegment> runTestTask(
|
||||||
ParseSpec parseSpec,
|
@Nullable TimestampSpec timestampSpec,
|
||||||
|
@Nullable DimensionsSpec dimensionsSpec,
|
||||||
|
@Nullable InputFormat inputFormat,
|
||||||
|
@Nullable ParseSpec parseSpec,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
File inputDir,
|
File inputDir,
|
||||||
String filter,
|
String filter,
|
||||||
@ -93,6 +106,9 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
|||||||
)
|
)
|
||||||
{
|
{
|
||||||
final ParallelIndexSupervisorTask task = newTask(
|
final ParallelIndexSupervisorTask task = newTask(
|
||||||
|
timestampSpec,
|
||||||
|
dimensionsSpec,
|
||||||
|
inputFormat,
|
||||||
parseSpec,
|
parseSpec,
|
||||||
interval,
|
interval,
|
||||||
inputDir,
|
inputDir,
|
||||||
@ -108,7 +124,10 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
|||||||
}
|
}
|
||||||
|
|
||||||
private ParallelIndexSupervisorTask newTask(
|
private ParallelIndexSupervisorTask newTask(
|
||||||
ParseSpec parseSpec,
|
@Nullable TimestampSpec timestampSpec,
|
||||||
|
@Nullable DimensionsSpec dimensionsSpec,
|
||||||
|
@Nullable InputFormat inputFormat,
|
||||||
|
@Nullable ParseSpec parseSpec,
|
||||||
Interval interval,
|
Interval interval,
|
||||||
File inputDir,
|
File inputDir,
|
||||||
String filter,
|
String filter,
|
||||||
@ -154,17 +173,18 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
|||||||
final ParallelIndexIngestionSpec ingestionSpec;
|
final ParallelIndexIngestionSpec ingestionSpec;
|
||||||
|
|
||||||
if (useInputFormatApi) {
|
if (useInputFormatApi) {
|
||||||
|
Preconditions.checkArgument(parseSpec == null);
|
||||||
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
|
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
|
||||||
null,
|
null,
|
||||||
new LocalInputSource(inputDir, filter),
|
new LocalInputSource(inputDir, filter),
|
||||||
parseSpec.toInputFormat(),
|
inputFormat,
|
||||||
false
|
false
|
||||||
);
|
);
|
||||||
ingestionSpec = new ParallelIndexIngestionSpec(
|
ingestionSpec = new ParallelIndexIngestionSpec(
|
||||||
new DataSchema(
|
new DataSchema(
|
||||||
"dataSource",
|
"dataSource",
|
||||||
parseSpec.getTimestampSpec(),
|
timestampSpec,
|
||||||
parseSpec.getDimensionsSpec(),
|
dimensionsSpec,
|
||||||
new AggregatorFactory[]{
|
new AggregatorFactory[]{
|
||||||
new LongSumAggregatorFactory("val", "val")
|
new LongSumAggregatorFactory("val", "val")
|
||||||
},
|
},
|
||||||
@ -175,6 +195,7 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
|
|||||||
tuningConfig
|
tuningConfig
|
||||||
);
|
);
|
||||||
} else {
|
} else {
|
||||||
|
Preconditions.checkArgument(inputFormat == null);
|
||||||
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
|
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
|
||||||
new LocalFirehoseFactory(inputDir, filter, null),
|
new LocalFirehoseFactory(inputDir, filter, null),
|
||||||
false
|
false
|
||||||
|
@ -38,6 +38,7 @@ import org.apache.druid.client.indexing.NoopIndexingServiceClient;
|
|||||||
import org.apache.druid.client.indexing.TaskStatusResponse;
|
import org.apache.druid.client.indexing.TaskStatusResponse;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.CSVParseSpec;
|
import org.apache.druid.data.input.impl.CSVParseSpec;
|
||||||
|
import org.apache.druid.data.input.impl.CsvInputFormat;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
import org.apache.druid.data.input.impl.ParseSpec;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
@ -131,7 +132,13 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
|
|||||||
false,
|
false,
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
static final InputFormat DEFAULT_INPUT_FORMAT = DEFAULT_PARSE_SPEC.toInputFormat();
|
static final InputFormat DEFAULT_INPUT_FORMAT = new CsvInputFormat(
|
||||||
|
Arrays.asList("ts", "dim", "val"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
0
|
||||||
|
);
|
||||||
static final ParallelIndexTuningConfig DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING = new ParallelIndexTuningConfig(
|
static final ParallelIndexTuningConfig DEFAULT_TUNING_CONFIG_FOR_PARALLEL_INDEXING = new ParallelIndexTuningConfig(
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
@ -20,7 +20,9 @@
|
|||||||
package org.apache.druid.indexing.common.task.batch.parallel;
|
package org.apache.druid.indexing.common.task.batch.parallel;
|
||||||
|
|
||||||
import com.google.common.collect.ImmutableList;
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.CSVParseSpec;
|
import org.apache.druid.data.input.impl.CSVParseSpec;
|
||||||
|
import org.apache.druid.data.input.impl.CsvInputFormat;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
import org.apache.druid.data.input.impl.ParseSpec;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
@ -54,18 +56,25 @@ import java.util.Set;
|
|||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPhaseParallelIndexingTest
|
public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPhaseParallelIndexingTest
|
||||||
{
|
{
|
||||||
|
private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec("ts", "auto", null);
|
||||||
|
private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec(
|
||||||
|
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2"))
|
||||||
|
);
|
||||||
private static final ParseSpec PARSE_SPEC = new CSVParseSpec(
|
private static final ParseSpec PARSE_SPEC = new CSVParseSpec(
|
||||||
new TimestampSpec(
|
TIMESTAMP_SPEC,
|
||||||
"ts",
|
DIMENSIONS_SPEC,
|
||||||
"auto",
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim1", "dim2"))),
|
|
||||||
null,
|
null,
|
||||||
Arrays.asList("ts", "dim1", "dim2", "val"),
|
Arrays.asList("ts", "dim1", "dim2", "val"),
|
||||||
false,
|
false,
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
|
private static final InputFormat INPUT_FORMAT = new CsvInputFormat(
|
||||||
|
Arrays.asList("ts", "dim1", "dim2", "val"),
|
||||||
|
null,
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
0
|
||||||
|
);
|
||||||
private static final int MAX_NUM_CONCURRENT_SUB_TASKS = 2;
|
private static final int MAX_NUM_CONCURRENT_SUB_TASKS = 2;
|
||||||
private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M");
|
private static final Interval INTERVAL_TO_INDEX = Intervals.of("2017-12/P1M");
|
||||||
|
|
||||||
@ -112,7 +121,25 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
@Test
|
@Test
|
||||||
public void testRun() throws Exception
|
public void testRun() throws Exception
|
||||||
{
|
{
|
||||||
final Set<DataSegment> publishedSegments = runTestTask(
|
final Set<DataSegment> publishedSegments;
|
||||||
|
if (isUseInputFormatApi()) {
|
||||||
|
publishedSegments = runTestTask(
|
||||||
|
TIMESTAMP_SPEC,
|
||||||
|
DIMENSIONS_SPEC,
|
||||||
|
INPUT_FORMAT,
|
||||||
|
null,
|
||||||
|
INTERVAL_TO_INDEX,
|
||||||
|
inputDir,
|
||||||
|
"test_*",
|
||||||
|
new HashedPartitionsSpec(null, 2, ImmutableList.of("dim1", "dim2")),
|
||||||
|
MAX_NUM_CONCURRENT_SUB_TASKS,
|
||||||
|
TaskState.SUCCESS
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
publishedSegments = runTestTask(
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
PARSE_SPEC,
|
PARSE_SPEC,
|
||||||
INTERVAL_TO_INDEX,
|
INTERVAL_TO_INDEX,
|
||||||
inputDir,
|
inputDir,
|
||||||
@ -121,6 +148,7 @@ public class HashPartitionMultiPhaseParallelIndexingTest extends AbstractMultiPh
|
|||||||
MAX_NUM_CONCURRENT_SUB_TASKS,
|
MAX_NUM_CONCURRENT_SUB_TASKS,
|
||||||
TaskState.SUCCESS
|
TaskState.SUCCESS
|
||||||
);
|
);
|
||||||
|
}
|
||||||
assertHashedPartition(publishedSegments);
|
assertHashedPartition(publishedSegments);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@ -25,7 +25,9 @@ import com.google.common.collect.ImmutableList;
|
|||||||
import com.google.common.collect.Multimap;
|
import com.google.common.collect.Multimap;
|
||||||
import com.google.common.collect.SetMultimap;
|
import com.google.common.collect.SetMultimap;
|
||||||
import org.apache.druid.common.config.NullValueHandlingConfig;
|
import org.apache.druid.common.config.NullValueHandlingConfig;
|
||||||
|
import org.apache.druid.data.input.InputFormat;
|
||||||
import org.apache.druid.data.input.impl.CSVParseSpec;
|
import org.apache.druid.data.input.impl.CSVParseSpec;
|
||||||
|
import org.apache.druid.data.input.impl.CsvInputFormat;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||||
import org.apache.druid.data.input.impl.ParseSpec;
|
import org.apache.druid.data.input.impl.ParseSpec;
|
||||||
import org.apache.druid.data.input.impl.TimestampSpec;
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
@ -79,18 +81,25 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
private static final String LIST_DELIMITER = "|";
|
private static final String LIST_DELIMITER = "|";
|
||||||
private static final List<String> DIMS = ImmutableList.of(DIM1, DIM2);
|
private static final List<String> DIMS = ImmutableList.of(DIM1, DIM2);
|
||||||
private static final String TEST_FILE_NAME_PREFIX = "test_";
|
private static final String TEST_FILE_NAME_PREFIX = "test_";
|
||||||
|
private static final TimestampSpec TIMESTAMP_SPEC = new TimestampSpec(TIME, "auto", null);
|
||||||
|
private static final DimensionsSpec DIMENSIONS_SPEC = new DimensionsSpec(
|
||||||
|
DimensionsSpec.getDefaultSchemas(Arrays.asList(TIME, DIM1, DIM2))
|
||||||
|
);
|
||||||
private static final ParseSpec PARSE_SPEC = new CSVParseSpec(
|
private static final ParseSpec PARSE_SPEC = new CSVParseSpec(
|
||||||
new TimestampSpec(
|
TIMESTAMP_SPEC,
|
||||||
TIME,
|
DIMENSIONS_SPEC,
|
||||||
"auto",
|
|
||||||
null
|
|
||||||
),
|
|
||||||
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList(TIME, DIM1, DIM2))),
|
|
||||||
LIST_DELIMITER,
|
LIST_DELIMITER,
|
||||||
Arrays.asList(TIME, DIM1, DIM2, "val"),
|
Arrays.asList(TIME, DIM1, DIM2, "val"),
|
||||||
false,
|
false,
|
||||||
0
|
0
|
||||||
);
|
);
|
||||||
|
private static final InputFormat INPUT_FORMAT = new CsvInputFormat(
|
||||||
|
Arrays.asList(TIME, DIM1, DIM2, "val"),
|
||||||
|
LIST_DELIMITER,
|
||||||
|
false,
|
||||||
|
false,
|
||||||
|
0
|
||||||
|
);
|
||||||
|
|
||||||
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}, maxNumConcurrentSubTasks={2}, useMultiValueDim={3}")
|
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}, maxNumConcurrentSubTasks={2}, useMultiValueDim={3}")
|
||||||
public static Iterable<Object[]> constructorFeeder()
|
public static Iterable<Object[]> constructorFeeder()
|
||||||
@ -192,7 +201,30 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
public void createsCorrectRangePartitions() throws Exception
|
public void createsCorrectRangePartitions() throws Exception
|
||||||
{
|
{
|
||||||
int targetRowsPerSegment = NUM_ROW / DIM_FILE_CARDINALITY / NUM_PARTITION;
|
int targetRowsPerSegment = NUM_ROW / DIM_FILE_CARDINALITY / NUM_PARTITION;
|
||||||
final Set<DataSegment> publishedSegments = runTestTask(
|
final Set<DataSegment> publishedSegments;
|
||||||
|
if (isUseInputFormatApi()) {
|
||||||
|
publishedSegments = runTestTask(
|
||||||
|
TIMESTAMP_SPEC,
|
||||||
|
DIMENSIONS_SPEC,
|
||||||
|
INPUT_FORMAT,
|
||||||
|
null,
|
||||||
|
INTERVAL_TO_INDEX,
|
||||||
|
inputDir,
|
||||||
|
TEST_FILE_NAME_PREFIX + "*",
|
||||||
|
new SingleDimensionPartitionsSpec(
|
||||||
|
targetRowsPerSegment,
|
||||||
|
null,
|
||||||
|
DIM1,
|
||||||
|
false
|
||||||
|
),
|
||||||
|
maxNumConcurrentSubTasks,
|
||||||
|
useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS
|
||||||
|
);
|
||||||
|
} else {
|
||||||
|
publishedSegments = runTestTask(
|
||||||
|
null,
|
||||||
|
null,
|
||||||
|
null,
|
||||||
PARSE_SPEC,
|
PARSE_SPEC,
|
||||||
INTERVAL_TO_INDEX,
|
INTERVAL_TO_INDEX,
|
||||||
inputDir,
|
inputDir,
|
||||||
@ -206,6 +238,7 @@ public class RangePartitionMultiPhaseParallelIndexingTest extends AbstractMultiP
|
|||||||
maxNumConcurrentSubTasks,
|
maxNumConcurrentSubTasks,
|
||||||
useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS
|
useMultivalueDim ? TaskState.FAILED : TaskState.SUCCESS
|
||||||
);
|
);
|
||||||
|
}
|
||||||
|
|
||||||
if (!useMultivalueDim) {
|
if (!useMultivalueDim) {
|
||||||
assertRangePartitions(publishedSegments);
|
assertRangePartitions(publishedSegments);
|
||||||
|
@ -19,8 +19,6 @@
|
|||||||
|
|
||||||
package org.apache.druid.indexing.seekablestream;
|
package org.apache.druid.indexing.seekablestream;
|
||||||
|
|
||||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
|
||||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
|
||||||
import com.fasterxml.jackson.core.type.TypeReference;
|
import com.fasterxml.jackson.core.type.TypeReference;
|
||||||
import com.fasterxml.jackson.databind.ObjectMapper;
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||||
@ -87,7 +85,6 @@ import org.easymock.EasyMockSupport;
|
|||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.lang.reflect.InvocationTargetException;
|
import java.lang.reflect.InvocationTargetException;
|
||||||
@ -149,12 +146,12 @@ public class SeekableStreamIndexTaskTestBase extends EasyMockSupport
|
|||||||
|
|
||||||
static {
|
static {
|
||||||
OBJECT_MAPPER = new TestUtils().getTestObjectMapper();
|
OBJECT_MAPPER = new TestUtils().getTestObjectMapper();
|
||||||
OBJECT_MAPPER.registerSubtypes(new NamedType(UnimplementedInputFormatJsonParseSpec.class, "json"));
|
OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json"));
|
||||||
OLD_DATA_SCHEMA = new DataSchema(
|
OLD_DATA_SCHEMA = new DataSchema(
|
||||||
"test_ds",
|
"test_ds",
|
||||||
OBJECT_MAPPER.convertValue(
|
OBJECT_MAPPER.convertValue(
|
||||||
new StringInputRowParser(
|
new StringInputRowParser(
|
||||||
new UnimplementedInputFormatJsonParseSpec(
|
new JSONParseSpec(
|
||||||
new TimestampSpec("timestamp", "iso", null),
|
new TimestampSpec("timestamp", "iso", null),
|
||||||
new DimensionsSpec(
|
new DimensionsSpec(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
@ -443,25 +440,4 @@ public class SeekableStreamIndexTaskTestBase extends EasyMockSupport
|
|||||||
return segmentDescriptor;
|
return segmentDescriptor;
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class UnimplementedInputFormatJsonParseSpec extends JSONParseSpec
|
|
||||||
{
|
|
||||||
@JsonCreator
|
|
||||||
private UnimplementedInputFormatJsonParseSpec(
|
|
||||||
@JsonProperty("timestampSpec") TimestampSpec timestampSpec,
|
|
||||||
@JsonProperty("dimensionsSpec") DimensionsSpec dimensionsSpec,
|
|
||||||
@JsonProperty("flattenSpec") JSONPathSpec flattenSpec,
|
|
||||||
@JsonProperty("featureSpec") Map<String, Boolean> featureSpec
|
|
||||||
)
|
|
||||||
{
|
|
||||||
super(timestampSpec, dimensionsSpec, flattenSpec, featureSpec);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
@Override
|
|
||||||
public InputFormat toInputFormat()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
@ -22,7 +22,6 @@ package org.apache.druid.indexing.seekablestream;
|
|||||||
import com.google.common.collect.Iterables;
|
import com.google.common.collect.Iterables;
|
||||||
import org.apache.druid.data.input.InputEntity;
|
import org.apache.druid.data.input.InputEntity;
|
||||||
import org.apache.druid.data.input.InputEntityReader;
|
import org.apache.druid.data.input.InputEntityReader;
|
||||||
import org.apache.druid.data.input.InputFormat;
|
|
||||||
import org.apache.druid.data.input.InputRow;
|
import org.apache.druid.data.input.InputRow;
|
||||||
import org.apache.druid.data.input.InputRowSchema;
|
import org.apache.druid.data.input.InputRowSchema;
|
||||||
import org.apache.druid.data.input.impl.DimensionsSpec;
|
import org.apache.druid.data.input.impl.DimensionsSpec;
|
||||||
@ -63,7 +62,12 @@ public class StreamChunkParserTest
|
|||||||
public void testWithParserAndNullInputformatParseProperly() throws IOException
|
public void testWithParserAndNullInputformatParseProperly() throws IOException
|
||||||
{
|
{
|
||||||
final InputRowParser<ByteBuffer> parser = new StringInputRowParser(
|
final InputRowParser<ByteBuffer> parser = new StringInputRowParser(
|
||||||
new NotConvertibleToInputFormatParseSpec(),
|
new JSONParseSpec(
|
||||||
|
TIMESTAMP_SPEC,
|
||||||
|
DimensionsSpec.EMPTY,
|
||||||
|
JSONPathSpec.DEFAULT,
|
||||||
|
Collections.emptyMap()
|
||||||
|
),
|
||||||
StringUtils.UTF8_STRING
|
StringUtils.UTF8_STRING
|
||||||
);
|
);
|
||||||
final StreamChunkParser chunkParser = new StreamChunkParser(
|
final StreamChunkParser chunkParser = new StreamChunkParser(
|
||||||
@ -109,7 +113,12 @@ public class StreamChunkParserTest
|
|||||||
public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws IOException
|
public void testBothParserAndInputFormatParseProperlyUsingInputFormat() throws IOException
|
||||||
{
|
{
|
||||||
final InputRowParser<ByteBuffer> parser = new StringInputRowParser(
|
final InputRowParser<ByteBuffer> parser = new StringInputRowParser(
|
||||||
new NotConvertibleToInputFormatParseSpec(),
|
new JSONParseSpec(
|
||||||
|
TIMESTAMP_SPEC,
|
||||||
|
DimensionsSpec.EMPTY,
|
||||||
|
JSONPathSpec.DEFAULT,
|
||||||
|
Collections.emptyMap()
|
||||||
|
),
|
||||||
StringUtils.UTF8_STRING
|
StringUtils.UTF8_STRING
|
||||||
);
|
);
|
||||||
final TrackingJsonInputFormat inputFormat = new TrackingJsonInputFormat(
|
final TrackingJsonInputFormat inputFormat = new TrackingJsonInputFormat(
|
||||||
@ -138,25 +147,6 @@ public class StreamChunkParserTest
|
|||||||
Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met")));
|
Assert.assertEquals("val2", Iterables.getOnlyElement(row.getDimension("met")));
|
||||||
}
|
}
|
||||||
|
|
||||||
private static class NotConvertibleToInputFormatParseSpec extends JSONParseSpec
|
|
||||||
{
|
|
||||||
private NotConvertibleToInputFormatParseSpec()
|
|
||||||
{
|
|
||||||
super(
|
|
||||||
TIMESTAMP_SPEC,
|
|
||||||
DimensionsSpec.EMPTY,
|
|
||||||
JSONPathSpec.DEFAULT,
|
|
||||||
Collections.emptyMap()
|
|
||||||
);
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public InputFormat toInputFormat()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
}
|
|
||||||
|
|
||||||
private static class TrackingJsonInputFormat extends JsonInputFormat
|
private static class TrackingJsonInputFormat extends JsonInputFormat
|
||||||
{
|
{
|
||||||
private boolean used;
|
private boolean used;
|
||||||
|
@ -1027,9 +1027,7 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
|
|||||||
true,
|
true,
|
||||||
minimumMessageTime,
|
minimumMessageTime,
|
||||||
maximumMessageTime,
|
maximumMessageTime,
|
||||||
ioConfig.getInputFormat(
|
ioConfig.getInputFormat()
|
||||||
getDataSchema().getParser() == null ? null : getDataSchema().getParser().getParseSpec()
|
|
||||||
)
|
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
};
|
};
|
||||||
|
Loading…
x
Reference in New Issue
Block a user