add DataSchema.Builder to tidy stuff up a bit (#17065)

* add DataSchema.Builder to tidy stuff up a bit

* fixes

* fixes

* more style fixes

* review stuff
This commit is contained in:
Clint Wylie 2024-09-15 11:18:34 -07:00 committed by GitHub
parent 6ed8632420
commit aa6336c5cf
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
64 changed files with 1527 additions and 1582 deletions

View File

@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.Tasks; import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
@ -66,18 +65,19 @@ public class K8sTestUtils
null, null,
null, null,
new IndexTask.IndexIngestionSpec( new IndexTask.IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, .withAggregators(new DoubleSumAggregatorFactory("met", "met"))
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
null, Granularities.DAY,
ImmutableList.of(Intervals.of("2010-01-01/P2D")) null,
), ImmutableList.of(Intervals.of("2010-01-01/P2D"))
null )
), )
.build(),
new IndexTask.IndexIOConfig( new IndexTask.IndexIOConfig(
new LocalInputSource(new File("lol"), "rofl"), new LocalInputSource(new File("lol"), "rofl"),
new NoopInputFormat(), new NoopInputFormat(),

View File

@ -48,7 +48,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.ChatHandlerProvider;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -211,14 +210,13 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
); );
// generate DataSchema // generate DataSchema
DataSchema dataSchema = new DataSchema( DataSchema dataSchema = DataSchema.builder()
dataSourceName, .withDataSource(dataSourceName)
parser, .withParserMap(parser)
aggregators, .withAggregators(aggregators)
granularitySpec, .withGranularity(granularitySpec)
TransformSpec.NONE, .withObjectMapper(objectMapper)
objectMapper .build();
);
// generate DatasourceIngestionSpec // generate DatasourceIngestionSpec
DatasourceIngestionSpec datasourceIngestionSpec = new DatasourceIngestionSpec( DatasourceIngestionSpec datasourceIngestionSpec = new DatasourceIngestionSpec(

View File

@ -53,7 +53,6 @@ import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig; import org.apache.druid.segment.metadata.CentralizedDatasourceSchemaConfig;
import org.apache.druid.segment.metadata.SegmentSchemaManager; import org.apache.druid.segment.metadata.SegmentSchemaManager;
import org.apache.druid.segment.realtime.ChatHandlerProvider; import org.apache.druid.segment.realtime.ChatHandlerProvider;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.security.AuthorizerMapper; import org.apache.druid.server.security.AuthorizerMapper;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
@ -237,14 +236,10 @@ public class MaterializedViewSupervisorTest
Map<Interval, HadoopIndexTask> runningTasks = runningTasksPair.lhs; Map<Interval, HadoopIndexTask> runningTasks = runningTasksPair.lhs;
Map<Interval, String> runningVersion = runningTasksPair.rhs; Map<Interval, String> runningVersion = runningTasksPair.rhs;
DataSchema dataSchema = new DataSchema( DataSchema dataSchema = DataSchema.builder()
"test_datasource", .withDataSource("test_datasource")
null, .withObjectMapper(objectMapper)
null, .build();
null,
TransformSpec.NONE,
objectMapper
);
HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(new HashMap<>(), null, null); HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(new HashMap<>(), null, null);
HadoopIngestionSpec spec = new HadoopIngestionSpec(dataSchema, hadoopIOConfig, null); HadoopIngestionSpec spec = new HadoopIngestionSpec(dataSchema, hadoopIOConfig, null);
HadoopIndexTask task1 = new HadoopIndexTask( HadoopIndexTask task1 = new HadoopIndexTask(

View File

@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
@ -44,7 +43,6 @@ import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.java.util.metrics.StubServiceEmitter;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
@ -102,16 +100,19 @@ public class RabbitStreamSupervisorTest extends EasyMockSupport
dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim1"));
dimensions.add(StringDimensionSchema.create("dim2")); dimensions.add(StringDimensionSchema.create("dim2"));
return new DataSchema( return DataSchema.builder()
dataSource, .withDataSource(dataSource)
new TimestampSpec("timestamp", "iso", null), .withTimestamp(new TimestampSpec("timestamp", "iso", null))
new DimensionsSpec(dimensions), .withDimensions(dimensions)
new AggregatorFactory[] {new CountAggregatorFactory("rows")}, .withAggregators(new CountAggregatorFactory("rows"))
new UniformGranularitySpec( .withGranularity(
Granularities.HOUR, new UniformGranularitySpec(
Granularities.NONE, Granularities.HOUR,
ImmutableList.of()), Granularities.NONE,
null); ImmutableList.of()
)
)
.build();
} }
@BeforeClass @BeforeClass

View File

@ -92,7 +92,6 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerFactory; import org.apache.druid.query.QueryRunnerFactory;
import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.filter.SelectorDimFilter;
@ -1262,28 +1261,27 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
final KafkaIndexTask task = createTask( final KafkaIndexTask task = createTask(
null, null,
new DataSchema( DataSchema.builder()
"test_ds", .withDataSource("test_ds")
new TimestampSpec("timestamp", "iso", null), .withTimestamp(new TimestampSpec("timestamp", "iso", null))
new DimensionsSpec( .withDimensions(
Arrays.asList( new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1"), new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim1t"), new StringDimensionSchema("dim2"),
new StringDimensionSchema("dim2"), new LongDimensionSchema("dimLong"),
new LongDimensionSchema("dimLong"), new FloatDimensionSchema("dimFloat"),
new FloatDimensionSchema("dimFloat"), new StringDimensionSchema("kafka.topic"),
new StringDimensionSchema("kafka.topic"), new LongDimensionSchema("kafka.offset"),
new LongDimensionSchema("kafka.offset"), new StringDimensionSchema("kafka.header.encoding")
new StringDimensionSchema("kafka.header.encoding") )
) .withAggregators(
), new DoubleSumAggregatorFactory("met1sum", "met1"),
new AggregatorFactory[]{ new CountAggregatorFactory("rows")
new DoubleSumAggregatorFactory("met1sum", "met1"), )
new CountAggregatorFactory("rows") .withGranularity(
}, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), )
null .build(),
),
new KafkaIndexTaskIOConfig( new KafkaIndexTaskIOConfig(
0, 0,
"sequence0", "sequence0",
@ -1337,26 +1335,25 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
final KafkaIndexTask task = createTask( final KafkaIndexTask task = createTask(
null, null,
new DataSchema( DataSchema.builder()
"test_ds", .withDataSource("test_ds")
new TimestampSpec("timestamp", "iso", null), .withTimestamp(new TimestampSpec("timestamp", "iso", null))
new DimensionsSpec( .withDimensions(
Arrays.asList( new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1"), new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim1t"), new StringDimensionSchema("dim2"),
new StringDimensionSchema("dim2"), new LongDimensionSchema("dimLong"),
new LongDimensionSchema("dimLong"), new FloatDimensionSchema("dimFloat"),
new FloatDimensionSchema("dimFloat"), new StringDimensionSchema("kafka.testheader.encoding")
new StringDimensionSchema("kafka.testheader.encoding") )
) .withAggregators(
), new DoubleSumAggregatorFactory("met1sum", "met1"),
new AggregatorFactory[]{ new CountAggregatorFactory("rows")
new DoubleSumAggregatorFactory("met1sum", "met1"), )
new CountAggregatorFactory("rows") .withGranularity(
}, new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), )
null .build(),
),
new KafkaIndexTaskIOConfig( new KafkaIndexTaskIOConfig(
0, 0,
"sequence0", "sequence0",
@ -2888,16 +2885,7 @@ public class KafkaIndexTaskTest extends SeekableStreamIndexTaskTestBase
private static DataSchema cloneDataSchema(final DataSchema dataSchema) private static DataSchema cloneDataSchema(final DataSchema dataSchema)
{ {
return new DataSchema( return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build();
dataSchema.getDataSource(),
dataSchema.getTimestampSpec(),
dataSchema.getDimensionsSpec(),
dataSchema.getAggregators(),
dataSchema.getGranularitySpec(),
dataSchema.getTransformSpec(),
dataSchema.getParserMap(),
OBJECT_MAPPER
);
} }
@Override @Override

View File

@ -46,7 +46,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
@ -81,45 +80,30 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper(); private static final ObjectMapper OBJECT_MAPPER = TestHelper.makeJsonMapper();
private static final String TOPIC = "sampling"; private static final String TOPIC = "sampling";
private static final DataSchema DATA_SCHEMA = new DataSchema( private static final DataSchema DATA_SCHEMA =
"test_ds", DataSchema.builder()
new TimestampSpec("timestamp", "iso", null), .withDataSource("test_ds")
new DimensionsSpec( .withTimestamp(new TimestampSpec("timestamp", "iso", null))
Arrays.asList( .withDimensions(
new StringDimensionSchema("dim1"), new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"), new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"), new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"), new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat") new FloatDimensionSchema("dimFloat")
) )
), .withAggregators(
new AggregatorFactory[]{ new DoubleSumAggregatorFactory("met1sum", "met1"),
new DoubleSumAggregatorFactory("met1sum", "met1"), new CountAggregatorFactory("rows")
new CountAggregatorFactory("rows") )
}, .withGranularity(
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
null )
); .build();
private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP = new DataSchema( private static final DataSchema DATA_SCHEMA_KAFKA_TIMESTAMP =
"test_ds", DataSchema.builder(DATA_SCHEMA)
new TimestampSpec("kafka.timestamp", "iso", null), .withTimestamp(new TimestampSpec("kafka.timestamp", "iso", null))
new DimensionsSpec( .build();
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
),
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null
);
private static TestingCluster zkServer; private static TestingCluster zkServer;
private static TestBroker kafkaServer; private static TestBroker kafkaServer;
@ -364,17 +348,18 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
); );
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
DataSchema dataSchema = new DataSchema( DataSchema dataSchema = DataSchema.builder()
"test_ds", .withDataSource("test_ds")
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), .withParserMap(
new AggregatorFactory[]{ objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class)
new DoubleSumAggregatorFactory("met1sum", "met1"), )
new CountAggregatorFactory("rows") .withAggregators(
}, new DoubleSumAggregatorFactory("met1sum", "met1"),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), new CountAggregatorFactory("rows")
null, )
objectMapper .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
); .withObjectMapper(objectMapper)
.build();
KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec( KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec(
null, null,

View File

@ -30,7 +30,6 @@ import com.google.common.util.concurrent.ListenableFuture;
import org.apache.curator.test.TestingCluster; import org.apache.curator.test.TestingCluster;
import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
@ -83,7 +82,6 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder;
import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.emitter.service.AlertEvent;
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.java.util.metrics.StubServiceEmitter;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.ParseExceptionReport; import org.apache.druid.segment.incremental.ParseExceptionReport;
@ -5135,18 +5133,19 @@ public class KafkaSupervisorTest extends EasyMockSupport
dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim1"));
dimensions.add(StringDimensionSchema.create("dim2")); dimensions.add(StringDimensionSchema.create("dim2"));
return new DataSchema( return DataSchema.builder()
dataSource, .withDataSource(dataSource)
new TimestampSpec("timestamp", "iso", null), .withTimestamp(new TimestampSpec("timestamp", "iso", null))
new DimensionsSpec(dimensions), .withDimensions(dimensions)
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, .withAggregators(new CountAggregatorFactory("rows"))
new UniformGranularitySpec( .withGranularity(
Granularities.HOUR, new UniformGranularitySpec(
Granularities.NONE, Granularities.HOUR,
ImmutableList.of() Granularities.NONE,
), ImmutableList.of()
null )
); )
.build();
} }
private KafkaIndexTask createKafkaIndexTask( private KafkaIndexTask createKafkaIndexTask(

View File

@ -50,7 +50,8 @@ import java.util.Collections;
public class KinesisIndexTaskSerdeTest public class KinesisIndexTaskSerdeTest
{ {
private static final DataSchema DATA_SCHEMA = new DataSchema("dataSource", null, null, null, null, null, null, null); private static final DataSchema DATA_SCHEMA =
DataSchema.builder().withDataSource("dataSource").build();
private static final KinesisIndexTaskTuningConfig TUNING_CONFIG = new KinesisIndexTaskTuningConfig( private static final KinesisIndexTaskTuningConfig TUNING_CONFIG = new KinesisIndexTaskTuningConfig(
null, null,
null, null,

View File

@ -2412,16 +2412,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
private static DataSchema cloneDataSchema(final DataSchema dataSchema) private static DataSchema cloneDataSchema(final DataSchema dataSchema)
{ {
return new DataSchema( return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build();
dataSchema.getDataSource(),
dataSchema.getTimestampSpec(),
dataSchema.getDimensionsSpec(),
dataSchema.getAggregators(),
dataSchema.getGranularitySpec(),
dataSchema.getTransformSpec(),
dataSchema.getParserMap(),
OBJECT_MAPPER
);
} }
@Override @Override

View File

@ -49,7 +49,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
@ -75,25 +74,25 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
{ {
private static final String STREAM = "sampling"; private static final String STREAM = "sampling";
private static final String SHARD_ID = "1"; private static final String SHARD_ID = "1";
private static final DataSchema DATA_SCHEMA = new DataSchema( private static final DataSchema DATA_SCHEMA =
"test_ds", DataSchema.builder()
new TimestampSpec("timestamp", "iso", null), .withDataSource("test_ds")
new DimensionsSpec( .withTimestamp(new TimestampSpec("timestamp", "iso", null))
Arrays.asList( .withDimensions(
new StringDimensionSchema("dim1"), new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"), new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"), new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"), new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat") new FloatDimensionSchema("dimFloat")
) )
), .withAggregators(
new AggregatorFactory[]{ new DoubleSumAggregatorFactory("met1sum", "met1"),
new DoubleSumAggregatorFactory("met1sum", "met1"), new CountAggregatorFactory("rows")
new CountAggregatorFactory("rows") )
}, .withGranularity(
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
null )
); .build();
static { static {
NullHandling.initializeForTests(); NullHandling.initializeForTests();
@ -192,17 +191,18 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
); );
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8"); InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
DataSchema dataSchema = new DataSchema( DataSchema dataSchema = DataSchema.builder()
"test_ds", .withDataSource("test_ds")
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class), .withParserMap(
new AggregatorFactory[]{ objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class)
new DoubleSumAggregatorFactory("met1sum", "met1"), )
new CountAggregatorFactory("rows") .withAggregators(
}, new DoubleSumAggregatorFactory("met1sum", "met1"),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), new CountAggregatorFactory("rows")
null, )
objectMapper .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
); .withObjectMapper(objectMapper)
.build();
KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec( KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec(
null, null,

View File

@ -28,7 +28,6 @@ import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.data.input.InputFormat; import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
@ -79,7 +78,6 @@ import org.apache.druid.java.util.emitter.service.AlertBuilder;
import org.apache.druid.java.util.emitter.service.AlertEvent; import org.apache.druid.java.util.emitter.service.AlertEvent;
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.java.util.metrics.StubServiceEmitter;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
@ -5482,18 +5480,19 @@ public class KinesisSupervisorTest extends EasyMockSupport
dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim1"));
dimensions.add(StringDimensionSchema.create("dim2")); dimensions.add(StringDimensionSchema.create("dim2"));
return new DataSchema( return DataSchema.builder()
dataSource, .withDataSource(dataSource)
new TimestampSpec("timestamp", "iso", null), .withTimestamp(new TimestampSpec("timestamp", "iso", null))
new DimensionsSpec(dimensions), .withDimensions(dimensions)
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, .withAggregators(new CountAggregatorFactory("rows"))
new UniformGranularitySpec( .withGranularity(
Granularities.HOUR, new UniformGranularitySpec(
Granularities.NONE, Granularities.HOUR,
ImmutableList.of() Granularities.NONE,
), ImmutableList.of()
null )
); )
.build();
} }

View File

@ -53,14 +53,12 @@ import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.sql.calcite.planner.ColumnMappings; import org.apache.druid.sql.calcite.planner.ColumnMappings;
import org.apache.druid.sql.calcite.rel.DruidQuery; import org.apache.druid.sql.calcite.rel.DruidQuery;
import org.apache.druid.utils.CollectionUtils; import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable; import javax.annotation.Nullable;
import java.util.ArrayList; import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap; import java.util.HashMap;
import java.util.LinkedHashSet; import java.util.LinkedHashSet;
import java.util.List; import java.util.List;
@ -96,14 +94,13 @@ public final class SegmentGenerationUtils
destination.getDimensionSchemas() destination.getDimensionSchemas()
); );
return new DataSchema( return DataSchema.builder()
destination.getDataSource(), .withDataSource(destination.getDataSource())
new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), .withTimestamp(new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null))
dimensionsAndAggregators.lhs, .withDimensions(dimensionsAndAggregators.lhs)
dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]), .withAggregators(dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]))
makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper), .withGranularity(makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper))
new TransformSpec(null, Collections.emptyList()) .build();
);
} }
private static GranularitySpec makeGranularitySpecForIngestion( private static GranularitySpec makeGranularitySpecForIngestion(

View File

@ -259,19 +259,21 @@ public class MSQCompactionRunnerTest
null null
); );
DataSchema dataSchema = new DataSchema( DataSchema dataSchema =
DATA_SOURCE, DataSchema.builder()
new TimestampSpec(TIMESTAMP_COLUMN, null, null), .withDataSource(DATA_SOURCE)
new DimensionsSpec(DIMENSIONS), .withTimestamp(new TimestampSpec(TIMESTAMP_COLUMN, null, null))
new AggregatorFactory[]{}, .withDimensions(DIMENSIONS)
new UniformGranularitySpec( .withGranularity(
SEGMENT_GRANULARITY.getDefaultGranularity(), new UniformGranularitySpec(
null, SEGMENT_GRANULARITY.getDefaultGranularity(),
false, null,
Collections.singletonList(COMPACTION_INTERVAL) false,
), Collections.singletonList(COMPACTION_INTERVAL)
new TransformSpec(dimFilter, Collections.emptyList()) )
); )
.withTransform(new TransformSpec(dimFilter, Collections.emptyList()))
.build();
List<MSQControllerTask> msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks( List<MSQControllerTask> msqControllerTasks = MSQ_COMPACTION_RUNNER.createMsqControllerTasks(

View File

@ -433,30 +433,33 @@ public class BatchDeltaIngestionTest
{ {
HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"website", .withDataSource("website")
MAPPER.convertValue( .withParserMap(MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new CSVParseSpec( new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
null, null,
ImmutableList.of("timestamp", "host", "host2", "visited_num"), ImmutableList.of("timestamp", "host", "host2", "visited_num"),
false, false,
0 0
), ),
null null
), ),
Map.class Map.class
), ))
aggregators != null ? aggregators : new AggregatorFactory[]{ .withAggregators(aggregators != null ? aggregators : new AggregatorFactory[]{
new LongSumAggregatorFactory("visited_sum", "visited_num"), new LongSumAggregatorFactory("visited_sum", "visited_num"),
new HyperUniquesAggregatorFactory("unique_hosts", "host2") new HyperUniquesAggregatorFactory("unique_hosts", "host2")
}, })
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)), .withGranularity(new UniformGranularitySpec(
null, Granularities.DAY,
MAPPER Granularities.NONE,
), ImmutableList.of(INTERVAL_FULL)
))
.withObjectMapper(MAPPER)
.build(),
new HadoopIOConfig( new HadoopIOConfig(
inputSpec, inputSpec,
null, null,

View File

@ -32,7 +32,6 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -158,46 +157,45 @@ public class DetermineHashedPartitionsJobTest
} }
HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec( HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"test_schema", .withDataSource("test_schema")
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( .withParserMap(HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new DelimitedParseSpec( new DelimitedParseSpec(
new TimestampSpec("ts", null, null), new TimestampSpec("ts", null, null),
new DimensionsSpec( new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of( DimensionsSpec.getDefaultSchemas(ImmutableList.of(
"market", "market",
"quality", "quality",
"placement", "placement",
"placementish" "placementish"
)) ))
), ),
"\t", "\t",
null, null,
Arrays.asList( Arrays.asList(
"ts", "ts",
"market", "market",
"quality", "quality",
"placement", "placement",
"placementish", "placementish",
"index" "index"
), ),
false, false,
0 0
), ),
null null
), ),
Map.class Map.class
), ))
new AggregatorFactory[]{new DoubleSumAggregatorFactory("index", "index")}, .withAggregators(new DoubleSumAggregatorFactory("index", "index"))
new UniformGranularitySpec( .withGranularity(new UniformGranularitySpec(
segmentGranularity, segmentGranularity,
Granularities.NONE, Granularities.NONE,
intervals intervals
), ))
null, .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
HadoopDruidIndexerConfig.JSON_MAPPER .build(),
),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.of( ImmutableMap.of(
"paths", "paths",

View File

@ -29,7 +29,6 @@ import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -280,33 +279,36 @@ public class DeterminePartitionsJobTest
config = new HadoopDruidIndexerConfig( config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"website", .withDataSource("website")
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( .withParserMap(
new StringInputRowParser( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new CSVParseSpec( new StringInputRowParser(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new CSVParseSpec(
new DimensionsSpec( new TimestampSpec("timestamp", "yyyyMMddHH", null),
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) new DimensionsSpec(
), DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
null, ),
ImmutableList.of("timestamp", "host", "country", "visited_num"), null,
false, ImmutableList.of("timestamp", "host", "country", "visited_num"),
0 false,
), 0
null ),
), null
Map.class ),
), Map.class
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, )
new UniformGranularitySpec( )
Granularities.DAY, .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
Granularities.NONE, .withGranularity(
ImmutableList.of(Intervals.of(interval)) new UniformGranularitySpec(
), Granularities.DAY,
null, Granularities.NONE,
HadoopDruidIndexerConfig.JSON_MAPPER ImmutableList.of(Intervals.of(interval))
), )
)
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.of( ImmutableMap.of(
"paths", "paths",

View File

@ -29,7 +29,6 @@ import org.apache.druid.indexer.partitions.DimensionRangePartitionsSpec;
import org.apache.druid.java.util.common.FileUtils; import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -328,35 +327,36 @@ public class DetermineRangePartitionsJobTest
config = new HadoopDruidIndexerConfig( config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"website", .withDataSource("website")
null, .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
null, .withGranularity(
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.NONE,
Granularities.NONE, ImmutableList.of(Intervals.of(interval))
ImmutableList.of(Intervals.of(interval)) )
), )
null, .withParserMap(
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser( new StringInputRowParser(
new CSVParseSpec( new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec( new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country")) DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
), ),
null, null,
ImmutableList.of("timestamp", "host", "country", "visited_num"), ImmutableList.of("timestamp", "host", "country", "visited_num"),
false, false,
0 0
), ),
null null
), ),
Map.class Map.class
), )
HadoopDruidIndexerConfig.JSON_MAPPER )
), .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.of( ImmutableMap.of(
"paths", "paths",

View File

@ -37,7 +37,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
@ -217,18 +216,18 @@ public class HadoopDruidIndexerConfigTest
private static class HadoopIngestionSpecBuilder private static class HadoopIngestionSpecBuilder
{ {
private static final DataSchema DATA_SCHEMA = new DataSchema( private static final DataSchema DATA_SCHEMA =
"foo", DataSchema.builder()
null, .withDataSource("foo")
new AggregatorFactory[0], .withGranularity(
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.MINUTE, Granularities.MINUTE,
Granularities.MINUTE, Granularities.MINUTE,
ImmutableList.of(Intervals.of("2010-01-01/P1D")) ImmutableList.of(Intervals.of("2010-01-01/P1D"))
), )
null, )
HadoopDruidIndexerConfigTest.JSON_MAPPER .withObjectMapper(HadoopDruidIndexerConfigTest.JSON_MAPPER)
); .build();
private static final HadoopIOConfig HADOOP_IO_CONFIG = new HadoopIOConfig( private static final HadoopIOConfig HADOOP_IO_CONFIG = new HadoopIOConfig(
ImmutableMap.of("paths", "bar", "type", "static"), ImmutableMap.of("paths", "bar", "type", "static"),

View File

@ -31,7 +31,6 @@ import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.math.expr.ExprMacroTable; import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.filter.SelectorDimFilter; import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
@ -58,27 +57,29 @@ import java.util.stream.Collectors;
public class HadoopDruidIndexerMapperTest public class HadoopDruidIndexerMapperTest
{ {
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
private static final DataSchema DATA_SCHEMA = new DataSchema( private static final DataSchema DATA_SCHEMA =
"test_ds", DataSchema.builder()
JSON_MAPPER.convertValue( .withDataSource("test_ds")
new HadoopyStringInputRowParser( .withParserMap(
new JSONParseSpec( JSON_MAPPER.convertValue(
new TimestampSpec("t", "auto", null), new HadoopyStringInputRowParser(
new DimensionsSpec( new JSONParseSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) new TimestampSpec("t", "auto", null),
), new DimensionsSpec(
new JSONPathSpec(true, ImmutableList.of()), DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
ImmutableMap.of(), ),
null new JSONPathSpec(true, ImmutableList.of()),
) ImmutableMap.of(),
), null
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT )
), ),
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), )
null, )
JSON_MAPPER .withAggregators(new CountAggregatorFactory("rows"))
); .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.withObjectMapper(JSON_MAPPER)
.build();
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
JSON_MAPPER.convertValue( JSON_MAPPER.convertValue(

View File

@ -34,7 +34,6 @@ import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.timeline.DataSegment; import org.apache.druid.timeline.DataSegment;
@ -274,18 +273,17 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
throws Exception throws Exception
{ {
HadoopIngestionSpec spec = new HadoopIngestionSpec( HadoopIngestionSpec spec = new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
null, .withGranularity(
new AggregatorFactory[0], new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, null,
null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
ImmutableList.of(Intervals.of("2010-01-01/P1D")) )
), )
null, .withObjectMapper(jsonMapper)
jsonMapper .build(),
),
new HadoopIOConfig( new HadoopIOConfig(
jsonMapper.convertValue(datasourcePathSpec, Map.class), jsonMapper.convertValue(datasourcePathSpec, Map.class),
null, null,

View File

@ -64,30 +64,33 @@ public class IndexGeneratorCombinerTest
{ {
HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig( HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"website", .withDataSource("website")
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( .withParserMap(
new StringInputRowParser( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new TimeAndDimsParseSpec( new StringInputRowParser(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new TimeAndDimsParseSpec(
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords"))) new TimestampSpec("timestamp", "yyyyMMddHH", null),
), new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "keywords")))
null ),
), null
Map.class ),
), Map.class
new AggregatorFactory[]{ )
new LongSumAggregatorFactory("visited_sum", "visited"), )
new HyperUniquesAggregatorFactory("unique_hosts", "host") .withAggregators(
}, new LongSumAggregatorFactory("visited_sum", "visited"),
new UniformGranularitySpec( new HyperUniquesAggregatorFactory("unique_hosts", "host")
Granularities.DAY, )
Granularities.NONE, .withGranularity(
ImmutableList.of(Intervals.of("2010/2011")) new UniformGranularitySpec(
), Granularities.DAY,
null, Granularities.NONE,
HadoopDruidIndexerConfig.JSON_MAPPER ImmutableList.of(Intervals.of("2010/2011"))
), )
)
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.of( ImmutableMap.of(
"paths", "paths",

View File

@ -506,17 +506,19 @@ public class IndexGeneratorJobTest
config = new HadoopDruidIndexerConfig( config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
datasourceName, .withDataSource(datasourceName)
mapper.convertValue( .withParserMap(mapper.convertValue(inputRowParser, Map.class))
inputRowParser, .withAggregators(aggs)
Map.class .withGranularity(
), new UniformGranularitySpec(
aggs, Granularities.DAY,
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)), Granularities.NONE,
null, ImmutableList.of(interval)
mapper )
), )
.withObjectMapper(mapper)
.build(),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.copyOf(inputSpec), ImmutableMap.copyOf(inputSpec),
null, null,

View File

@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.jackson.JacksonUtils; import org.apache.druid.java.util.common.jackson.JacksonUtils;
import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
@ -68,27 +67,30 @@ import java.util.Map;
public class JobHelperTest public class JobHelperTest
{ {
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper(); private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
private static final DataSchema DATA_SCHEMA = new DataSchema(
"test_ds", private static final DataSchema DATA_SCHEMA =
JSON_MAPPER.convertValue( DataSchema.builder()
new HadoopyStringInputRowParser( .withDataSource("test_ds")
new JSONParseSpec( .withParserMap(
new TimestampSpec("t", "auto", null), JSON_MAPPER.convertValue(
new DimensionsSpec( new HadoopyStringInputRowParser(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2")) new JSONParseSpec(
), new TimestampSpec("t", "auto", null),
new JSONPathSpec(true, ImmutableList.of()), new DimensionsSpec(
ImmutableMap.of(), DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
null ),
) new JSONPathSpec(true, ImmutableList.of()),
), ImmutableMap.of(),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT null
), )
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, ),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
null, )
JSON_MAPPER )
); .withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.withObjectMapper(JSON_MAPPER)
.build();
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig( private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
JSON_MAPPER.convertValue( JSON_MAPPER.convertValue(
@ -123,27 +125,34 @@ public class JobHelperTest
dataFile = temporaryFolder.newFile(); dataFile = temporaryFolder.newFile();
config = new HadoopDruidIndexerConfig( config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"website", .withDataSource("website")
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( .withParserMap(
new StringInputRowParser( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new CSVParseSpec( new StringInputRowParser(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new CSVParseSpec(
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))), new TimestampSpec("timestamp", "yyyyMMddHH", null),
null, new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
ImmutableList.of("timestamp", "host", "visited_num"), null,
false, ImmutableList.of("timestamp", "host", "visited_num"),
0 false,
), 0
null ),
), null
Map.class ),
), Map.class
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")}, )
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)), )
null, .withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
HadoopDruidIndexerConfig.JSON_MAPPER .withGranularity(
), new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(this.interval)
)
)
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.of( ImmutableMap.of(
"paths", "paths",

View File

@ -44,7 +44,6 @@ import org.apache.druid.initialization.Initialization;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -308,33 +307,34 @@ public class DatasourcePathSpecTest
{ {
return new HadoopDruidIndexerConfig( return new HadoopDruidIndexerConfig(
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
ingestionSpec1.getDataSource(), .withDataSource(ingestionSpec1.getDataSource())
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue( .withParserMap(
new StringInputRowParser( HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new CSVParseSpec( new StringInputRowParser(
new TimestampSpec("timestamp", "yyyyMMddHH", null), new CSVParseSpec(
DimensionsSpec.EMPTY, new TimestampSpec("timestamp", "yyyyMMddHH", null),
null, DimensionsSpec.EMPTY,
ImmutableList.of("timestamp", "host", "visited"), null,
false, ImmutableList.of("timestamp", "host", "visited"),
0 false,
), 0
null ),
), null
Map.class ),
), Map.class
new AggregatorFactory[]{ )
new LongSumAggregatorFactory("visited_sum", "visited") )
}, .withAggregators(new LongSumAggregatorFactory("visited_sum", "visited"))
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
Granularities.NONE, Granularities.DAY,
ImmutableList.of(Intervals.of("2000/3000")) Granularities.NONE,
), ImmutableList.of(Intervals.of("2000/3000"))
null, )
HadoopDruidIndexerConfig.JSON_MAPPER )
), .withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig( new HadoopIOConfig(
ImmutableMap.of( ImmutableMap.of(
"paths", "paths",

View File

@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.granularity.PeriodGranularity; import org.apache.druid.java.util.common.granularity.PeriodGranularity;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
@ -152,18 +151,17 @@ public class GranularityPathSpecTest
{ {
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"})); UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
HadoopIngestionSpec spec = new HadoopIngestionSpec( HadoopIngestionSpec spec = new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
null, .withGranularity(
new AggregatorFactory[0], new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.MINUTE,
Granularities.MINUTE, ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z"))
ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z")) )
), )
null, .withObjectMapper(jsonMapper)
jsonMapper .build(),
),
new HadoopIOConfig(null, null, null), new HadoopIOConfig(null, null, null),
DEFAULT_TUNING_CONFIG DEFAULT_TUNING_CONFIG
); );
@ -204,18 +202,17 @@ public class GranularityPathSpecTest
{ {
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"})); UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
HadoopIngestionSpec spec = new HadoopIngestionSpec( HadoopIngestionSpec spec = new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
null, .withGranularity(
new AggregatorFactory[0], new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.ALL,
Granularities.ALL, ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z"))
ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z")) )
), )
null, .withObjectMapper(jsonMapper)
jsonMapper .build(),
),
new HadoopIOConfig(null, null, null), new HadoopIOConfig(null, null, null),
DEFAULT_TUNING_CONFIG DEFAULT_TUNING_CONFIG
); );

View File

@ -24,7 +24,6 @@ import org.apache.druid.indexer.HadoopDruidIndexerConfig;
import org.apache.druid.indexer.HadoopIOConfig; import org.apache.druid.indexer.HadoopIOConfig;
import org.apache.druid.indexer.HadoopIngestionSpec; import org.apache.druid.indexer.HadoopIngestionSpec;
import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.hadoop.mapreduce.Job; import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.input.MultipleInputs; import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
@ -54,7 +53,7 @@ public class StaticPathSpecTest
Job job = new Job(); Job job = new Job();
StaticPathSpec pathSpec = new StaticPathSpec("/a/c,/a/b/{c,d}", null); StaticPathSpec pathSpec = new StaticPathSpec("/a/c,/a/b/{c,d}", null);
DataSchema schema = new DataSchema("ds", null, new AggregatorFactory[0], null, null, jsonMapper); DataSchema schema = DataSchema.builder().withDataSource("ds").withObjectMapper(jsonMapper).build();
HadoopIOConfig io = new HadoopIOConfig(null, null, null); HadoopIOConfig io = new HadoopIOConfig(null, null, null);
pathSpec.addInputPaths(new HadoopDruidIndexerConfig(new HadoopIngestionSpec(schema, io, null)), job); pathSpec.addInputPaths(new HadoopDruidIndexerConfig(new HadoopIngestionSpec(schema, io, null)), job);

View File

@ -67,14 +67,13 @@ public class InputSourceSampler
{ {
private static final String SAMPLER_DATA_SOURCE = "sampler"; private static final String SAMPLER_DATA_SOURCE = "sampler";
private static final DataSchema DEFAULT_DATA_SCHEMA = new DataSchema(
SAMPLER_DATA_SOURCE, private static final DataSchema DEFAULT_DATA_SCHEMA =
new TimestampSpec(null, null, null), DataSchema.builder()
new DimensionsSpec(null), .withDataSource(SAMPLER_DATA_SOURCE)
null, .withTimestamp(new TimestampSpec(null, null, null))
null, .withDimensions(DimensionsSpec.builder().build())
null .build();
);
// We want to be able to sort the list of processed results back into the same order that we read them from the input // We want to be able to sort the list of processed results back into the same order that we read them from the input
// source so that the rows in the data loader are not always changing. To do this, we add a temporary column to the // source so that the rows in the data loader are not always changing. To do this, we add a temporary column to the

View File

@ -33,7 +33,6 @@ import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.common.task.TuningConfigBuilder;
import org.apache.druid.indexing.overlord.SegmentPublishResult; import org.apache.druid.indexing.overlord.SegmentPublishResult;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.SegmentSchemaMapping; import org.apache.druid.segment.SegmentSchemaMapping;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
@ -62,7 +61,7 @@ public class TestIndexTask extends IndexTask
id, id,
taskResource, taskResource,
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper), DataSchema.builder().withDataSource(dataSource).withObjectMapper(mapper).build(),
new IndexTask.IndexIOConfig( new IndexTask.IndexIOConfig(
new LocalInputSource(new File("lol"), "rofl"), new LocalInputSource(new File("lol"), "rofl"),
new JsonInputFormat(null, null, null, null, null), new JsonInputFormat(null, null, null, null, null),

View File

@ -939,18 +939,19 @@ public class CompactionTaskParallelRunTest extends AbstractParallelIndexSupervis
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATA_SOURCE, .withDataSource(DATA_SOURCE)
new TimestampSpec("ts", "auto", null), .withTimestamp(new TimestampSpec("ts", "auto", null))
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim"))), .withDimensions(DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "dim")))
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, .withAggregators(new LongSumAggregatorFactory("val", "val"))
new UniformGranularitySpec( .withGranularity(
Granularities.HOUR, new UniformGranularitySpec(
Granularities.MINUTE, Granularities.HOUR,
ImmutableList.of(INTERVAL_TO_INDEX) Granularities.MINUTE,
), ImmutableList.of(INTERVAL_TO_INDEX)
null )
), )
.build(),
ioConfig, ioConfig,
tuningConfig tuningConfig
), ),

View File

@ -27,7 +27,6 @@ import org.apache.druid.indexer.HadoopIngestionSpec;
import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Action;
@ -50,15 +49,19 @@ public class HadoopIndexTaskTest
final HadoopIndexTask task = new HadoopIndexTask( final HadoopIndexTask task = new HadoopIndexTask(
null, null,
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", null, new AggregatorFactory[0], new UniformGranularitySpec( .withDataSource("foo")
Granularities.DAY, .withGranularity(
null, new UniformGranularitySpec(
ImmutableList.of(Intervals.of("2010-01-01/P1D")) Granularities.DAY,
), null,
null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
jsonMapper )
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null )
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
null
), ),
null, null,
null, null,

View File

@ -25,7 +25,6 @@ import org.apache.druid.data.input.impl.NoopInputSource;
import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig; import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec; import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.junit.Rule; import org.junit.Rule;
@ -45,14 +44,11 @@ public class IndexIngestionSpecTest
"Cannot use parser and inputSource together. Try using inputFormat instead of parser." "Cannot use parser and inputSource together. Try using inputFormat instead of parser."
); );
final IndexIngestionSpec spec = new IndexIngestionSpec( final IndexIngestionSpec spec = new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
ImmutableMap.of("fake", "parser map"), .withParserMap(ImmutableMap.of("fake", "parser map"))
new AggregatorFactory[0], .withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null))
new ArbitraryGranularitySpec(Granularities.NONE, null), .build(),
null,
null
),
new IndexIOConfig( new IndexIOConfig(
new NoopInputSource(), new NoopInputSource(),
new NoopInputFormat(), new NoopInputFormat(),
@ -69,14 +65,11 @@ public class IndexIngestionSpecTest
expectedException.expect(IllegalArgumentException.class); expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("Cannot use parser and inputSource together."); expectedException.expectMessage("Cannot use parser and inputSource together.");
final IndexIngestionSpec spec = new IndexIngestionSpec( final IndexIngestionSpec spec = new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
ImmutableMap.of("fake", "parser map"), .withParserMap(ImmutableMap.of("fake", "parser map"))
new AggregatorFactory[0], .withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null))
new ArbitraryGranularitySpec(Granularities.NONE, null), .build(),
null,
null
),
new IndexIOConfig( new IndexIOConfig(
new NoopInputSource(), new NoopInputSource(),
null, null,

View File

@ -166,6 +166,25 @@ public class IndexTaskTest extends IngestionTestBase
0 0
); );
private static final DataSchema DATA_SCHEMA =
DataSchema.builder()
.withDataSource("test-json")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(
new StringDimensionSchema("ts"),
new StringDimensionSchema("dim"),
new LongDimensionSchema("valDim")
)
.withAggregators(new LongSumAggregatorFactory("valMet", "val"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/P1D"))
)
)
.build();
@Parameterized.Parameters(name = "{0}, useInputFormatApi={1}") @Parameterized.Parameters(name = "{0}, useInputFormatApi={1}")
public static Iterable<Object[]> constructorFeeder() public static Iterable<Object[]> constructorFeeder()
{ {
@ -225,24 +244,7 @@ public class IndexTaskTest extends IngestionTestBase
{ {
IndexTask indexTask = createIndexTask( IndexTask indexTask = createIndexTask(
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DATA_SCHEMA,
"test-json",
DEFAULT_TIMESTAMP_SPEC,
new DimensionsSpec(
ImmutableList.of(
new StringDimensionSchema("ts"),
new StringDimensionSchema("dim"),
new LongDimensionSchema("valDim")
)
),
new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/P1D"))
),
null
),
new IndexIOConfig( new IndexIOConfig(
new LocalInputSource(tmpDir, "druid*"), new LocalInputSource(tmpDir, "druid*"),
DEFAULT_INPUT_FORMAT, DEFAULT_INPUT_FORMAT,
@ -275,24 +277,7 @@ public class IndexTaskTest extends IngestionTestBase
IndexTask indexTask = createIndexTask( IndexTask indexTask = createIndexTask(
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DATA_SCHEMA,
"test-json",
DEFAULT_TIMESTAMP_SPEC,
new DimensionsSpec(
ImmutableList.of(
new StringDimensionSchema("ts"),
new StringDimensionSchema("dim"),
new LongDimensionSchema("valDim")
)
),
new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/P1D"))
),
null
),
new IndexIOConfig( new IndexIOConfig(
new LocalInputSource(tmpDir, "druid*"), new LocalInputSource(tmpDir, "druid*"),
DEFAULT_INPUT_FORMAT, DEFAULT_INPUT_FORMAT,
@ -337,24 +322,7 @@ public class IndexTaskTest extends IngestionTestBase
IndexTask indexTask = createIndexTask( IndexTask indexTask = createIndexTask(
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DATA_SCHEMA,
"test-json",
DEFAULT_TIMESTAMP_SPEC,
new DimensionsSpec(
ImmutableList.of(
new StringDimensionSchema("ts"),
new StringDimensionSchema("dim"),
new LongDimensionSchema("valDim")
)
),
new AggregatorFactory[]{new LongSumAggregatorFactory("valMet", "val")},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/P1D"))
),
null
),
new IndexIOConfig( new IndexIOConfig(
new LocalInputSource(tmpDir, "druid*"), new LocalInputSource(tmpDir, "druid*"),
DEFAULT_INPUT_FORMAT, DEFAULT_INPUT_FORMAT,
@ -2698,20 +2666,20 @@ public class IndexTaskTest extends IngestionTestBase
if (inputFormat != null) { if (inputFormat != null) {
Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec"); Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec");
return new IndexIngestionSpec( return new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
Preconditions.checkNotNull(timestampSpec, "timestampSpec"), .withTimestamp(Preconditions.checkNotNull(timestampSpec, "timestampSpec"))
Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"), .withDimensions(Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"))
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
granularitySpec != null ? granularitySpec : new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.MINUTE,
Granularities.MINUTE, Collections.singletonList(Intervals.of("2014/2015"))
Collections.singletonList(Intervals.of("2014/2015")) )
), )
transformSpec .withTransform(transformSpec)
), .build(),
new IndexIOConfig( new IndexIOConfig(
new LocalInputSource(baseDir, "druid*"), new LocalInputSource(baseDir, "druid*"),
inputFormat, inputFormat,
@ -2723,22 +2691,21 @@ public class IndexTaskTest extends IngestionTestBase
} else { } else {
parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC; parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC;
return new IndexIngestionSpec( return new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
parseSpec.getTimestampSpec(), .withTimestamp(parseSpec.getTimestampSpec())
parseSpec.getDimensionsSpec(), .withDimensions(parseSpec.getDimensionsSpec())
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
granularitySpec != null ? granularitySpec : new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.MINUTE,
Granularities.MINUTE, Collections.singletonList(Intervals.of("2014/2015"))
Collections.singletonList(Intervals.of("2014/2015")) )
), )
transformSpec, .withTransform(transformSpec)
null, .withObjectMapper(objectMapper)
objectMapper .build(),
),
new IndexIOConfig( new IndexIOConfig(
new LocalInputSource(baseDir, "druid*"), new LocalInputSource(baseDir, "druid*"),
createInputFormatFromParseSpec(parseSpec), createInputFormatFromParseSpec(parseSpec),

View File

@ -38,7 +38,6 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig; import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
@ -220,18 +219,19 @@ public class TaskSerdeTest
null, null,
null, null,
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, .withAggregators(new DoubleSumAggregatorFactory("met", "met"))
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
null, Granularities.DAY,
ImmutableList.of(Intervals.of("2010-01-01/P2D")) null,
), ImmutableList.of(Intervals.of("2010-01-01/P2D"))
null )
), )
.build(),
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
TuningConfigBuilder.forIndexTask() TuningConfigBuilder.forIndexTask()
.withMaxRowsInMemory(10) .withMaxRowsInMemory(10)
@ -288,18 +288,19 @@ public class TaskSerdeTest
null, null,
new TaskResource("rofl", 2), new TaskResource("rofl", 2),
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, .withAggregators(new DoubleSumAggregatorFactory("met", "met"))
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
null, Granularities.DAY,
ImmutableList.of(Intervals.of("2010-01-01/P2D")) null,
), ImmutableList.of(Intervals.of("2010-01-01/P2D"))
null )
), )
.build(),
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false), new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
TuningConfigBuilder.forIndexTask() TuningConfigBuilder.forIndexTask()
.withMaxRowsInMemory(10) .withMaxRowsInMemory(10)
@ -412,15 +413,19 @@ public class TaskSerdeTest
final HadoopIndexTask task = new HadoopIndexTask( final HadoopIndexTask task = new HadoopIndexTask(
null, null,
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", null, new AggregatorFactory[0], new UniformGranularitySpec( .withDataSource("foo")
Granularities.DAY, .withGranularity(
null, new UniformGranularitySpec(
ImmutableList.of(Intervals.of("2010-01-01/P1D")) Granularities.DAY,
), null,
null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
jsonMapper )
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null )
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
null
), ),
null, null,
null, null,
@ -454,19 +459,18 @@ public class TaskSerdeTest
final HadoopIndexTask task = new HadoopIndexTask( final HadoopIndexTask task = new HadoopIndexTask(
null, null,
new HadoopIngestionSpec( new HadoopIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
null, .withGranularity(
null, new UniformGranularitySpec(
new AggregatorFactory[0], Granularities.DAY,
new UniformGranularitySpec( null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
Granularities.DAY, )
null, ImmutableList.of(Intervals.of("2010-01-01/P1D")) )
), .withObjectMapper(jsonMapper)
null, .build(),
null, new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
jsonMapper null
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null
), ),
null, null,
null, null,

View File

@ -221,14 +221,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
dropExisting dropExisting
); );
ingestionSpec = new ParallelIndexIngestionSpec( ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
timestampSpec, .withTimestamp(timestampSpec)
dimensionsSpec, .withDimensions(dimensionsSpec)
DEFAULT_METRICS_SPEC, .withAggregators(DEFAULT_METRICS_SPEC)
granularitySpec, .withGranularity(granularitySpec)
null .build(),
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );
@ -241,14 +240,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
dropExisting dropExisting
); );
ingestionSpec = new ParallelIndexIngestionSpec( ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
parseSpec.getTimestampSpec(), .withTimestamp(parseSpec.getTimestampSpec())
parseSpec.getDimensionsSpec(), .withDimensions(parseSpec.getDimensionsSpec())
DEFAULT_METRICS_SPEC, .withAggregators(DEFAULT_METRICS_SPEC)
granularitySpec, .withGranularity(granularitySpec)
null .build(),
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );

View File

@ -41,7 +41,6 @@ import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Comparators; import org.apache.druid.java.util.common.guava.Comparators;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec;
@ -250,14 +249,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
null null
); );
ingestionSpec = new ParallelIndexIngestionSpec( ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
timestampSpec, .withTimestamp(timestampSpec)
dimensionsSpec, .withDimensions(dimensionsSpec)
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, .withAggregators(new LongSumAggregatorFactory("val", "val"))
granularitySpec, .withGranularity(granularitySpec)
null .build(),
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );
@ -271,16 +269,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
); );
//noinspection unchecked //noinspection unchecked
ingestionSpec = new ParallelIndexIngestionSpec( ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
parseSpec.getTimestampSpec(), .withTimestamp(parseSpec.getTimestampSpec())
parseSpec.getDimensionsSpec(), .withDimensions(parseSpec.getDimensionsSpec())
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(granularitySpec)
}, .build(),
granularitySpec,
null
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );

View File

@ -127,18 +127,19 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DIMENSIONS_SPEC.withDimensions(dimensionSchemas), .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
DEFAULT_METRICS_SPEC, .withAggregators(DEFAULT_METRICS_SPEC)
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
Granularities.MINUTE, Granularities.DAY,
INTERVAL_TO_INDEX Granularities.MINUTE,
), INTERVAL_TO_INDEX
null )
), )
.build(),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
getInputSource(), getInputSource(),
JSON_FORMAT, JSON_FORMAT,
@ -177,18 +178,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
TIMESTAMP_SPEC, .withTimestamp(TIMESTAMP_SPEC)
new DimensionsSpec.Builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build(), .withDimensions(
DEFAULT_METRICS_SPEC, DimensionsSpec.builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build()
new UniformGranularitySpec( )
Granularities.DAY, .withAggregators(DEFAULT_METRICS_SPEC)
Granularities.MINUTE, .withGranularity(
INTERVAL_TO_INDEX new UniformGranularitySpec(
), Granularities.DAY,
null Granularities.MINUTE,
), INTERVAL_TO_INDEX
)
)
.build(),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
getInputSource(), getInputSource(),
new JsonInputFormat( new JsonInputFormat(
@ -237,18 +241,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
TIMESTAMP_SPEC, .withTimestamp(TIMESTAMP_SPEC)
new DimensionsSpec.Builder().setIncludeAllDimensions(true).build(), .withDimensions(
DEFAULT_METRICS_SPEC, DimensionsSpec.builder().setIncludeAllDimensions(true).build()
new UniformGranularitySpec( )
Granularities.DAY, .withAggregators(DEFAULT_METRICS_SPEC)
Granularities.MINUTE, .withGranularity(
null new UniformGranularitySpec(
), Granularities.DAY,
null Granularities.MINUTE,
), null
)
)
.build(),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
getInputSource(), getInputSource(),
new JsonInputFormat( new JsonInputFormat(
@ -303,20 +310,23 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
TIMESTAMP_SPEC, .withTimestamp(TIMESTAMP_SPEC)
DIMENSIONS_SPEC.withDimensions( .withDimensions(
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim")) DIMENSIONS_SPEC.withDimensions(
), DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim"))
DEFAULT_METRICS_SPEC, )
new UniformGranularitySpec( )
Granularities.DAY, .withAggregators(DEFAULT_METRICS_SPEC)
Granularities.MINUTE, .withGranularity(
INTERVAL_TO_INDEX new UniformGranularitySpec(
), Granularities.DAY,
null Granularities.MINUTE,
), INTERVAL_TO_INDEX
)
)
.build(),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
getInputSource(), getInputSource(),
JSON_FORMAT, JSON_FORMAT,

View File

@ -36,7 +36,6 @@ import org.apache.druid.indexing.common.task.TuningConfigBuilder;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -147,20 +146,19 @@ public class ParallelIndexSupervisorTaskKillTest extends AbstractParallelIndexSu
final int numTotalSubTasks = inputSource.estimateNumSplits(new NoopInputFormat(), null); final int numTotalSubTasks = inputSource.estimateNumSplits(new NoopInputFormat(), null);
// set up ingestion spec // set up ingestion spec
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DEFAULT_DIMENSIONS_SPEC, .withDimensions(DEFAULT_DIMENSIONS_SPEC)
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.MINUTE,
Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval)
interval == null ? null : Collections.singletonList(interval) )
), )
null .build(),
),
ioConfig, ioConfig,
TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(numTotalSubTasks).build() TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(numTotalSubTasks).build()
); );

View File

@ -45,7 +45,6 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -400,20 +399,19 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd
) )
{ {
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec( final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DEFAULT_DIMENSIONS_SPEC, .withDimensions(DEFAULT_DIMENSIONS_SPEC)
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.MINUTE,
Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval)
interval == null ? null : Collections.singletonList(interval) )
), )
null .build(),
),
ioConfig, ioConfig,
TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(NUM_SUB_TASKS).build() TuningConfigBuilder.forParallelIndexTask().withMaxNumConcurrentSubTasks(NUM_SUB_TASKS).build()
); );

View File

@ -34,7 +34,6 @@ import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.indexing.common.task.TuningConfigBuilder;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -220,16 +219,19 @@ public class ParallelIndexSupervisorTaskSerdeTest
ParallelIndexIngestionSpec build() ParallelIndexIngestionSpec build()
{ {
DataSchema dataSchema = new DataSchema( DataSchema dataSchema = DataSchema.builder()
"dataSource", .withDataSource("datasource")
TIMESTAMP_SPEC, .withTimestamp(TIMESTAMP_SPEC)
DIMENSIONS_SPEC, .withDimensions(DIMENSIONS_SPEC)
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, new UniformGranularitySpec(
new UniformGranularitySpec(Granularities.DAY, Granularities.MINUTE, inputIntervals), Granularities.DAY,
null Granularities.MINUTE,
); inputIntervals
)
)
.build();
ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder
.forParallelIndexTask() .forParallelIndexTask()

View File

@ -263,14 +263,11 @@ public class ParallelIndexSupervisorTaskTest
.withLogParseExceptions(false) .withLogParseExceptions(false)
.build(); .build();
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"datasource", .withDataSource("datasource")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
null, .build(),
null,
null
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );
@ -325,25 +322,24 @@ public class ParallelIndexSupervisorTaskTest
expectedException.expect(IAE.class); expectedException.expect(IAE.class);
expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser."); expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"datasource", .withDataSource("datasource")
mapper.convertValue( .withParserMap(
new StringInputRowParser( mapper.convertValue(
new JSONParseSpec( new StringInputRowParser(
new TimestampSpec(null, null, null), new JSONParseSpec(
DimensionsSpec.EMPTY, new TimestampSpec(null, null, null),
null, DimensionsSpec.EMPTY,
null, null,
null null,
) null
), )
Map.class ),
), Map.class
null, )
null, )
null, .withObjectMapper(mapper)
mapper .build(),
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );
@ -559,14 +555,11 @@ public class ParallelIndexSupervisorTaskTest
.build(); .build();
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec( final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"datasource", .withDataSource("datasource")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
null, .build(),
null,
null
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );

View File

@ -31,11 +31,9 @@ import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec; import org.apache.druid.timeline.partition.BuildingHashBasedNumberedShardSpec;
import org.apache.druid.timeline.partition.HashPartitionFunction; import org.apache.druid.timeline.partition.HashPartitionFunction;
import org.joda.time.Interval; import org.joda.time.Interval;
@ -97,16 +95,13 @@ class ParallelIndexTestingFactory
DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION)) DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION))
); );
return new DataSchema( return DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
timestampSpec, .withTimestamp(timestampSpec)
dimensionsSpec, .withDimensions(dimensionsSpec)
new AggregatorFactory[]{}, .withGranularity(granularitySpec)
granularitySpec, .withObjectMapper(NESTED_OBJECT_MAPPER)
TransformSpec.NONE, .build();
null,
NESTED_OBJECT_MAPPER
);
} }
static ParallelIndexIngestionSpec createIngestionSpec( static ParallelIndexIngestionSpec createIngestionSpec(

View File

@ -39,7 +39,6 @@ import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec; import org.apache.druid.segment.indexing.granularity.GranularitySpec;
@ -329,14 +328,13 @@ public class RangePartitionTaskKillTest extends AbstractMultiPhaseParallelIndexi
null null
); );
ingestionSpec = new ParallelIndexIngestionSpec( ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
timestampSpec, .withTimestamp(timestampSpec)
dimensionsSpec, .withDimensions(dimensionsSpec)
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")}, .withAggregators(new LongSumAggregatorFactory("val", "val"))
granularitySpec, .withGranularity(granularitySpec)
null .build(),
),
ioConfig, ioConfig,
tuningConfig tuningConfig
); );

View File

@ -43,7 +43,6 @@ import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.DataSegmentsWithSchemas; import org.apache.druid.segment.DataSegmentsWithSchemas;
@ -391,20 +390,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas), .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.MINUTE,
Granularities.MINUTE, Collections.singletonList(Intervals.of("2017-12/P1M"))
Collections.singletonList(Intervals.of("2017-12/P1M")) )
), )
null .build(),
),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
DEFAULT_INPUT_FORMAT, DEFAULT_INPUT_FORMAT,
@ -444,20 +442,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas), .withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, new UniformGranularitySpec(
new UniformGranularitySpec( Granularities.DAY,
Granularities.DAY, Granularities.MINUTE,
Granularities.MINUTE, Collections.singletonList(Intervals.of("2017-12/P1M"))
Collections.singletonList(Intervals.of("2017-12/P1M")) )
), )
null .build(),
),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true), new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
DEFAULT_INPUT_FORMAT, DEFAULT_INPUT_FORMAT,
@ -785,21 +782,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DimensionsSpec.builder() .withDimensions(
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) DimensionsSpec.builder()
.setIncludeAllDimensions(true) .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
.build(), .setIncludeAllDimensions(true)
new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, .build()
new UniformGranularitySpec( )
Granularities.DAY, .withAggregators(new CountAggregatorFactory("cnt"))
Granularities.MINUTE, .withGranularity(
Collections.singletonList(interval) new UniformGranularitySpec(
), Granularities.DAY,
null Granularities.MINUTE,
), Collections.singletonList(interval)
)
)
.build(),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, "*.json", true), new SettableSplittableLocalInputSource(inputDir, "*.json", true),
new JsonInputFormat( new JsonInputFormat(
@ -868,21 +868,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null, null,
null, null,
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DimensionsSpec.builder() .withDimensions(
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim")) DimensionsSpec.builder()
.useSchemaDiscovery(true) .setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
.build(), .useSchemaDiscovery(true)
new AggregatorFactory[]{new CountAggregatorFactory("cnt")}, .build()
new UniformGranularitySpec( )
Granularities.DAY, .withAggregators(new CountAggregatorFactory("cnt"))
Granularities.MINUTE, .withGranularity(
Collections.singletonList(interval) new UniformGranularitySpec(
), Granularities.DAY,
null Granularities.MINUTE,
), Collections.singletonList(interval)
)
)
.build(),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, "*.json", true), new SettableSplittableLocalInputSource(inputDir, "*.json", true),
new JsonInputFormat( new JsonInputFormat(
@ -948,20 +951,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
final ParallelIndexIngestionSpec ingestionSpec; final ParallelIndexIngestionSpec ingestionSpec;
if (useInputFormatApi) { if (useInputFormatApi) {
ingestionSpec = new ParallelIndexIngestionSpec( ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DEFAULT_DIMENSIONS_SPEC, .withDimensions(DEFAULT_DIMENSIONS_SPEC)
new AggregatorFactory[]{ .withAggregators(new LongSumAggregatorFactory("val", "val"))
new LongSumAggregatorFactory("val", "val") .withGranularity(
}, new UniformGranularitySpec(
new UniformGranularitySpec( segmentGranularity,
segmentGranularity, Granularities.MINUTE,
Granularities.MINUTE, interval == null ? null : Collections.singletonList(interval)
interval == null ? null : Collections.singletonList(interval) )
), )
null .build(),
),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource), new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource),
DEFAULT_INPUT_FORMAT, DEFAULT_INPUT_FORMAT,
@ -972,18 +974,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
); );
} else { } else {
ingestionSpec = new ParallelIndexIngestionSpec( ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
DEFAULT_TIMESTAMP_SPEC, .withTimestamp(DEFAULT_TIMESTAMP_SPEC)
DEFAULT_DIMENSIONS_SPEC, .withDimensions(DEFAULT_DIMENSIONS_SPEC)
DEFAULT_METRICS_SPEC, .withAggregators(DEFAULT_METRICS_SPEC)
new UniformGranularitySpec( .withGranularity(
segmentGranularity, new UniformGranularitySpec(
Granularities.MINUTE, segmentGranularity,
interval == null ? null : Collections.singletonList(interval) Granularities.MINUTE,
), interval == null ? null : Collections.singletonList(interval)
null )
), )
.build(),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
new LocalInputSource(inputDir, inputSourceFilter), new LocalInputSource(inputDir, inputSourceFilter),
createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC), createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC),

View File

@ -26,7 +26,6 @@ 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.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.common.TestUtils; import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.Resource; import org.apache.druid.server.security.Resource;
@ -48,14 +47,11 @@ public class SinglePhaseSubTaskSpecTest
"groupId", "groupId",
"supervisorTaskId", "supervisorTaskId",
new ParallelIndexIngestionSpec( new ParallelIndexIngestionSpec(
new DataSchema( DataSchema.builder()
"dataSource", .withDataSource("dataSource")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
new DimensionsSpec(null), .withDimensions(DimensionsSpec.builder().build())
new AggregatorFactory[0], .build(),
null,
null
),
new ParallelIndexIOConfig( new ParallelIndexIOConfig(
new LocalInputSource(new File("baseDir"), "filter"), new LocalInputSource(new File("baseDir"), "filter"),
new JsonInputFormat(null, null, null, null, null), new JsonInputFormat(null, null, null, null, null),

View File

@ -67,8 +67,7 @@ public class TombstoneHelperTest
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false, GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
Collections.singletonList(interval) Collections.singletonList(interval)
); );
DataSchema dataSchema = DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build();
new DataSchema("test", null, null, null, granularitySpec, null);
// no segments will be pushed when all rows are thrown away, assume that: // no segments will be pushed when all rows are thrown away, assume that:
List<DataSegment> pushedSegments = Collections.emptyList(); List<DataSegment> pushedSegments = Collections.emptyList();
@ -93,8 +92,7 @@ public class TombstoneHelperTest
GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false, GranularitySpec granularitySpec = new UniformGranularitySpec(Granularities.DAY, null, false,
Collections.singletonList(interval) Collections.singletonList(interval)
); );
DataSchema dataSchema = DataSchema dataSchema = DataSchema.builder().withDataSource("test").withGranularity(granularitySpec).build();
new DataSchema("test", null, null, null, granularitySpec, null);
// no segments will be pushed when all rows are thrown away, assume that: // no segments will be pushed when all rows are thrown away, assume that:
List<DataSegment> pushedSegments = Collections.emptyList(); List<DataSegment> pushedSegments = Collections.emptyList();

View File

@ -122,17 +122,19 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest
new DoubleDimensionSchema("d5") new DoubleDimensionSchema("d5")
) )
); );
DataSchema schema = new DataSchema( DataSchema schema =
"dataSourceName", DataSchema.builder()
new TimestampSpec(null, null, null), .withDataSource("dataSourceName")
dimensionsSpec, .withTimestamp(new TimestampSpec(null, null, null))
new AggregatorFactory[]{ .withDimensions(dimensionsSpec)
new CountAggregatorFactory("count"), .withAggregators(
new LongSumAggregatorFactory("met", "met") new CountAggregatorFactory("count"),
}, new LongSumAggregatorFactory("met", "met")
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), )
null .withGranularity(
); new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)
)
.build();
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec());
@ -154,14 +156,12 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest
new DoubleDimensionSchema("d5") new DoubleDimensionSchema("d5")
) )
); );
DataSchema schema = new DataSchema( DataSchema schema = DataSchema.builder()
"dataSourceName", .withDataSource("dataSourceName")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
dimensionsSpec, .withDimensions(dimensionsSpec)
new AggregatorFactory[]{}, .withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null))
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), .build();
null
);
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema); InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec()); Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec());

View File

@ -109,7 +109,6 @@ import org.apache.druid.query.DirectQueryProcessingPool;
import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.ForwardingQueryProcessingPool;
import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9Factory; import org.apache.druid.segment.IndexMergerV9Factory;
@ -672,18 +671,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null, null,
null, null,
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, .withAggregators(new DoubleSumAggregatorFactory("met", "met"))
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
null, Granularities.DAY,
ImmutableList.of(Intervals.of("2010-01-01/P2D")) null,
), ImmutableList.of(Intervals.of("2010-01-01/P2D"))
null )
), )
.build(),
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask() TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000) .withMaxRowsPerSegment(10000)
@ -735,18 +735,18 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null, null,
null, null,
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
null, .withAggregators(new DoubleSumAggregatorFactory("met", "met"))
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, .withGranularity(
new UniformGranularitySpec( new UniformGranularitySpec(
Granularities.DAY, Granularities.DAY,
null, null,
ImmutableList.of(Intervals.of("2010-01-01/P1D")) ImmutableList.of(Intervals.of("2010-01-01/P1D"))
), )
null, )
mapper .withObjectMapper(mapper)
), .build(),
new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false), new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask() TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000) .withMaxRowsPerSegment(10000)
@ -1165,18 +1165,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null, null,
null, null,
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, .withAggregators(new DoubleSumAggregatorFactory("met", "met"))
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
null, Granularities.DAY,
ImmutableList.of(Intervals.of("2010-01-01/P2D")) null,
), ImmutableList.of(Intervals.of("2010-01-01/P2D"))
null )
), )
.build(),
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask() TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000) .withMaxRowsPerSegment(10000)
@ -1253,18 +1254,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null, null,
null, null,
new IndexIngestionSpec( new IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
new TimestampSpec(null, null, null), .withTimestamp(new TimestampSpec(null, null, null))
DimensionsSpec.EMPTY, .withDimensions(DimensionsSpec.EMPTY)
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")}, .withAggregators(new DoubleSumAggregatorFactory("met", "met"))
new UniformGranularitySpec( .withGranularity(
Granularities.DAY, new UniformGranularitySpec(
null, Granularities.DAY,
ImmutableList.of(Intervals.of("2010-01-01/P2D")) null,
), ImmutableList.of(Intervals.of("2010-01-01/P2D"))
null )
), )
.build(),
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false), new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask() TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000) .withMaxRowsPerSegment(10000)

View File

@ -549,14 +549,15 @@ public class TaskQueueTest extends IngestionTestBase
new NoopTaskContextEnricher() new NoopTaskContextEnricher()
); );
final DataSchema dataSchema = new DataSchema( final DataSchema dataSchema =
"DS", DataSchema.builder()
new TimestampSpec(null, null, null), .withDataSource("DS")
new DimensionsSpec(null), .withTimestamp(new TimestampSpec(null, null, null))
null, .withDimensions(DimensionsSpec.builder().build())
new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null), .withGranularity(
null new UniformGranularitySpec(Granularities.YEAR, Granularities.DAY, null)
); )
.build();
final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig( final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
new HttpInputSource(Collections.singletonList(URI.create("http://host.org")), new HttpInputSource(Collections.singletonList(URI.create("http://host.org")),
"user", "user",

View File

@ -47,14 +47,11 @@ public class CsvInputSourceSamplerTest extends InitializedNullHandlingTest
{ {
final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970")); final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970"));
final DimensionsSpec dimensionsSpec = new DimensionsSpec(null); final DimensionsSpec dimensionsSpec = new DimensionsSpec(null);
final DataSchema dataSchema = new DataSchema( final DataSchema dataSchema = DataSchema.builder()
"sampler", .withDataSource("sampler")
timestampSpec, .withTimestamp(timestampSpec)
dimensionsSpec, .withDimensions(dimensionsSpec)
null, .build();
null,
null
);
final List<String> strCsvRows = ImmutableList.of( final List<String> strCsvRows = ImmutableList.of(
"FirstName,LastName,Number,Gender", "FirstName,LastName,Number,Gender",

View File

@ -66,14 +66,11 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
final SamplerResponse response = inputSourceSampler.sample( final SamplerResponse response = inputSourceSampler.sample(
inputSource, inputSource,
new JsonInputFormat(null, null, null, null, null), new JsonInputFormat(null, null, null, null, null),
new DataSchema( DataSchema.builder()
"test", .withDataSource("test")
new TimestampSpec("t", null, null), .withTimestamp(new TimestampSpec("t", null, null))
DimensionsSpec.builder().useSchemaDiscovery(true).build(), .withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build())
null, .build(),
null,
null
),
null null
); );
@ -131,14 +128,11 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
final SamplerResponse response = inputSourceSampler.sample( final SamplerResponse response = inputSourceSampler.sample(
inputSource, inputSource,
new JsonInputFormat(null, null, null, null, null), new JsonInputFormat(null, null, null, null, null),
new DataSchema( DataSchema.builder()
"test", .withDataSource("test")
new TimestampSpec("t", null, null), .withTimestamp(new TimestampSpec("t", null, null))
DimensionsSpec.builder().useSchemaDiscovery(true).build(), .withDimensions(DimensionsSpec.builder().useSchemaDiscovery(true).build())
null, .build(),
null,
null
),
null null
); );
@ -189,14 +183,12 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
public void testTypesClassicDiscovery() public void testTypesClassicDiscovery()
{ {
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n')); final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
final DataSchema dataSchema = new DataSchema( final DataSchema dataSchema =
"test", DataSchema.builder()
new TimestampSpec("t", null, null), .withDataSource("test")
DimensionsSpec.builder().build(), .withTimestamp(new TimestampSpec("t", null, null))
null, .withDimensions(DimensionsSpec.builder().build())
null, .build();
null
);
final SamplerResponse response = inputSourceSampler.sample( final SamplerResponse response = inputSourceSampler.sample(
inputSource, inputSource,
new JsonInputFormat(null, null, null, null, null), new JsonInputFormat(null, null, null, null, null),
@ -248,23 +240,20 @@ public class InputSourceSamplerDiscoveryTest extends InitializedNullHandlingTest
public void testTypesNoDiscoveryExplicitSchema() public void testTypesNoDiscoveryExplicitSchema()
{ {
final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n')); final InputSource inputSource = new InlineInputSource(Strings.join(STR_JSON_ROWS, '\n'));
final DataSchema dataSchema = new DataSchema( final DataSchema dataSchema =
"test", DataSchema.builder()
new TimestampSpec("t", null, null), .withDataSource("test")
DimensionsSpec.builder().setDimensions( .withTimestamp(new TimestampSpec("t", null, null))
ImmutableList.of(new StringDimensionSchema("string"), .withDimensions(
new LongDimensionSchema("long"), new StringDimensionSchema("string"),
new DoubleDimensionSchema("double"), new LongDimensionSchema("long"),
new StringDimensionSchema("bool"), new DoubleDimensionSchema("double"),
new AutoTypeColumnSchema("variant", null), new StringDimensionSchema("bool"),
new AutoTypeColumnSchema("array", null), new AutoTypeColumnSchema("variant", null),
new AutoTypeColumnSchema("nested", null) new AutoTypeColumnSchema("array", null),
) new AutoTypeColumnSchema("nested", null)
).build(), )
null, .build();
null,
null
);
final SamplerResponse response = inputSourceSampler.sample( final SamplerResponse response = inputSourceSampler.sample(
inputSource, inputSource,
new JsonInputFormat(null, null, null, null, null), new JsonInputFormat(null, null, null, null, null),

View File

@ -1497,24 +1497,24 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
) throws IOException ) throws IOException
{ {
if (useInputFormatApi) { if (useInputFormatApi) {
return new DataSchema( return DataSchema.builder()
"sampler", .withDataSource("sampler")
timestampSpec, .withTimestamp(timestampSpec)
dimensionsSpec, .withDimensions(dimensionsSpec)
aggregators, .withAggregators(aggregators)
granularitySpec, .withGranularity(granularitySpec)
transformSpec .withTransform(transformSpec)
); .build();
} else { } else {
final Map<String, Object> parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec)); final Map<String, Object> parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec));
return new DataSchema( return DataSchema.builder()
"sampler", .withDataSource("sampler")
parserMap, .withParserMap(parserMap)
aggregators, .withAggregators(aggregators)
granularitySpec, .withGranularity(granularitySpec)
transformSpec, .withTransform(transformSpec)
OBJECT_MAPPER .withObjectMapper(OBJECT_MAPPER)
); .build();
} }
} }

View File

@ -33,10 +33,8 @@ import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
import org.apache.druid.indexing.seekablestream.common.StreamPartition; import org.apache.druid.indexing.seekablestream.common.StreamPartition;
import org.apache.druid.java.util.common.DateTimes; import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.granularity.AllGranularity; import org.apache.druid.java.util.common.granularity.AllGranularity;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec; import org.apache.druid.segment.indexing.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.security.Access; import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.Action; import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthConfig; import org.apache.druid.server.security.AuthConfig;
@ -107,16 +105,13 @@ public class SeekableStreamIndexTaskRunnerAuthTest
} }
}; };
DataSchema dataSchema = new DataSchema( DataSchema dataSchema =
"datasource", DataSchema.builder()
new TimestampSpec(null, null, null), .withDataSource("datasource")
new DimensionsSpec(Collections.emptyList()), .withTimestamp(new TimestampSpec(null, null, null))
new AggregatorFactory[]{}, .withDimensions(new DimensionsSpec(Collections.emptyList()))
new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()), .withGranularity(new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()))
TransformSpec.NONE, .build();
null,
null
);
SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class); SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class);
SeekableStreamIndexTaskIOConfig<String, String> ioConfig = new TestSeekableStreamIndexTaskIOConfig(); SeekableStreamIndexTaskIOConfig<String, String> ioConfig = new TestSeekableStreamIndexTaskIOConfig();

View File

@ -95,7 +95,6 @@ import org.apache.druid.query.QueryPlus;
import org.apache.druid.query.QueryRunnerFactoryConglomerate; import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.Result; import org.apache.druid.query.Result;
import org.apache.druid.query.SegmentDescriptor; import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
@ -164,25 +163,23 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
protected static final ObjectMapper OBJECT_MAPPER; protected static final ObjectMapper OBJECT_MAPPER;
protected static final DataSchema OLD_DATA_SCHEMA; protected static final DataSchema OLD_DATA_SCHEMA;
protected static final DataSchema NEW_DATA_SCHEMA = new DataSchema( protected static final DataSchema NEW_DATA_SCHEMA =
"test_ds", DataSchema.builder()
new TimestampSpec("timestamp", "iso", null), .withDataSource("test_ds")
new DimensionsSpec( .withTimestamp(new TimestampSpec("timestamp", "iso", null))
Arrays.asList( .withDimensions(
new StringDimensionSchema("dim1"), new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"), new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"), new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"), new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat") new FloatDimensionSchema("dimFloat")
) )
), .withAggregators(
new AggregatorFactory[]{ new DoubleSumAggregatorFactory("met1sum", "met1"),
new DoubleSumAggregatorFactory("met1sum", "met1"), new CountAggregatorFactory("rows")
new CountAggregatorFactory("rows") )
}, .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), .build();
null
);
protected static final InputFormat INPUT_FORMAT = new JsonInputFormat( protected static final InputFormat INPUT_FORMAT = new JsonInputFormat(
new JSONPathSpec(true, ImmutableList.of()), new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(), ImmutableMap.of(),
@ -211,37 +208,38 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
static { static {
OBJECT_MAPPER = new TestUtils().getTestObjectMapper(); OBJECT_MAPPER = new TestUtils().getTestObjectMapper();
OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json")); OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json"));
OLD_DATA_SCHEMA = new DataSchema( OLD_DATA_SCHEMA = DataSchema.builder()
"test_ds", .withDataSource("test_ds")
OBJECT_MAPPER.convertValue( .withParserMap(
new StringInputRowParser( OBJECT_MAPPER.convertValue(
new JSONParseSpec( new StringInputRowParser(
new TimestampSpec("timestamp", "iso", null), new JSONParseSpec(
new DimensionsSpec( new TimestampSpec("timestamp", "iso", null),
Arrays.asList( new DimensionsSpec(
new StringDimensionSchema("dim1"), Arrays.asList(
new StringDimensionSchema("dim1t"), new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2"), new StringDimensionSchema("dim1t"),
new LongDimensionSchema("dimLong"), new StringDimensionSchema("dim2"),
new FloatDimensionSchema("dimFloat") new LongDimensionSchema("dimLong"),
) new FloatDimensionSchema("dimFloat")
), )
new JSONPathSpec(true, ImmutableList.of()), ),
ImmutableMap.of(), new JSONPathSpec(true, ImmutableList.of()),
false ImmutableMap.of(),
), false
StandardCharsets.UTF_8.name() ),
), StandardCharsets.UTF_8.name()
Map.class ),
), Map.class
new AggregatorFactory[]{ )
new DoubleSumAggregatorFactory("met1sum", "met1"), )
new CountAggregatorFactory("rows") .withAggregators(
}, new DoubleSumAggregatorFactory("met1sum", "met1"),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), new CountAggregatorFactory("rows")
null, )
OBJECT_MAPPER .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
); .withObjectMapper(OBJECT_MAPPER)
.build();
} }
public SeekableStreamIndexTaskTestBase( public SeekableStreamIndexTaskTestBase(

View File

@ -48,7 +48,6 @@ import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.parsers.JSONPathSpec; import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory; import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
@ -106,36 +105,37 @@ public class SeekableStreamSamplerSpecTest extends EasyMockSupport
@Test(timeout = 10_000L) @Test(timeout = 10_000L)
public void testSampleWithInputRowParser() throws Exception public void testSampleWithInputRowParser() throws Exception
{ {
final DataSchema dataSchema = new DataSchema( DataSchema dataSchema = DataSchema.builder()
"test_ds", .withDataSource("test_ds")
OBJECT_MAPPER.convertValue( .withParserMap(
new StringInputRowParser( OBJECT_MAPPER.convertValue(
new JSONParseSpec( new StringInputRowParser(
new TimestampSpec("timestamp", "iso", null), new JSONParseSpec(
new DimensionsSpec( new TimestampSpec("timestamp", "iso", null),
Arrays.asList( new DimensionsSpec(
new StringDimensionSchema("dim1"), Arrays.asList(
new StringDimensionSchema("dim1t"), new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2"), new StringDimensionSchema("dim1t"),
new LongDimensionSchema("dimLong"), new StringDimensionSchema("dim2"),
new FloatDimensionSchema("dimFloat") new LongDimensionSchema("dimLong"),
) new FloatDimensionSchema("dimFloat")
), )
new JSONPathSpec(true, ImmutableList.of()), ),
ImmutableMap.of(), new JSONPathSpec(true, ImmutableList.of()),
false ImmutableMap.of(),
) false
), )
Map.class ),
), Map.class
new AggregatorFactory[]{ )
new DoubleSumAggregatorFactory("met1sum", "met1"), )
new CountAggregatorFactory("rows") .withAggregators(
}, new DoubleSumAggregatorFactory("met1sum", "met1"),
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null), new CountAggregatorFactory("rows")
null, )
OBJECT_MAPPER .withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
); .withObjectMapper(OBJECT_MAPPER)
.build();
final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig( final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig(
STREAM, STREAM,

View File

@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
@ -59,7 +58,6 @@ import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig; import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.java.util.metrics.StubServiceEmitter;
import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.DruidMetrics;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
@ -1261,18 +1259,19 @@ public class SeekableStreamSupervisorSpecTest extends EasyMockSupport
dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim1"));
dimensions.add(StringDimensionSchema.create("dim2")); dimensions.add(StringDimensionSchema.create("dim2"));
return new DataSchema( return DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
new TimestampSpec("timestamp", "iso", null), .withTimestamp(new TimestampSpec("timestamp", "iso", null))
new DimensionsSpec(dimensions), .withDimensions(dimensions)
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, .withAggregators(new CountAggregatorFactory("rows"))
new UniformGranularitySpec( .withGranularity(
Granularities.HOUR, new UniformGranularitySpec(
Granularities.NONE, Granularities.HOUR,
ImmutableList.of() Granularities.NONE,
), ImmutableList.of()
null )
); )
.build();
} }
private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scaleOut) private SeekableStreamSupervisorIOConfig getIOConfig(int taskCount, boolean scaleOut)

View File

@ -31,7 +31,6 @@ import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture; import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.data.input.impl.ByteEntity; import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.DimensionSchema; import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat; import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.StringDimensionSchema; import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.TimestampSpec; import org.apache.druid.data.input.impl.TimestampSpec;
@ -83,7 +82,6 @@ import org.apache.druid.java.util.metrics.DruidMonitorSchedulerConfig;
import org.apache.druid.java.util.metrics.StubServiceEmitter; import org.apache.druid.java.util.metrics.StubServiceEmitter;
import org.apache.druid.metadata.PendingSegmentRecord; import org.apache.druid.metadata.PendingSegmentRecord;
import org.apache.druid.query.DruidMetrics; import org.apache.druid.query.DruidMetrics;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper; import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory; import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
@ -2562,18 +2560,19 @@ public class SeekableStreamSupervisorStateTest extends EasyMockSupport
dimensions.add(StringDimensionSchema.create("dim1")); dimensions.add(StringDimensionSchema.create("dim1"));
dimensions.add(StringDimensionSchema.create("dim2")); dimensions.add(StringDimensionSchema.create("dim2"));
return new DataSchema( return DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
new TimestampSpec("timestamp", "iso", null), .withTimestamp(new TimestampSpec("timestamp", "iso", null))
new DimensionsSpec(dimensions), .withDimensions(dimensions)
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, .withAggregators(new CountAggregatorFactory("rows"))
new UniformGranularitySpec( .withGranularity(
Granularities.HOUR, new UniformGranularitySpec(
Granularities.NONE, Granularities.HOUR,
ImmutableList.of() Granularities.NONE,
), ImmutableList.of()
null )
); )
.build();
} }
private static SeekableStreamSupervisorIOConfig getIOConfig() private static SeekableStreamSupervisorIOConfig getIOConfig()

View File

@ -28,7 +28,6 @@ import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.Task; import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.junit.Assert; import org.junit.Assert;
import org.junit.Test; import org.junit.Test;
@ -51,7 +50,7 @@ public class TaskAnnouncementTest
"theid", "theid",
new TaskResource("rofl", 2), new TaskResource("rofl", 2),
new IndexTask.IndexIngestionSpec( new IndexTask.IndexIngestionSpec(
new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()), DataSchema.builder().withDataSource("foo").withObjectMapper(new DefaultObjectMapper()).build(),
ioConfig, ioConfig,
null null
), ),

View File

@ -25,7 +25,6 @@ import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonInclude.Include; import com.fasterxml.jackson.annotation.JsonInclude.Include;
import com.fasterxml.jackson.annotation.JsonProperty; import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper; import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions; import com.google.common.base.Preconditions;
import com.google.common.base.Strings; import com.google.common.base.Strings;
import com.google.common.collect.Multiset; import com.google.common.collect.Multiset;
@ -64,6 +63,17 @@ import java.util.stream.Collectors;
public class DataSchema public class DataSchema
{ {
private static final Logger log = new Logger(DataSchema.class); private static final Logger log = new Logger(DataSchema.class);
public static Builder builder()
{
return new Builder();
}
public static Builder builder(DataSchema schema)
{
return new Builder(schema);
}
private final String dataSource; private final String dataSource;
private final AggregatorFactory[] aggregators; private final AggregatorFactory[] aggregators;
private final GranularitySpec granularitySpec; private final GranularitySpec granularitySpec;
@ -126,33 +136,6 @@ public class DataSchema
} }
} }
@VisibleForTesting
public DataSchema(
String dataSource,
TimestampSpec timestampSpec,
DimensionsSpec dimensionsSpec,
AggregatorFactory[] aggregators,
GranularitySpec granularitySpec,
TransformSpec transformSpec
)
{
this(dataSource, timestampSpec, dimensionsSpec, aggregators, granularitySpec, transformSpec, null, null);
}
// old constructor for backward compatibility
@Deprecated
public DataSchema(
String dataSource,
Map<String, Object> parserMap,
AggregatorFactory[] aggregators,
GranularitySpec granularitySpec,
TransformSpec transformSpec,
ObjectMapper objectMapper
)
{
this(dataSource, null, null, aggregators, granularitySpec, transformSpec, parserMap, objectMapper);
}
private static void validateDatasourceName(String dataSource) private static void validateDatasourceName(String dataSource)
{ {
IdUtils.validateId("dataSource", dataSource); IdUtils.validateId("dataSource", dataSource);
@ -403,44 +386,17 @@ public class DataSchema
public DataSchema withGranularitySpec(GranularitySpec granularitySpec) public DataSchema withGranularitySpec(GranularitySpec granularitySpec)
{ {
return new DataSchema( return builder(this).withGranularity(granularitySpec).build();
dataSource,
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec,
parserMap,
objectMapper
);
} }
public DataSchema withTransformSpec(TransformSpec transformSpec) public DataSchema withTransformSpec(TransformSpec transformSpec)
{ {
return new DataSchema( return builder(this).withTransform(transformSpec).build();
dataSource,
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec,
parserMap,
objectMapper
);
} }
public DataSchema withDimensionsSpec(DimensionsSpec dimensionsSpec) public DataSchema withDimensionsSpec(DimensionsSpec dimensionsSpec)
{ {
return new DataSchema( return builder(this).withDimensions(dimensionsSpec).build();
dataSource,
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec,
parserMap,
objectMapper
);
} }
@Override @Override
@ -457,4 +413,110 @@ public class DataSchema
", inputRowParser=" + inputRowParser + ", inputRowParser=" + inputRowParser +
'}'; '}';
} }
public static class Builder
{
private String dataSource;
private AggregatorFactory[] aggregators;
private GranularitySpec granularitySpec;
private TransformSpec transformSpec;
private Map<String, Object> parserMap;
private ObjectMapper objectMapper;
// The below fields can be initialized lazily from parser for backward compatibility.
private TimestampSpec timestampSpec;
private DimensionsSpec dimensionsSpec;
public Builder()
{
}
public Builder(DataSchema schema)
{
this.dataSource = schema.dataSource;
this.aggregators = schema.aggregators;
this.granularitySpec = schema.granularitySpec;
this.transformSpec = schema.transformSpec;
this.parserMap = schema.parserMap;
this.objectMapper = schema.objectMapper;
this.timestampSpec = schema.timestampSpec;
this.dimensionsSpec = schema.dimensionsSpec;
}
public Builder withDataSource(String dataSource)
{
this.dataSource = dataSource;
return this;
}
public Builder withTimestamp(TimestampSpec timestampSpec)
{
this.timestampSpec = timestampSpec;
return this;
}
public Builder withDimensions(DimensionsSpec dimensionsSpec)
{
this.dimensionsSpec = dimensionsSpec;
return this;
}
public Builder withDimensions(List<DimensionSchema> dimensions)
{
this.dimensionsSpec = DimensionsSpec.builder().setDimensions(dimensions).build();
return this;
}
public Builder withDimensions(DimensionSchema... dimensions)
{
return withDimensions(Arrays.asList(dimensions));
}
public Builder withAggregators(AggregatorFactory... aggregators)
{
this.aggregators = aggregators;
return this;
}
public Builder withGranularity(GranularitySpec granularitySpec)
{
this.granularitySpec = granularitySpec;
return this;
}
public Builder withTransform(TransformSpec transformSpec)
{
this.transformSpec = transformSpec;
return this;
}
@Deprecated
public Builder withObjectMapper(ObjectMapper objectMapper)
{
this.objectMapper = objectMapper;
return this;
}
@Deprecated
public Builder withParserMap(Map<String, Object> parserMap)
{
this.parserMap = parserMap;
return this;
}
public DataSchema build()
{
return new DataSchema(
dataSource,
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec,
parserMap,
objectMapper
);
}
}
} }

View File

@ -71,6 +71,11 @@ import java.util.Set;
public class DataSchemaTest extends InitializedNullHandlingTest public class DataSchemaTest extends InitializedNullHandlingTest
{ {
private static ArbitraryGranularitySpec ARBITRARY_GRANULARITY = new ArbitraryGranularitySpec(
Granularities.DAY,
ImmutableList.of(Intervals.of("2014/2015"))
);
@Rule @Rule
public ExpectedException expectedException = ExpectedException.none(); public ExpectedException expectedException = ExpectedException.none();
@ -92,17 +97,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
); );
DataSchema schema = new DataSchema( DataSchema schema = DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
parser, .withParserMap(parser)
new AggregatorFactory[]{ .withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"), new DoubleSumAggregatorFactory("metric2", "col2")
}, )
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .withGranularity(ARBITRARY_GRANULARITY)
null, .withObjectMapper(jsonMapper)
jsonMapper .build();
);
Assert.assertEquals( Assert.assertEquals(
ImmutableSet.of("__time", "time", "col1", "col2", "metric1", "metric2"), ImmutableSet.of("__time", "time", "col1", "col2", "metric1", "metric2"),
@ -130,18 +134,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
null null
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
); );
DataSchema schema = DataSchema.builder()
DataSchema schema = new DataSchema( .withDataSource(IdUtilsTest.VALID_ID_CHARS)
IdUtilsTest.VALID_ID_CHARS, .withParserMap(parser)
parser, .withAggregators(
new AggregatorFactory[]{ new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric2", "col2")
new DoubleSumAggregatorFactory("metric2", "col2"), )
}, .withGranularity(ARBITRARY_GRANULARITY)
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .withObjectMapper(jsonMapper)
null, .build();
jsonMapper
);
Assert.assertEquals( Assert.assertEquals(
ImmutableSet.of("__time", "dimC", "col1", "metric1", "metric2"), ImmutableSet.of("__time", "dimC", "col1", "metric1", "metric2"),
@ -167,22 +169,28 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
); );
DataSchema schema = new DataSchema( DataSchema schema = DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
parserMap, .withParserMap(parserMap)
new AggregatorFactory[]{ .withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"), new DoubleSumAggregatorFactory("metric2", "col2")
}, )
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .withGranularity(ARBITRARY_GRANULARITY)
new TransformSpec( .withTransform(
new SelectorDimFilter("dimA", "foo", null), new TransformSpec(
ImmutableList.of( new SelectorDimFilter("dimA", "foo", null),
new ExpressionTransform("expr", "concat(dimA,dimA)", TestExprMacroTable.INSTANCE) ImmutableList.of(
) new ExpressionTransform(
), "expr",
jsonMapper "concat(dimA,dimA)",
); TestExprMacroTable.INSTANCE
)
)
)
)
.withObjectMapper(jsonMapper)
.build();
// Test hack that produces a StringInputRowParser. // Test hack that produces a StringInputRowParser.
final StringInputRowParser parser = (StringInputRowParser) schema.getParser(); final StringInputRowParser parser = (StringInputRowParser) schema.getParser();
@ -233,17 +241,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
); );
DataSchema schema = new DataSchema( DataSchema schema = DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
parser, .withParserMap(parser)
new AggregatorFactory[]{ .withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"), new DoubleSumAggregatorFactory("metric2", "col2")
}, )
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .withGranularity(ARBITRARY_GRANULARITY)
null, .withObjectMapper(jsonMapper)
jsonMapper .build();
);
expectedException.expect(DruidException.class); expectedException.expect(DruidException.class);
expectedException.expectMessage( expectedException.expectMessage(
@ -256,25 +263,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest
@Test @Test
public void testOverlapTimeAndDimPositionZero() public void testOverlapTimeAndDimPositionZero()
{ {
DataSchema schema = new DataSchema( DataSchema schema = DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
new TimestampSpec("time", "auto", null), .withTimestamp(new TimestampSpec("time", "auto", null))
DimensionsSpec.builder() .withDimensions(
.setDimensions( DimensionsSpec.builder()
ImmutableList.of( .setDimensions(
new LongDimensionSchema("__time"), ImmutableList.of(
new StringDimensionSchema("dimA"), new LongDimensionSchema("__time"),
new StringDimensionSchema("dimB") new StringDimensionSchema("dimA"),
) new StringDimensionSchema("dimB")
) )
.setDimensionExclusions(ImmutableList.of("dimC")) )
.build(), .setDimensionExclusions(ImmutableList.of("dimC"))
null, .build()
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), )
null, .withGranularity(ARBITRARY_GRANULARITY)
null, .withObjectMapper(jsonMapper)
jsonMapper .build();
);
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of("__time", "dimA", "dimB"), ImmutableList.of("__time", "dimA", "dimB"),
@ -290,25 +296,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest
expectedException.expect(DruidException.class); expectedException.expect(DruidException.class);
expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'.");
DataSchema schema = new DataSchema( DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
new TimestampSpec("time", "auto", null), .withTimestamp(new TimestampSpec("time", "auto", null))
DimensionsSpec.builder() .withDimensions(
.setDimensions( DimensionsSpec.builder()
ImmutableList.of( .setDimensions(
new StringDimensionSchema("__time"), ImmutableList.of(
new StringDimensionSchema("dimA"), new StringDimensionSchema("__time"),
new StringDimensionSchema("dimB") new StringDimensionSchema("dimA"),
) new StringDimensionSchema("dimB")
) )
.setDimensionExclusions(ImmutableList.of("dimC")) )
.build(), .setDimensionExclusions(ImmutableList.of("dimC"))
null, .build()
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), )
null, .withGranularity(ARBITRARY_GRANULARITY)
null, .withObjectMapper(jsonMapper)
jsonMapper .build();
);
} }
@Test @Test
@ -321,50 +326,49 @@ public class DataSchemaTest extends InitializedNullHandlingTest
+ DimensionsSpec.WARNING_NON_TIME_SORT_ORDER + DimensionsSpec.WARNING_NON_TIME_SORT_ORDER
); );
DataSchema schema = new DataSchema( DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
new TimestampSpec("time", "auto", null), .withTimestamp(new TimestampSpec("time", "auto", null))
DimensionsSpec.builder() .withDimensions(
.setDimensions( DimensionsSpec.builder()
ImmutableList.of( .setDimensions(
new StringDimensionSchema("dimA"), ImmutableList.of(
new LongDimensionSchema("__time"), new StringDimensionSchema("dimA"),
new StringDimensionSchema("dimB") new LongDimensionSchema("__time"),
) new StringDimensionSchema("dimB")
) )
.setDimensionExclusions(ImmutableList.of("dimC")) )
.build(), .setDimensionExclusions(ImmutableList.of("dimC"))
null, .build()
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), )
null, .withGranularity(ARBITRARY_GRANULARITY)
null, .withObjectMapper(jsonMapper)
jsonMapper .build();
);
} }
@Test @Test
public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder() public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder()
{ {
DataSchema schema = new DataSchema( DataSchema schema =
IdUtilsTest.VALID_ID_CHARS, DataSchema.builder()
new TimestampSpec("time", "auto", null), .withDataSource(IdUtilsTest.VALID_ID_CHARS)
DimensionsSpec.builder() .withTimestamp(new TimestampSpec("time", "auto", null))
.setDimensions( .withDimensions(
ImmutableList.of( DimensionsSpec.builder()
new StringDimensionSchema("dimA"), .setDimensions(
new LongDimensionSchema("__time"), ImmutableList.of(
new StringDimensionSchema("dimB") new StringDimensionSchema("dimA"),
) new LongDimensionSchema("__time"),
) new StringDimensionSchema("dimB")
.setDimensionExclusions(ImmutableList.of("dimC")) )
.setForceSegmentSortByTime(false) )
.build(), .setDimensionExclusions(ImmutableList.of("dimC"))
null, .setForceSegmentSortByTime(false)
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .build()
null, )
null, .withGranularity(ARBITRARY_GRANULARITY)
jsonMapper .withObjectMapper(jsonMapper)
); .build();
Assert.assertEquals( Assert.assertEquals(
ImmutableList.of("dimA", "__time", "dimB"), ImmutableList.of("dimA", "__time", "dimB"),
@ -402,14 +406,13 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
); );
DataSchema schema = new DataSchema( DataSchema schema = DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
parser, .withParserMap(parser)
null, .withGranularity(ARBITRARY_GRANULARITY)
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .withObjectMapper(jsonMapper)
null, .build();
jsonMapper
);
expectedException.expect(DruidException.class); expectedException.expect(DruidException.class);
expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'."); expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'.");
@ -442,20 +445,19 @@ public class DataSchemaTest extends InitializedNullHandlingTest
+ "[metric3] seen in metricsSpec list (2 occurrences)" + "[metric3] seen in metricsSpec list (2 occurrences)"
); );
DataSchema schema = new DataSchema( DataSchema schema = DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
parser, .withParserMap(parser)
new AggregatorFactory[]{ .withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"), new DoubleSumAggregatorFactory("metric2", "col2"),
new DoubleSumAggregatorFactory("metric1", "col3"), new DoubleSumAggregatorFactory("metric1", "col3"),
new DoubleSumAggregatorFactory("metric3", "col4"), new DoubleSumAggregatorFactory("metric3", "col4"),
new DoubleSumAggregatorFactory("metric3", "col5"), new DoubleSumAggregatorFactory("metric3", "col5")
}, )
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .withGranularity(ARBITRARY_GRANULARITY)
null, .withObjectMapper(jsonMapper)
jsonMapper .build();
);
} }
@Test @Test
@ -510,24 +512,20 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
); );
DruidExceptionMatcher DruidExceptionMatcher.ThrowingSupplier thrower =
.invalidInput() () -> DataSchema.builder()
.expectMessageIs("Invalid value for field [dataSource]: must not be null") .withDataSource("")
.assertThrowsAndMatches( .withParserMap(parser)
() -> new DataSchema( .withAggregators(
"", new DoubleSumAggregatorFactory("metric1", "col1"),
parser, new DoubleSumAggregatorFactory("metric2", "col2")
new AggregatorFactory[]{ )
new DoubleSumAggregatorFactory("metric1", "col1"), .withGranularity(ARBITRARY_GRANULARITY)
new DoubleSumAggregatorFactory("metric2", "col2"), .withObjectMapper(jsonMapper)
}, .build();
new ArbitraryGranularitySpec( DruidExceptionMatcher.invalidInput()
Granularities.DAY, .expectMessageIs("Invalid value for field [dataSource]: must not be null")
ImmutableList.of(Intervals.of("2014/2015")) .assertThrowsAndMatches(thrower);
),
null,
jsonMapper
));
} }
@ -547,14 +545,11 @@ public class DataSchemaTest extends InitializedNullHandlingTest
dataSource dataSource
); );
DruidExceptionMatcher.invalidInput().expectMessageIs(msg).assertThrowsAndMatches( DruidExceptionMatcher.invalidInput().expectMessageIs(msg).assertThrowsAndMatches(
() -> new DataSchema( () -> DataSchema.builder()
dataSource, .withDataSource(dataSource)
Collections.emptyMap(), .withParserMap(Collections.emptyMap())
null, .withObjectMapper(jsonMapper)
null, .build()
null,
jsonMapper
)
); );
} }
} }
@ -686,17 +681,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT ), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
); );
DataSchema originalSchema = new DataSchema( DataSchema originalSchema = DataSchema.builder()
IdUtilsTest.VALID_ID_CHARS, .withDataSource(IdUtilsTest.VALID_ID_CHARS)
parser, .withParserMap(parser)
new AggregatorFactory[]{ .withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"), new DoubleSumAggregatorFactory("metric2", "col2")
}, )
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), .withGranularity(ARBITRARY_GRANULARITY)
null, .withObjectMapper(jsonMapper)
jsonMapper .build();
);
String serialized = jsonMapper.writeValueAsString(originalSchema); String serialized = jsonMapper.writeValueAsString(originalSchema);
TestModifiedDataSchema deserialized = jsonMapper.readValue(serialized, TestModifiedDataSchema.class); TestModifiedDataSchema deserialized = jsonMapper.readValue(serialized, TestModifiedDataSchema.class);
@ -734,7 +728,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest
new DoubleSumAggregatorFactory("metric1", "col1"), new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"), new DoubleSumAggregatorFactory("metric2", "col2"),
}, },
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), ARBITRARY_GRANULARITY,
null, null,
parser, parser,
jsonMapper, jsonMapper,
@ -765,10 +759,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
Map<String, Object> parserMap = Mockito.mock(Map.class); Map<String, Object> parserMap = Mockito.mock(Map.class);
Mockito.when(newDimSpec.withDimensionExclusions(ArgumentMatchers.any(Set.class))).thenReturn(newDimSpec); Mockito.when(newDimSpec.withDimensionExclusions(ArgumentMatchers.any(Set.class))).thenReturn(newDimSpec);
DataSchema oldSchema = new DataSchema("dataSource", tsSpec, oldDimSpec, DataSchema oldSchema = DataSchema.builder()
new AggregatorFactory[]{aggFactory}, gSpec, .withDataSource("dataSource")
transSpec, parserMap, jsonMapper .withTimestamp(tsSpec)
); .withDimensions(oldDimSpec)
.withAggregators(aggFactory)
.withGranularity(gSpec)
.withTransform(transSpec)
.withParserMap(parserMap)
.withObjectMapper(jsonMapper)
.build();
DataSchema newSchema = oldSchema.withDimensionsSpec(newDimSpec); DataSchema newSchema = oldSchema.withDimensionsSpec(newDimSpec);
Assert.assertSame(oldSchema.getDataSource(), newSchema.getDataSource()); Assert.assertSame(oldSchema.getDataSource(), newSchema.getDataSource());
Assert.assertSame(oldSchema.getTimestampSpec(), newSchema.getTimestampSpec()); Assert.assertSame(oldSchema.getTimestampSpec(), newSchema.getTimestampSpec());
@ -795,7 +795,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest
.setDimensionExclusions(ImmutableList.of("dimC")) .setDimensionExclusions(ImmutableList.of("dimC"))
.build(), .build(),
null, null,
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), ARBITRARY_GRANULARITY,
null, null,
multiValuedDimensions multiValuedDimensions
); );

View File

@ -30,7 +30,6 @@ import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.emitter.EmittingLogger; import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.java.util.emitter.core.NoopEmitter; import org.apache.druid.java.util.emitter.core.NoopEmitter;
import org.apache.druid.java.util.emitter.service.ServiceEmitter; import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.IndexIO; import org.apache.druid.segment.IndexIO;
@ -151,19 +150,18 @@ public class BatchAppenderatorTester implements AutoCloseable
Map.class Map.class
); );
schema = new DataSchema( schema = DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
null, .withAggregators(
null, new CountAggregatorFactory("count"),
new AggregatorFactory[]{ new LongSumAggregatorFactory("met", "met")
new CountAggregatorFactory("count"), )
new LongSumAggregatorFactory("met", "met") .withGranularity(
}, new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), )
null, .withParserMap(parserMap)
parserMap, .withObjectMapper(objectMapper)
objectMapper .build();
);
tuningConfig = new TestAppenderatorConfig( tuningConfig = new TestAppenderatorConfig(
TuningConfig.DEFAULT_APPENDABLE_INDEX, TuningConfig.DEFAULT_APPENDABLE_INDEX,

View File

@ -44,7 +44,6 @@ import org.apache.druid.query.DefaultGenericQueryMetricsFactory;
import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate; import org.apache.druid.query.DefaultQueryRunnerFactoryConglomerate;
import org.apache.druid.query.ForwardingQueryProcessingPool; import org.apache.druid.query.ForwardingQueryProcessingPool;
import org.apache.druid.query.QueryRunnerTestHelper; import org.apache.druid.query.QueryRunnerTestHelper;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory; import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.expression.TestExprMacroTable; import org.apache.druid.query.expression.TestExprMacroTable;
@ -135,17 +134,16 @@ public class StreamAppenderatorTester implements AutoCloseable
), ),
Map.class Map.class
); );
schema = new DataSchema( schema = DataSchema.builder()
DATASOURCE, .withDataSource(DATASOURCE)
parserMap, .withParserMap(parserMap)
new AggregatorFactory[]{ .withAggregators(
new CountAggregatorFactory("count"), new CountAggregatorFactory("count"),
new LongSumAggregatorFactory("met", "met") new LongSumAggregatorFactory("met", "met")
}, )
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null), .withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null))
null, .withObjectMapper(objectMapper)
objectMapper .build();
);
tuningConfig = new TestAppenderatorConfig( tuningConfig = new TestAppenderatorConfig(
TuningConfig.DEFAULT_APPENDABLE_INDEX, TuningConfig.DEFAULT_APPENDABLE_INDEX,
maxRowsInMemory, maxRowsInMemory,

View File

@ -98,14 +98,11 @@ public class UnifiedIndexerAppenderatorsManagerTest extends InitializedNullHandl
EasyMock.replay(appenderatorConfig); EasyMock.replay(appenderatorConfig);
appenderator = manager.createBatchAppenderatorForTask( appenderator = manager.createBatchAppenderatorForTask(
"taskId", "taskId",
new DataSchema( DataSchema.builder()
"myDataSource", .withDataSource("myDataSource")
new TimestampSpec("__time", "millis", null), .withTimestamp(new TimestampSpec("__time", "millis", null))
null, .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()))
null, .build(),
new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()),
null
),
appenderatorConfig, appenderatorConfig,
new SegmentGenerationMetrics(), new SegmentGenerationMetrics(),
new NoopDataSegmentPusher(), new NoopDataSegmentPusher(),

View File

@ -34,7 +34,6 @@ import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.java.util.common.guava.Sequences;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory; import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.RowAdapters; import org.apache.druid.segment.RowAdapters;
import org.apache.druid.segment.RowBasedSegment; import org.apache.druid.segment.RowBasedSegment;
@ -76,14 +75,14 @@ public class SinkTest extends InitializedNullHandlingTest
@Test @Test
public void testSwap() throws Exception public void testSwap() throws Exception
{ {
final DataSchema schema = new DataSchema( final DataSchema schema =
"test", DataSchema.builder()
new TimestampSpec(null, null, null), .withDataSource("test")
DimensionsSpec.EMPTY, .withTimestamp(new TimestampSpec(null, null, null))
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, .withDimensions(DimensionsSpec.EMPTY)
new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), .withAggregators(new CountAggregatorFactory("rows"))
null .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null))
); .build();
final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final Interval interval = Intervals.of("2013-01-01/2013-01-02");
final String version = DateTimes.nowUtc().toString(); final String version = DateTimes.nowUtc().toString();
@ -256,18 +255,17 @@ public class SinkTest extends InitializedNullHandlingTest
@Test @Test
public void testGetSinkSignature() throws IndexSizeExceededException public void testGetSinkSignature() throws IndexSizeExceededException
{ {
final DataSchema schema = new DataSchema( final DataSchema schema =
"test", DataSchema.builder()
new TimestampSpec(null, null, null), .withDataSource("test")
new DimensionsSpec( .withTimestamp(new TimestampSpec(null, null, null))
Arrays.asList( .withDimensions(
new StringDimensionSchema("dim1"), new StringDimensionSchema("dim1"),
new LongDimensionSchema("dimLong") new LongDimensionSchema("dimLong")
)), )
new AggregatorFactory[]{new CountAggregatorFactory("rows")}, .withAggregators(new CountAggregatorFactory("rows"))
new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null), .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null))
null .build();
);
final Interval interval = Intervals.of("2013-01-01/2013-01-02"); final Interval interval = Intervals.of("2013-01-01/2013-01-02");
final String version = DateTimes.nowUtc().toString(); final String version = DateTimes.nowUtc().toString();

View File

@ -30,7 +30,6 @@ import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.TaskResource; import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -133,14 +132,11 @@ public class DruidJsonValidatorTest
null, null,
new TaskResource("rofl", 2), new TaskResource("rofl", 2),
new IndexTask.IndexIngestionSpec( new IndexTask.IndexIngestionSpec(
new DataSchema( DataSchema.builder()
"foo", .withDataSource("foo")
null, .withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null))
new AggregatorFactory[0], .withObjectMapper(jsonMapper)
new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null), .build(),
null,
jsonMapper
),
new IndexTask.IndexIOConfig( new IndexTask.IndexIOConfig(
new LocalInputSource(new File("lol"), "rofl"), new LocalInputSource(new File("lol"), "rofl"),
new JsonInputFormat(null, null, null, null, null), new JsonInputFormat(null, null, null, null, null),