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.java.util.common.Intervals;
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.segment.IndexSpec;
import org.apache.druid.segment.indexing.DataSchema;
@ -66,18 +65,19 @@ public class K8sTestUtils
null,
null,
new IndexTask.IndexIngestionSpec(
new DataSchema(
"foo",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
),
null
),
DataSchema.builder()
.withDataSource("foo")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
)
)
.build(),
new IndexTask.IndexIOConfig(
new LocalInputSource(new File("lol"), "rofl"),
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.granularity.ArbitraryGranularitySpec;
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.timeline.DataSegment;
import org.joda.time.Interval;
@ -211,14 +210,13 @@ public class MaterializedViewSupervisorSpec implements SupervisorSpec
);
// generate DataSchema
DataSchema dataSchema = new DataSchema(
dataSourceName,
parser,
aggregators,
granularitySpec,
TransformSpec.NONE,
objectMapper
);
DataSchema dataSchema = DataSchema.builder()
.withDataSource(dataSourceName)
.withParserMap(parser)
.withAggregators(aggregators)
.withGranularity(granularitySpec)
.withObjectMapper(objectMapper)
.build();
// generate 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.SegmentSchemaManager;
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.timeline.DataSegment;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
@ -237,14 +236,10 @@ public class MaterializedViewSupervisorTest
Map<Interval, HadoopIndexTask> runningTasks = runningTasksPair.lhs;
Map<Interval, String> runningVersion = runningTasksPair.rhs;
DataSchema dataSchema = new DataSchema(
"test_datasource",
null,
null,
null,
TransformSpec.NONE,
objectMapper
);
DataSchema dataSchema = DataSchema.builder()
.withDataSource("test_datasource")
.withObjectMapper(objectMapper)
.build();
HadoopIOConfig hadoopIOConfig = new HadoopIOConfig(new HashMap<>(), null, null);
HadoopIngestionSpec spec = new HadoopIngestionSpec(dataSchema, hadoopIOConfig, null);
HadoopIndexTask task1 = new HadoopIndexTask(

View File

@ -24,7 +24,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.InputFormat;
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.StringDimensionSchema;
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.metrics.DruidMonitorSchedulerConfig;
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.segment.TestHelper;
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("dim2"));
return new DataSchema(
dataSource,
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(dimensions),
new AggregatorFactory[] {new CountAggregatorFactory("rows")},
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()),
null);
return DataSchema.builder()
.withDataSource(dataSource)
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
.withDimensions(dimensions)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
)
)
.build();
}
@BeforeClass

View File

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

View File

@ -30,7 +30,6 @@ import com.google.common.util.concurrent.ListenableFuture;
import org.apache.curator.test.TestingCluster;
import org.apache.druid.data.input.InputFormat;
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.StringDimensionSchema;
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.metrics.DruidMonitorSchedulerConfig;
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.segment.TestHelper;
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("dim2"));
return new DataSchema(
dataSource,
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(dimensions),
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
),
null
);
return DataSchema.builder()
.withDataSource(dataSource)
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
.withDimensions(dimensions)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
)
)
.build();
}
private KafkaIndexTask createKafkaIndexTask(

View File

@ -50,7 +50,8 @@ import java.util.Collections;
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(
null,
null,

View File

@ -2412,16 +2412,7 @@ public class KinesisIndexTaskTest extends SeekableStreamIndexTaskTestBase
private static DataSchema cloneDataSchema(final DataSchema dataSchema)
{
return new DataSchema(
dataSchema.getDataSource(),
dataSchema.getTimestampSpec(),
dataSchema.getDimensionsSpec(),
dataSchema.getAggregators(),
dataSchema.getGranularitySpec(),
dataSchema.getTransformSpec(),
dataSchema.getParserMap(),
OBJECT_MAPPER
);
return DataSchema.builder(dataSchema).withObjectMapper(OBJECT_MAPPER).build();
}
@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.granularity.Granularities;
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.DoubleSumAggregatorFactory;
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 SHARD_ID = "1";
private static final DataSchema DATA_SCHEMA = new DataSchema(
"test_ds",
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
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 final DataSchema DATA_SCHEMA =
DataSchema.builder()
.withDataSource("test_ds")
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
.withDimensions(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
.withAggregators(
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
)
.withGranularity(
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null)
)
.build();
static {
NullHandling.initializeForTests();
@ -192,17 +191,18 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
);
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
DataSchema dataSchema = new DataSchema(
"test_ds",
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class),
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
objectMapper
);
DataSchema dataSchema = DataSchema.builder()
.withDataSource("test_ds")
.withParserMap(
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class)
)
.withAggregators(
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
)
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.withObjectMapper(objectMapper)
.build();
KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec(
null,

View File

@ -28,7 +28,6 @@ import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.data.input.InputFormat;
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.StringDimensionSchema;
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.metrics.DruidMonitorSchedulerConfig;
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.segment.TestHelper;
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("dim2"));
return new DataSchema(
dataSource,
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(dimensions),
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
),
null
);
return DataSchema.builder()
.withDataSource(dataSource)
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
.withDimensions(dimensions)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
)
)
.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.granularity.ArbitraryGranularitySpec;
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.rel.DruidQuery;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.HashMap;
import java.util.LinkedHashSet;
import java.util.List;
@ -96,14 +94,13 @@ public final class SegmentGenerationUtils
destination.getDimensionSchemas()
);
return new DataSchema(
destination.getDataSource(),
new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null),
dimensionsAndAggregators.lhs,
dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]),
makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper),
new TransformSpec(null, Collections.emptyList())
);
return DataSchema.builder()
.withDataSource(destination.getDataSource())
.withTimestamp(new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null))
.withDimensions(dimensionsAndAggregators.lhs)
.withAggregators(dimensionsAndAggregators.rhs.toArray(new AggregatorFactory[0]))
.withGranularity(makeGranularitySpecForIngestion(querySpec.getQuery(), querySpec.getColumnMappings(), isRollupQuery, jsonMapper))
.build();
}
private static GranularitySpec makeGranularitySpecForIngestion(

View File

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

View File

@ -433,30 +433,33 @@ public class BatchDeltaIngestionTest
{
HadoopDruidIndexerConfig config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec(
new DataSchema(
"website",
MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
null,
ImmutableList.of("timestamp", "host", "host2", "visited_num"),
false,
0
),
null
),
Map.class
),
aggregators != null ? aggregators : new AggregatorFactory[]{
new LongSumAggregatorFactory("visited_sum", "visited_num"),
new HyperUniquesAggregatorFactory("unique_hosts", "host2")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(INTERVAL_FULL)),
null,
MAPPER
),
DataSchema.builder()
.withDataSource("website")
.withParserMap(MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
null,
ImmutableList.of("timestamp", "host", "host2", "visited_num"),
false,
0
),
null
),
Map.class
))
.withAggregators(aggregators != null ? aggregators : new AggregatorFactory[]{
new LongSumAggregatorFactory("visited_sum", "visited_num"),
new HyperUniquesAggregatorFactory("unique_hosts", "host2")
})
.withGranularity(new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(INTERVAL_FULL)
))
.withObjectMapper(MAPPER)
.build(),
new HadoopIOConfig(
inputSpec,
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.Granularity;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -158,46 +157,45 @@ public class DetermineHashedPartitionsJobTest
}
HadoopIngestionSpec ingestionSpec = new HadoopIngestionSpec(
new DataSchema(
"test_schema",
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new DelimitedParseSpec(
new TimestampSpec("ts", null, null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of(
"market",
"quality",
"placement",
"placementish"
))
),
"\t",
null,
Arrays.asList(
"ts",
"market",
"quality",
"placement",
"placementish",
"index"
),
false,
0
),
null
),
Map.class
),
new AggregatorFactory[]{new DoubleSumAggregatorFactory("index", "index")},
new UniformGranularitySpec(
segmentGranularity,
Granularities.NONE,
intervals
),
null,
HadoopDruidIndexerConfig.JSON_MAPPER
),
DataSchema.builder()
.withDataSource("test_schema")
.withParserMap(HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new DelimitedParseSpec(
new TimestampSpec("ts", null, null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of(
"market",
"quality",
"placement",
"placementish"
))
),
"\t",
null,
Arrays.asList(
"ts",
"market",
"quality",
"placement",
"placementish",
"index"
),
false,
0
),
null
),
Map.class
))
.withAggregators(new DoubleSumAggregatorFactory("index", "index"))
.withGranularity(new UniformGranularitySpec(
segmentGranularity,
Granularities.NONE,
intervals
))
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig(
ImmutableMap.of(
"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.Intervals;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -280,33 +279,36 @@ public class DeterminePartitionsJobTest
config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec(
new DataSchema(
"website",
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
),
null,
ImmutableList.of("timestamp", "host", "country", "visited_num"),
false,
0
),
null
),
Map.class
),
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of(interval))
),
null,
HadoopDruidIndexerConfig.JSON_MAPPER
),
DataSchema.builder()
.withDataSource("website")
.withParserMap(
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
),
null,
ImmutableList.of("timestamp", "host", "country", "visited_num"),
false,
0
),
null
),
Map.class
)
)
.withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of(interval))
)
)
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig(
ImmutableMap.of(
"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.Intervals;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -328,35 +327,36 @@ public class DetermineRangePartitionsJobTest
config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec(
new DataSchema(
"website",
null,
null,
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of(interval))
),
null,
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
),
null,
ImmutableList.of("timestamp", "host", "country", "visited_num"),
false,
0
),
null
),
Map.class
),
HadoopDruidIndexerConfig.JSON_MAPPER
),
DataSchema.builder()
.withDataSource("website")
.withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of(interval))
)
)
.withParserMap(
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("host", "country"))
),
null,
ImmutableList.of("timestamp", "host", "country", "visited_num"),
false,
0
),
null
),
Map.class
)
)
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig(
ImmutableMap.of(
"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.Intervals;
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.granularity.UniformGranularitySpec;
import org.apache.druid.timeline.partition.HashBasedNumberedShardSpec;
@ -217,18 +216,18 @@ public class HadoopDruidIndexerConfigTest
private static class HadoopIngestionSpecBuilder
{
private static final DataSchema DATA_SCHEMA = new DataSchema(
"foo",
null,
new AggregatorFactory[0],
new UniformGranularitySpec(
Granularities.MINUTE,
Granularities.MINUTE,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
null,
HadoopDruidIndexerConfigTest.JSON_MAPPER
);
private static final DataSchema DATA_SCHEMA =
DataSchema.builder()
.withDataSource("foo")
.withGranularity(
new UniformGranularitySpec(
Granularities.MINUTE,
Granularities.MINUTE,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
)
)
.withObjectMapper(HadoopDruidIndexerConfigTest.JSON_MAPPER)
.build();
private static final HadoopIOConfig HADOOP_IO_CONFIG = new HadoopIOConfig(
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.parsers.JSONPathSpec;
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.filter.SelectorDimFilter;
import org.apache.druid.segment.TestHelper;
@ -58,27 +57,29 @@ import java.util.stream.Collectors;
public class HadoopDruidIndexerMapperTest
{
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
private static final DataSchema DATA_SCHEMA = new DataSchema(
"test_ds",
JSON_MAPPER.convertValue(
new HadoopyStringInputRowParser(
new JSONParseSpec(
new TimestampSpec("t", "auto", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
null
)
),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
),
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
JSON_MAPPER
);
private static final DataSchema DATA_SCHEMA =
DataSchema.builder()
.withDataSource("test_ds")
.withParserMap(
JSON_MAPPER.convertValue(
new HadoopyStringInputRowParser(
new JSONParseSpec(
new TimestampSpec("t", "auto", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
null
)
),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.withObjectMapper(JSON_MAPPER)
.build();
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
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.java.util.common.Intervals;
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.granularity.UniformGranularitySpec;
import org.apache.druid.timeline.DataSegment;
@ -274,18 +273,17 @@ public class HadoopIngestionSpecUpdateDatasourcePathSpecSegmentsTest
throws Exception
{
HadoopIngestionSpec spec = new HadoopIngestionSpec(
new DataSchema(
"foo",
null,
new AggregatorFactory[0],
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
null,
jsonMapper
),
DataSchema.builder()
.withDataSource("foo")
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
)
)
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(
jsonMapper.convertValue(datasourcePathSpec, Map.class),
null,

View File

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

View File

@ -506,17 +506,19 @@ public class IndexGeneratorJobTest
config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec(
new DataSchema(
datasourceName,
mapper.convertValue(
inputRowParser,
Map.class
),
aggs,
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)),
null,
mapper
),
DataSchema.builder()
.withDataSource(datasourceName)
.withParserMap(mapper.convertValue(inputRowParser, Map.class))
.withAggregators(aggs)
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(interval)
)
)
.withObjectMapper(mapper)
.build(),
new HadoopIOConfig(
ImmutableMap.copyOf(inputSpec),
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.jackson.JacksonUtils;
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.LongSumAggregatorFactory;
import org.apache.druid.segment.TestHelper;
@ -68,27 +67,30 @@ import java.util.Map;
public class JobHelperTest
{
private static final ObjectMapper JSON_MAPPER = TestHelper.makeJsonMapper();
private static final DataSchema DATA_SCHEMA = new DataSchema(
"test_ds",
JSON_MAPPER.convertValue(
new HadoopyStringInputRowParser(
new JSONParseSpec(
new TimestampSpec("t", "auto", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
null
)
),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
),
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
JSON_MAPPER
);
private static final DataSchema DATA_SCHEMA =
DataSchema.builder()
.withDataSource("test_ds")
.withParserMap(
JSON_MAPPER.convertValue(
new HadoopyStringInputRowParser(
new JSONParseSpec(
new TimestampSpec("t", "auto", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim1t", "dim2"))
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
null
)
),
JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
)
)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.withObjectMapper(JSON_MAPPER)
.build();
private static final HadoopIOConfig IO_CONFIG = new HadoopIOConfig(
JSON_MAPPER.convertValue(
@ -123,27 +125,34 @@ public class JobHelperTest
dataFile = temporaryFolder.newFile();
config = new HadoopDruidIndexerConfig(
new HadoopIngestionSpec(
new DataSchema(
"website",
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
null,
ImmutableList.of("timestamp", "host", "visited_num"),
false,
0
),
null
),
Map.class
),
new AggregatorFactory[]{new LongSumAggregatorFactory("visited_num", "visited_num")},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, ImmutableList.of(this.interval)),
null,
HadoopDruidIndexerConfig.JSON_MAPPER
),
DataSchema.builder()
.withDataSource("website")
.withParserMap(
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("host"))),
null,
ImmutableList.of("timestamp", "host", "visited_num"),
false,
0
),
null
),
Map.class
)
)
.withAggregators(new LongSumAggregatorFactory("visited_num", "visited_num"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(this.interval)
)
)
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig(
ImmutableMap.of(
"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.Intervals;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -308,33 +307,34 @@ public class DatasourcePathSpecTest
{
return new HadoopDruidIndexerConfig(
new HadoopIngestionSpec(
new DataSchema(
ingestionSpec1.getDataSource(),
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
DimensionsSpec.EMPTY,
null,
ImmutableList.of("timestamp", "host", "visited"),
false,
0
),
null
),
Map.class
),
new AggregatorFactory[]{
new LongSumAggregatorFactory("visited_sum", "visited")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of("2000/3000"))
),
null,
HadoopDruidIndexerConfig.JSON_MAPPER
),
DataSchema.builder()
.withDataSource(ingestionSpec1.getDataSource())
.withParserMap(
HadoopDruidIndexerConfig.JSON_MAPPER.convertValue(
new StringInputRowParser(
new CSVParseSpec(
new TimestampSpec("timestamp", "yyyyMMddHH", null),
DimensionsSpec.EMPTY,
null,
ImmutableList.of("timestamp", "host", "visited"),
false,
0
),
null
),
Map.class
)
)
.withAggregators(new LongSumAggregatorFactory("visited_sum", "visited"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.NONE,
ImmutableList.of(Intervals.of("2000/3000"))
)
)
.withObjectMapper(HadoopDruidIndexerConfig.JSON_MAPPER)
.build(),
new HadoopIOConfig(
ImmutableMap.of(
"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.Granularity;
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.granularity.UniformGranularitySpec;
import org.apache.hadoop.mapreduce.Job;
@ -152,18 +151,17 @@ public class GranularityPathSpecTest
{
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
HadoopIngestionSpec spec = new HadoopIngestionSpec(
new DataSchema(
"foo",
null,
new AggregatorFactory[0],
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z"))
),
null,
jsonMapper
),
DataSchema.builder()
.withDataSource("foo")
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
ImmutableList.of(Intervals.of("2015-11-06T00:00Z/2015-11-07T00:00Z"))
)
)
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(null, null, null),
DEFAULT_TUNING_CONFIG
);
@ -204,18 +202,17 @@ public class GranularityPathSpecTest
{
UserGroupInformation.setLoginUser(UserGroupInformation.createUserForTesting("test", new String[]{"testGroup"}));
HadoopIngestionSpec spec = new HadoopIngestionSpec(
new DataSchema(
"foo",
null,
new AggregatorFactory[0],
new UniformGranularitySpec(
Granularities.DAY,
Granularities.ALL,
ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z"))
),
null,
jsonMapper
),
DataSchema.builder()
.withDataSource("foo")
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.ALL,
ImmutableList.of(Intervals.of("2015-01-01T11Z/2015-01-02T05Z"))
)
)
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(null, null, null),
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.HadoopIngestionSpec;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.hadoop.mapreduce.Job;
import org.apache.hadoop.mapreduce.lib.input.MultipleInputs;
@ -54,7 +53,7 @@ public class StaticPathSpecTest
Job job = new Job();
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);
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 DataSchema DEFAULT_DATA_SCHEMA = new DataSchema(
SAMPLER_DATA_SOURCE,
new TimestampSpec(null, null, null),
new DimensionsSpec(null),
null,
null,
null
);
private static final DataSchema DEFAULT_DATA_SCHEMA =
DataSchema.builder()
.withDataSource(SAMPLER_DATA_SOURCE)
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.builder().build())
.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
// 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.TuningConfigBuilder;
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.SegmentSchemaMapping;
import org.apache.druid.segment.indexing.DataSchema;
@ -62,7 +61,7 @@ public class TestIndexTask extends IndexTask
id,
taskResource,
new IndexIngestionSpec(
new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper),
DataSchema.builder().withDataSource(dataSource).withObjectMapper(mapper).build(),
new IndexTask.IndexIOConfig(
new LocalInputSource(new File("lol"), "rofl"),
new JsonInputFormat(null, null, null, null, null),

View File

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

View File

@ -27,7 +27,6 @@ import org.apache.druid.indexer.HadoopIngestionSpec;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.Intervals;
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.granularity.UniformGranularitySpec;
import org.apache.druid.server.security.Action;
@ -50,15 +49,19 @@ public class HadoopIndexTaskTest
final HadoopIndexTask task = new HadoopIndexTask(
null,
new HadoopIngestionSpec(
new DataSchema(
"foo", null, new AggregatorFactory[0], new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
null,
jsonMapper
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null
DataSchema.builder()
.withDataSource("foo")
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
)
)
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), 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.IndexIngestionSpec;
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.granularity.ArbitraryGranularitySpec;
import org.junit.Rule;
@ -45,14 +44,11 @@ public class IndexIngestionSpecTest
"Cannot use parser and inputSource together. Try using inputFormat instead of parser."
);
final IndexIngestionSpec spec = new IndexIngestionSpec(
new DataSchema(
"dataSource",
ImmutableMap.of("fake", "parser map"),
new AggregatorFactory[0],
new ArbitraryGranularitySpec(Granularities.NONE, null),
null,
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withParserMap(ImmutableMap.of("fake", "parser map"))
.withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null))
.build(),
new IndexIOConfig(
new NoopInputSource(),
new NoopInputFormat(),
@ -69,14 +65,11 @@ public class IndexIngestionSpecTest
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("Cannot use parser and inputSource together.");
final IndexIngestionSpec spec = new IndexIngestionSpec(
new DataSchema(
"dataSource",
ImmutableMap.of("fake", "parser map"),
new AggregatorFactory[0],
new ArbitraryGranularitySpec(Granularities.NONE, null),
null,
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withParserMap(ImmutableMap.of("fake", "parser map"))
.withGranularity(new ArbitraryGranularitySpec(Granularities.NONE, null))
.build(),
new IndexIOConfig(
new NoopInputSource(),
null,

View File

@ -166,6 +166,25 @@ public class IndexTaskTest extends IngestionTestBase
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}")
public static Iterable<Object[]> constructorFeeder()
{
@ -225,24 +244,7 @@ public class IndexTaskTest extends IngestionTestBase
{
IndexTask indexTask = createIndexTask(
new IndexIngestionSpec(
new DataSchema(
"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
),
DATA_SCHEMA,
new IndexIOConfig(
new LocalInputSource(tmpDir, "druid*"),
DEFAULT_INPUT_FORMAT,
@ -275,24 +277,7 @@ public class IndexTaskTest extends IngestionTestBase
IndexTask indexTask = createIndexTask(
new IndexIngestionSpec(
new DataSchema(
"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
),
DATA_SCHEMA,
new IndexIOConfig(
new LocalInputSource(tmpDir, "druid*"),
DEFAULT_INPUT_FORMAT,
@ -337,24 +322,7 @@ public class IndexTaskTest extends IngestionTestBase
IndexTask indexTask = createIndexTask(
new IndexIngestionSpec(
new DataSchema(
"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
),
DATA_SCHEMA,
new IndexIOConfig(
new LocalInputSource(tmpDir, "druid*"),
DEFAULT_INPUT_FORMAT,
@ -2698,20 +2666,20 @@ public class IndexTaskTest extends IngestionTestBase
if (inputFormat != null) {
Preconditions.checkArgument(parseSpec == null, "Can't use parseSpec");
return new IndexIngestionSpec(
new DataSchema(
DATASOURCE,
Preconditions.checkNotNull(timestampSpec, "timestampSpec"),
Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/2015"))
),
transformSpec
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(Preconditions.checkNotNull(timestampSpec, "timestampSpec"))
.withDimensions(Preconditions.checkNotNull(dimensionsSpec, "dimensionsSpec"))
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/2015"))
)
)
.withTransform(transformSpec)
.build(),
new IndexIOConfig(
new LocalInputSource(baseDir, "druid*"),
inputFormat,
@ -2723,22 +2691,21 @@ public class IndexTaskTest extends IngestionTestBase
} else {
parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC;
return new IndexIngestionSpec(
new DataSchema(
DATASOURCE,
parseSpec.getTimestampSpec(),
parseSpec.getDimensionsSpec(),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/2015"))
),
transformSpec,
null,
objectMapper
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(parseSpec.getTimestampSpec())
.withDimensions(parseSpec.getDimensionsSpec())
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
granularitySpec != null ? granularitySpec : new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2014/2015"))
)
)
.withTransform(transformSpec)
.withObjectMapper(objectMapper)
.build(),
new IndexIOConfig(
new LocalInputSource(baseDir, "druid*"),
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.java.util.common.Intervals;
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.segment.IndexSpec;
import org.apache.druid.segment.indexing.DataSchema;
@ -220,18 +219,19 @@ public class TaskSerdeTest
null,
null,
new IndexIngestionSpec(
new DataSchema(
"foo",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
),
null
),
DataSchema.builder()
.withDataSource("foo")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
)
)
.build(),
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
TuningConfigBuilder.forIndexTask()
.withMaxRowsInMemory(10)
@ -288,18 +288,19 @@ public class TaskSerdeTest
null,
new TaskResource("rofl", 2),
new IndexIngestionSpec(
new DataSchema(
"foo",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
),
null
),
DataSchema.builder()
.withDataSource("foo")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
)
)
.build(),
new IndexIOConfig(new LocalInputSource(new File("lol"), "rofl"), new NoopInputFormat(), true, false),
TuningConfigBuilder.forIndexTask()
.withMaxRowsInMemory(10)
@ -412,15 +413,19 @@ public class TaskSerdeTest
final HadoopIndexTask task = new HadoopIndexTask(
null,
new HadoopIngestionSpec(
new DataSchema(
"foo", null, new AggregatorFactory[0], new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
null,
jsonMapper
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null
DataSchema.builder()
.withDataSource("foo")
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
)
)
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
null
),
null,
null,
@ -454,19 +459,18 @@ public class TaskSerdeTest
final HadoopIndexTask task = new HadoopIndexTask(
null,
new HadoopIngestionSpec(
new DataSchema(
"foo",
null,
null,
new AggregatorFactory[0],
new UniformGranularitySpec(
Granularities.DAY,
null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
null,
null,
jsonMapper
), new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null), null
DataSchema.builder()
.withDataSource("foo")
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null, ImmutableList.of(Intervals.of("2010-01-01/P1D"))
)
)
.withObjectMapper(jsonMapper)
.build(),
new HadoopIOConfig(ImmutableMap.of("paths", "bar"), null, null),
null
),
null,
null,

View File

@ -221,14 +221,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
dropExisting
);
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
timestampSpec,
dimensionsSpec,
DEFAULT_METRICS_SPEC,
granularitySpec,
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(timestampSpec)
.withDimensions(dimensionsSpec)
.withAggregators(DEFAULT_METRICS_SPEC)
.withGranularity(granularitySpec)
.build(),
ioConfig,
tuningConfig
);
@ -241,14 +240,13 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
dropExisting
);
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
parseSpec.getTimestampSpec(),
parseSpec.getDimensionsSpec(),
DEFAULT_METRICS_SPEC,
granularitySpec,
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(parseSpec.getTimestampSpec())
.withDimensions(parseSpec.getDimensionsSpec())
.withAggregators(DEFAULT_METRICS_SPEC)
.withGranularity(granularitySpec)
.build(),
ioConfig,
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.granularity.Granularities;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
@ -250,14 +249,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
null
);
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
timestampSpec,
dimensionsSpec,
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")},
granularitySpec,
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(timestampSpec)
.withDimensions(dimensionsSpec)
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(granularitySpec)
.build(),
ioConfig,
tuningConfig
);
@ -271,16 +269,13 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
);
//noinspection unchecked
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
parseSpec.getTimestampSpec(),
parseSpec.getDimensionsSpec(),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
granularitySpec,
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(parseSpec.getTimestampSpec())
.withDimensions(parseSpec.getDimensionsSpec())
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(granularitySpec)
.build(),
ioConfig,
tuningConfig
);

View File

@ -127,18 +127,19 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
TIMESTAMP_SPEC,
DIMENSIONS_SPEC.withDimensions(dimensionSchemas),
DEFAULT_METRICS_SPEC,
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
INTERVAL_TO_INDEX
),
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
.withAggregators(DEFAULT_METRICS_SPEC)
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
INTERVAL_TO_INDEX
)
)
.build(),
new ParallelIndexIOConfig(
getInputSource(),
JSON_FORMAT,
@ -177,18 +178,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
TIMESTAMP_SPEC,
new DimensionsSpec.Builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build(),
DEFAULT_METRICS_SPEC,
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
INTERVAL_TO_INDEX
),
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(TIMESTAMP_SPEC)
.withDimensions(
DimensionsSpec.builder().setDimensions(dimensionSchemas).setIncludeAllDimensions(true).build()
)
.withAggregators(DEFAULT_METRICS_SPEC)
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
INTERVAL_TO_INDEX
)
)
.build(),
new ParallelIndexIOConfig(
getInputSource(),
new JsonInputFormat(
@ -237,18 +241,21 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
TIMESTAMP_SPEC,
new DimensionsSpec.Builder().setIncludeAllDimensions(true).build(),
DEFAULT_METRICS_SPEC,
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
null
),
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(TIMESTAMP_SPEC)
.withDimensions(
DimensionsSpec.builder().setIncludeAllDimensions(true).build()
)
.withAggregators(DEFAULT_METRICS_SPEC)
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
null
)
)
.build(),
new ParallelIndexIOConfig(
getInputSource(),
new JsonInputFormat(
@ -303,20 +310,23 @@ public class MultiPhaseParallelIndexingWithNullColumnTest extends AbstractMultiP
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
TIMESTAMP_SPEC,
DIMENSIONS_SPEC.withDimensions(
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim"))
),
DEFAULT_METRICS_SPEC,
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
INTERVAL_TO_INDEX
),
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(TIMESTAMP_SPEC)
.withDimensions(
DIMENSIONS_SPEC.withDimensions(
DimensionsSpec.getDefaultSchemas(Arrays.asList("ts", "unknownDim"))
)
)
.withAggregators(DEFAULT_METRICS_SPEC)
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
INTERVAL_TO_INDEX
)
)
.build(),
new ParallelIndexIOConfig(
getInputSource(),
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.Pair;
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.segment.indexing.DataSchema;
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);
// set up ingestion spec
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DEFAULT_DIMENSIONS_SPEC,
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
)
)
.build(),
ioConfig,
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.Intervals;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -400,20 +399,19 @@ public class ParallelIndexSupervisorTaskResourceTest extends AbstractParallelInd
)
{
final ParallelIndexIngestionSpec ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DEFAULT_DIMENSIONS_SPEC,
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
)
)
.build(),
ioConfig,
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.java.util.common.Intervals;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -220,16 +219,19 @@ public class ParallelIndexSupervisorTaskSerdeTest
ParallelIndexIngestionSpec build()
{
DataSchema dataSchema = new DataSchema(
"dataSource",
TIMESTAMP_SPEC,
DIMENSIONS_SPEC,
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.MINUTE, inputIntervals),
null
);
DataSchema dataSchema = DataSchema.builder()
.withDataSource("datasource")
.withTimestamp(TIMESTAMP_SPEC)
.withDimensions(DIMENSIONS_SPEC)
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
inputIntervals
)
)
.build();
ParallelIndexTuningConfig tuningConfig = TuningConfigBuilder
.forParallelIndexTask()

View File

@ -263,14 +263,11 @@ public class ParallelIndexSupervisorTaskTest
.withLogParseExceptions(false)
.build();
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"datasource",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
null,
null,
null
),
DataSchema.builder()
.withDataSource("datasource")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.build(),
ioConfig,
tuningConfig
);
@ -325,25 +322,24 @@ public class ParallelIndexSupervisorTaskTest
expectedException.expect(IAE.class);
expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
new ParallelIndexIngestionSpec(
new DataSchema(
"datasource",
mapper.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
null,
null,
null
)
),
Map.class
),
null,
null,
null,
mapper
),
DataSchema.builder()
.withDataSource("datasource")
.withParserMap(
mapper.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
null,
null,
null
)
),
Map.class
)
)
.withObjectMapper(mapper)
.build(),
ioConfig,
tuningConfig
);
@ -559,14 +555,11 @@ public class ParallelIndexSupervisorTaskTest
.build();
final ParallelIndexIngestionSpec indexIngestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"datasource",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
null,
null,
null
),
DataSchema.builder()
.withDataSource("datasource")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.build(),
ioConfig,
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.java.util.common.Intervals;
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.granularity.ArbitraryGranularitySpec;
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.HashPartitionFunction;
import org.joda.time.Interval;
@ -97,16 +95,13 @@ class ParallelIndexTestingFactory
DimensionsSpec.getDefaultSchemas(ImmutableList.of(SCHEMA_DIMENSION))
);
return new DataSchema(
DATASOURCE,
timestampSpec,
dimensionsSpec,
new AggregatorFactory[]{},
granularitySpec,
TransformSpec.NONE,
null,
NESTED_OBJECT_MAPPER
);
return DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(timestampSpec)
.withDimensions(dimensionsSpec)
.withGranularity(granularitySpec)
.withObjectMapper(NESTED_OBJECT_MAPPER)
.build();
}
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.Intervals;
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.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
@ -329,14 +328,13 @@ public class RangePartitionTaskKillTest extends AbstractMultiPhaseParallelIndexi
null
);
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
DATASOURCE,
timestampSpec,
dimensionsSpec,
new AggregatorFactory[]{new LongSumAggregatorFactory("val", "val")},
granularitySpec,
null
),
DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(timestampSpec)
.withDimensions(dimensionsSpec)
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(granularitySpec)
.build(),
ioConfig,
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.Granularity;
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.LongSumAggregatorFactory;
import org.apache.druid.segment.DataSegmentsWithSchemas;
@ -391,20 +390,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2017-12/P1M"))
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2017-12/P1M"))
)
)
.build(),
new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
DEFAULT_INPUT_FORMAT,
@ -444,20 +442,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2017-12/P1M"))
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(DEFAULT_DIMENSIONS_SPEC.withDimensions(dimensionSchemas))
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(Intervals.of("2017-12/P1M"))
)
)
.build(),
new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, VALID_INPUT_SOURCE_FILTER, true),
DEFAULT_INPUT_FORMAT,
@ -785,21 +782,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DimensionsSpec.builder()
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
.setIncludeAllDimensions(true)
.build(),
new AggregatorFactory[]{new CountAggregatorFactory("cnt")},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(interval)
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(
DimensionsSpec.builder()
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
.setIncludeAllDimensions(true)
.build()
)
.withAggregators(new CountAggregatorFactory("cnt"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(interval)
)
)
.build(),
new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, "*.json", true),
new JsonInputFormat(
@ -868,21 +868,24 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
null,
null,
new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DimensionsSpec.builder()
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
.useSchemaDiscovery(true)
.build(),
new AggregatorFactory[]{new CountAggregatorFactory("cnt")},
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(interval)
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(
DimensionsSpec.builder()
.setDefaultSchemaDimensions(ImmutableList.of("ts", "explicitDim"))
.useSchemaDiscovery(true)
.build()
)
.withAggregators(new CountAggregatorFactory("cnt"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
Granularities.MINUTE,
Collections.singletonList(interval)
)
)
.build(),
new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, "*.json", true),
new JsonInputFormat(
@ -948,20 +951,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
final ParallelIndexIngestionSpec ingestionSpec;
if (useInputFormatApi) {
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DEFAULT_DIMENSIONS_SPEC,
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
segmentGranularity,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
.withAggregators(new LongSumAggregatorFactory("val", "val"))
.withGranularity(
new UniformGranularitySpec(
segmentGranularity,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
)
)
.build(),
new ParallelIndexIOConfig(
new SettableSplittableLocalInputSource(inputDir, inputSourceFilter, splittableInputSource),
DEFAULT_INPUT_FORMAT,
@ -972,18 +974,19 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
);
} else {
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
DEFAULT_TIMESTAMP_SPEC,
DEFAULT_DIMENSIONS_SPEC,
DEFAULT_METRICS_SPEC,
new UniformGranularitySpec(
segmentGranularity,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(DEFAULT_TIMESTAMP_SPEC)
.withDimensions(DEFAULT_DIMENSIONS_SPEC)
.withAggregators(DEFAULT_METRICS_SPEC)
.withGranularity(
new UniformGranularitySpec(
segmentGranularity,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
)
)
.build(),
new ParallelIndexIOConfig(
new LocalInputSource(inputDir, inputSourceFilter),
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.TimestampSpec;
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.server.security.Action;
import org.apache.druid.server.security.Resource;
@ -48,14 +47,11 @@ public class SinglePhaseSubTaskSpecTest
"groupId",
"supervisorTaskId",
new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
new TimestampSpec(null, null, null),
new DimensionsSpec(null),
new AggregatorFactory[0],
null,
null
),
DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.builder().build())
.build(),
new ParallelIndexIOConfig(
new LocalInputSource(new File("baseDir"), "filter"),
new JsonInputFormat(null, null, null, null, null),

View File

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

View File

@ -122,17 +122,19 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest
new DoubleDimensionSchema("d5")
)
);
DataSchema schema = new DataSchema(
"dataSourceName",
new TimestampSpec(null, null, null),
dimensionsSpec,
new AggregatorFactory[]{
new CountAggregatorFactory("count"),
new LongSumAggregatorFactory("met", "met")
},
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
null
);
DataSchema schema =
DataSchema.builder()
.withDataSource("dataSourceName")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(dimensionsSpec)
.withAggregators(
new CountAggregatorFactory("count"),
new LongSumAggregatorFactory("met", "met")
)
.withGranularity(
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)
)
.build();
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
Assert.assertEquals(timestampSpec, inputRowSchema.getTimestampSpec());
@ -154,14 +156,12 @@ public class InputRowSchemasTest extends InitializedNullHandlingTest
new DoubleDimensionSchema("d5")
)
);
DataSchema schema = new DataSchema(
"dataSourceName",
new TimestampSpec(null, null, null),
dimensionsSpec,
new AggregatorFactory[]{},
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
null
);
DataSchema schema = DataSchema.builder()
.withDataSource("dataSourceName")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(dimensionsSpec)
.withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null))
.build();
InputRowSchema inputRowSchema = InputRowSchemas.fromDataSchema(schema);
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.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.SegmentDescriptor;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9Factory;
@ -672,18 +671,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null,
null,
new IndexIngestionSpec(
new DataSchema(
"foo",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
),
null
),
DataSchema.builder()
.withDataSource("foo")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
)
)
.build(),
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000)
@ -735,18 +735,18 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null,
null,
new IndexIngestionSpec(
new DataSchema(
"foo",
null,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
),
null,
mapper
),
DataSchema.builder()
.withDataSource("foo")
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P1D"))
)
)
.withObjectMapper(mapper)
.build(),
new IndexIOConfig(new MockExceptionInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000)
@ -1165,18 +1165,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null,
null,
new IndexIngestionSpec(
new DataSchema(
"foo",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
),
null
),
DataSchema.builder()
.withDataSource("foo")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
)
)
.build(),
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000)
@ -1253,18 +1254,19 @@ public class TaskLifecycleTest extends InitializedNullHandlingTest
null,
null,
new IndexIngestionSpec(
new DataSchema(
"foo",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[]{new DoubleSumAggregatorFactory("met", "met")},
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
),
null
),
DataSchema.builder()
.withDataSource("foo")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.withAggregators(new DoubleSumAggregatorFactory("met", "met"))
.withGranularity(
new UniformGranularitySpec(
Granularities.DAY,
null,
ImmutableList.of(Intervals.of("2010-01-01/P2D"))
)
)
.build(),
new IndexIOConfig(new MockInputSource(), new NoopInputFormat(), false, false),
TuningConfigBuilder.forIndexTask()
.withMaxRowsPerSegment(10000)

View File

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

View File

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

View File

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

View File

@ -1497,24 +1497,24 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
) throws IOException
{
if (useInputFormatApi) {
return new DataSchema(
"sampler",
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec
);
return DataSchema.builder()
.withDataSource("sampler")
.withTimestamp(timestampSpec)
.withDimensions(dimensionsSpec)
.withAggregators(aggregators)
.withGranularity(granularitySpec)
.withTransform(transformSpec)
.build();
} else {
final Map<String, Object> parserMap = getParserMap(createInputRowParser(timestampSpec, dimensionsSpec));
return new DataSchema(
"sampler",
parserMap,
aggregators,
granularitySpec,
transformSpec,
OBJECT_MAPPER
);
return DataSchema.builder()
.withDataSource("sampler")
.withParserMap(parserMap)
.withAggregators(aggregators)
.withGranularity(granularitySpec)
.withTransform(transformSpec)
.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.java.util.common.DateTimes;
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.granularity.ArbitraryGranularitySpec;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.security.Access;
import org.apache.druid.server.security.Action;
import org.apache.druid.server.security.AuthConfig;
@ -107,16 +105,13 @@ public class SeekableStreamIndexTaskRunnerAuthTest
}
};
DataSchema dataSchema = new DataSchema(
"datasource",
new TimestampSpec(null, null, null),
new DimensionsSpec(Collections.emptyList()),
new AggregatorFactory[]{},
new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()),
TransformSpec.NONE,
null,
null
);
DataSchema dataSchema =
DataSchema.builder()
.withDataSource("datasource")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(new DimensionsSpec(Collections.emptyList()))
.withGranularity(new ArbitraryGranularitySpec(new AllGranularity(), Collections.emptyList()))
.build();
SeekableStreamIndexTaskTuningConfig tuningConfig = mock(SeekableStreamIndexTaskTuningConfig.class);
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.Result;
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.DoubleSumAggregatorFactory;
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 DataSchema OLD_DATA_SCHEMA;
protected static final DataSchema NEW_DATA_SCHEMA = new DataSchema(
"test_ds",
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
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
);
protected static final DataSchema NEW_DATA_SCHEMA =
DataSchema.builder()
.withDataSource("test_ds")
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
.withDimensions(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
.withAggregators(
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
)
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.build();
protected static final InputFormat INPUT_FORMAT = new JsonInputFormat(
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
@ -211,37 +208,38 @@ public abstract class SeekableStreamIndexTaskTestBase extends EasyMockSupport
static {
OBJECT_MAPPER = new TestUtils().getTestObjectMapper();
OBJECT_MAPPER.registerSubtypes(new NamedType(JSONParseSpec.class, "json"));
OLD_DATA_SCHEMA = new DataSchema(
"test_ds",
OBJECT_MAPPER.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
false
),
StandardCharsets.UTF_8.name()
),
Map.class
),
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
OBJECT_MAPPER
);
OLD_DATA_SCHEMA = DataSchema.builder()
.withDataSource("test_ds")
.withParserMap(
OBJECT_MAPPER.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
false
),
StandardCharsets.UTF_8.name()
),
Map.class
)
)
.withAggregators(
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
)
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.withObjectMapper(OBJECT_MAPPER)
.build();
}
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.granularity.Granularities;
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.DoubleSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
@ -106,36 +105,37 @@ public class SeekableStreamSamplerSpecTest extends EasyMockSupport
@Test(timeout = 10_000L)
public void testSampleWithInputRowParser() throws Exception
{
final DataSchema dataSchema = new DataSchema(
"test_ds",
OBJECT_MAPPER.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
false
)
),
Map.class
),
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
OBJECT_MAPPER
);
DataSchema dataSchema = DataSchema.builder()
.withDataSource("test_ds")
.withParserMap(
OBJECT_MAPPER.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
false
)
),
Map.class
)
)
.withAggregators(
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
)
.withGranularity(new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null))
.withObjectMapper(OBJECT_MAPPER)
.build();
final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig(
STREAM,

View File

@ -25,7 +25,6 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.impl.ByteEntity;
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.StringDimensionSchema;
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.StubServiceEmitter;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper;
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("dim2"));
return new DataSchema(
DATASOURCE,
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(dimensions),
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
),
null
);
return DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
.withDimensions(dimensions)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
)
)
.build();
}
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 org.apache.druid.data.input.impl.ByteEntity;
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.StringDimensionSchema;
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.metadata.PendingSegmentRecord;
import org.apache.druid.query.DruidMetrics;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.segment.TestHelper;
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("dim2"));
return new DataSchema(
DATASOURCE,
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(dimensions),
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
),
null
);
return DataSchema.builder()
.withDataSource(DATASOURCE)
.withTimestamp(new TimestampSpec("timestamp", "iso", null))
.withDimensions(dimensions)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.NONE,
ImmutableList.of()
)
)
.build();
}
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.TaskResource;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.junit.Assert;
import org.junit.Test;
@ -51,7 +50,7 @@ public class TaskAnnouncementTest
"theid",
new TaskResource("rofl", 2),
new IndexTask.IndexIngestionSpec(
new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()),
DataSchema.builder().withDataSource("foo").withObjectMapper(new DefaultObjectMapper()).build(),
ioConfig,
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.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.base.Strings;
import com.google.common.collect.Multiset;
@ -64,6 +63,17 @@ import java.util.stream.Collectors;
public class DataSchema
{
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 AggregatorFactory[] aggregators;
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)
{
IdUtils.validateId("dataSource", dataSource);
@ -403,44 +386,17 @@ public class DataSchema
public DataSchema withGranularitySpec(GranularitySpec granularitySpec)
{
return new DataSchema(
dataSource,
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec,
parserMap,
objectMapper
);
return builder(this).withGranularity(granularitySpec).build();
}
public DataSchema withTransformSpec(TransformSpec transformSpec)
{
return new DataSchema(
dataSource,
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec,
parserMap,
objectMapper
);
return builder(this).withTransform(transformSpec).build();
}
public DataSchema withDimensionsSpec(DimensionsSpec dimensionsSpec)
{
return new DataSchema(
dataSource,
timestampSpec,
dimensionsSpec,
aggregators,
granularitySpec,
transformSpec,
parserMap,
objectMapper
);
return builder(this).withDimensions(dimensionsSpec).build();
}
@Override
@ -457,4 +413,110 @@ public class DataSchema
", 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
{
private static ArbitraryGranularitySpec ARBITRARY_GRANULARITY = new ArbitraryGranularitySpec(
Granularities.DAY,
ImmutableList.of(Intervals.of("2014/2015"))
);
@Rule
public ExpectedException expectedException = ExpectedException.none();
@ -92,17 +97,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
parser,
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
jsonMapper
);
DataSchema schema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withParserMap(parser)
.withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2")
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
Assert.assertEquals(
ImmutableSet.of("__time", "time", "col1", "col2", "metric1", "metric2"),
@ -130,18 +134,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
null
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
parser,
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
jsonMapper
);
DataSchema schema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withParserMap(parser)
.withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2")
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
Assert.assertEquals(
ImmutableSet.of("__time", "dimC", "col1", "metric1", "metric2"),
@ -167,22 +169,28 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
parserMap,
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
new TransformSpec(
new SelectorDimFilter("dimA", "foo", null),
ImmutableList.of(
new ExpressionTransform("expr", "concat(dimA,dimA)", TestExprMacroTable.INSTANCE)
)
),
jsonMapper
);
DataSchema schema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withParserMap(parserMap)
.withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2")
)
.withGranularity(ARBITRARY_GRANULARITY)
.withTransform(
new TransformSpec(
new SelectorDimFilter("dimA", "foo", null),
ImmutableList.of(
new ExpressionTransform(
"expr",
"concat(dimA,dimA)",
TestExprMacroTable.INSTANCE
)
)
)
)
.withObjectMapper(jsonMapper)
.build();
// Test hack that produces a StringInputRowParser.
final StringInputRowParser parser = (StringInputRowParser) schema.getParser();
@ -233,17 +241,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
parser,
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
jsonMapper
);
DataSchema schema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withParserMap(parser)
.withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2")
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
expectedException.expect(DruidException.class);
expectedException.expectMessage(
@ -256,25 +263,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest
@Test
public void testOverlapTimeAndDimPositionZero()
{
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
new TimestampSpec("time", "auto", null),
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new LongDimensionSchema("__time"),
new StringDimensionSchema("dimA"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.build(),
null,
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
null,
jsonMapper
);
DataSchema schema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withTimestamp(new TimestampSpec("time", "auto", null))
.withDimensions(
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new LongDimensionSchema("__time"),
new StringDimensionSchema("dimA"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.build()
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
Assert.assertEquals(
ImmutableList.of("__time", "dimA", "dimB"),
@ -290,25 +296,24 @@ public class DataSchemaTest extends InitializedNullHandlingTest
expectedException.expect(DruidException.class);
expectedException.expectMessage("Encountered dimension[__time] with incorrect type[STRING]. Type must be 'long'.");
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
new TimestampSpec("time", "auto", null),
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new StringDimensionSchema("__time"),
new StringDimensionSchema("dimA"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.build(),
null,
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
null,
jsonMapper
);
DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withTimestamp(new TimestampSpec("time", "auto", null))
.withDimensions(
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new StringDimensionSchema("__time"),
new StringDimensionSchema("dimA"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.build()
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
}
@Test
@ -321,50 +326,49 @@ public class DataSchemaTest extends InitializedNullHandlingTest
+ DimensionsSpec.WARNING_NON_TIME_SORT_ORDER
);
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
new TimestampSpec("time", "auto", null),
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new StringDimensionSchema("dimA"),
new LongDimensionSchema("__time"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.build(),
null,
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
null,
jsonMapper
);
DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withTimestamp(new TimestampSpec("time", "auto", null))
.withDimensions(
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new StringDimensionSchema("dimA"),
new LongDimensionSchema("__time"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.build()
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
}
@Test
public void testOverlapTimeAndDimPositionOne_withExplicitSortOrder()
{
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
new TimestampSpec("time", "auto", null),
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new StringDimensionSchema("dimA"),
new LongDimensionSchema("__time"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.setForceSegmentSortByTime(false)
.build(),
null,
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
null,
jsonMapper
);
DataSchema schema =
DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withTimestamp(new TimestampSpec("time", "auto", null))
.withDimensions(
DimensionsSpec.builder()
.setDimensions(
ImmutableList.of(
new StringDimensionSchema("dimA"),
new LongDimensionSchema("__time"),
new StringDimensionSchema("dimB")
)
)
.setDimensionExclusions(ImmutableList.of("dimC"))
.setForceSegmentSortByTime(false)
.build()
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
Assert.assertEquals(
ImmutableList.of("dimA", "__time", "dimB"),
@ -402,14 +406,13 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
parser,
null,
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
jsonMapper
);
DataSchema schema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withParserMap(parser)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
expectedException.expect(DruidException.class);
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)"
);
DataSchema schema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
parser,
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
new DoubleSumAggregatorFactory("metric1", "col3"),
new DoubleSumAggregatorFactory("metric3", "col4"),
new DoubleSumAggregatorFactory("metric3", "col5"),
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
jsonMapper
);
DataSchema schema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withParserMap(parser)
.withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
new DoubleSumAggregatorFactory("metric1", "col3"),
new DoubleSumAggregatorFactory("metric3", "col4"),
new DoubleSumAggregatorFactory("metric3", "col5")
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
}
@Test
@ -510,24 +512,20 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DruidExceptionMatcher
.invalidInput()
.expectMessageIs("Invalid value for field [dataSource]: must not be null")
.assertThrowsAndMatches(
() -> new DataSchema(
"",
parser,
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
new ArbitraryGranularitySpec(
Granularities.DAY,
ImmutableList.of(Intervals.of("2014/2015"))
),
null,
jsonMapper
));
DruidExceptionMatcher.ThrowingSupplier thrower =
() -> DataSchema.builder()
.withDataSource("")
.withParserMap(parser)
.withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2")
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
DruidExceptionMatcher.invalidInput()
.expectMessageIs("Invalid value for field [dataSource]: must not be null")
.assertThrowsAndMatches(thrower);
}
@ -547,14 +545,11 @@ public class DataSchemaTest extends InitializedNullHandlingTest
dataSource
);
DruidExceptionMatcher.invalidInput().expectMessageIs(msg).assertThrowsAndMatches(
() -> new DataSchema(
dataSource,
Collections.emptyMap(),
null,
null,
null,
jsonMapper
)
() -> DataSchema.builder()
.withDataSource(dataSource)
.withParserMap(Collections.emptyMap())
.withObjectMapper(jsonMapper)
.build()
);
}
}
@ -686,17 +681,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
), JacksonUtils.TYPE_REFERENCE_MAP_STRING_OBJECT
);
DataSchema originalSchema = new DataSchema(
IdUtilsTest.VALID_ID_CHARS,
parser,
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
null,
jsonMapper
);
DataSchema originalSchema = DataSchema.builder()
.withDataSource(IdUtilsTest.VALID_ID_CHARS)
.withParserMap(parser)
.withAggregators(
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2")
)
.withGranularity(ARBITRARY_GRANULARITY)
.withObjectMapper(jsonMapper)
.build();
String serialized = jsonMapper.writeValueAsString(originalSchema);
TestModifiedDataSchema deserialized = jsonMapper.readValue(serialized, TestModifiedDataSchema.class);
@ -734,7 +728,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest
new DoubleSumAggregatorFactory("metric1", "col1"),
new DoubleSumAggregatorFactory("metric2", "col2"),
},
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
ARBITRARY_GRANULARITY,
null,
parser,
jsonMapper,
@ -765,10 +759,16 @@ public class DataSchemaTest extends InitializedNullHandlingTest
Map<String, Object> parserMap = Mockito.mock(Map.class);
Mockito.when(newDimSpec.withDimensionExclusions(ArgumentMatchers.any(Set.class))).thenReturn(newDimSpec);
DataSchema oldSchema = new DataSchema("dataSource", tsSpec, oldDimSpec,
new AggregatorFactory[]{aggFactory}, gSpec,
transSpec, parserMap, jsonMapper
);
DataSchema oldSchema = DataSchema.builder()
.withDataSource("dataSource")
.withTimestamp(tsSpec)
.withDimensions(oldDimSpec)
.withAggregators(aggFactory)
.withGranularity(gSpec)
.withTransform(transSpec)
.withParserMap(parserMap)
.withObjectMapper(jsonMapper)
.build();
DataSchema newSchema = oldSchema.withDimensionsSpec(newDimSpec);
Assert.assertSame(oldSchema.getDataSource(), newSchema.getDataSource());
Assert.assertSame(oldSchema.getTimestampSpec(), newSchema.getTimestampSpec());
@ -795,7 +795,7 @@ public class DataSchemaTest extends InitializedNullHandlingTest
.setDimensionExclusions(ImmutableList.of("dimC"))
.build(),
null,
new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))),
ARBITRARY_GRANULARITY,
null,
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.core.NoopEmitter;
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.LongSumAggregatorFactory;
import org.apache.druid.segment.IndexIO;
@ -151,19 +150,18 @@ public class BatchAppenderatorTester implements AutoCloseable
Map.class
);
schema = new DataSchema(
DATASOURCE,
null,
null,
new AggregatorFactory[]{
new CountAggregatorFactory("count"),
new LongSumAggregatorFactory("met", "met")
},
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
null,
parserMap,
objectMapper
);
schema = DataSchema.builder()
.withDataSource(DATASOURCE)
.withAggregators(
new CountAggregatorFactory("count"),
new LongSumAggregatorFactory("met", "met")
)
.withGranularity(
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null)
)
.withParserMap(parserMap)
.withObjectMapper(objectMapper)
.build();
tuningConfig = new TestAppenderatorConfig(
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.ForwardingQueryProcessingPool;
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.LongSumAggregatorFactory;
import org.apache.druid.query.expression.TestExprMacroTable;
@ -135,17 +134,16 @@ public class StreamAppenderatorTester implements AutoCloseable
),
Map.class
);
schema = new DataSchema(
DATASOURCE,
parserMap,
new AggregatorFactory[]{
new CountAggregatorFactory("count"),
new LongSumAggregatorFactory("met", "met")
},
new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null),
null,
objectMapper
);
schema = DataSchema.builder()
.withDataSource(DATASOURCE)
.withParserMap(parserMap)
.withAggregators(
new CountAggregatorFactory("count"),
new LongSumAggregatorFactory("met", "met")
)
.withGranularity(new UniformGranularitySpec(Granularities.MINUTE, Granularities.NONE, null))
.withObjectMapper(objectMapper)
.build();
tuningConfig = new TestAppenderatorConfig(
TuningConfig.DEFAULT_APPENDABLE_INDEX,
maxRowsInMemory,

View File

@ -98,14 +98,11 @@ public class UnifiedIndexerAppenderatorsManagerTest extends InitializedNullHandl
EasyMock.replay(appenderatorConfig);
appenderator = manager.createBatchAppenderatorForTask(
"taskId",
new DataSchema(
"myDataSource",
new TimestampSpec("__time", "millis", null),
null,
null,
new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()),
null
),
DataSchema.builder()
.withDataSource("myDataSource")
.withTimestamp(new TimestampSpec("__time", "millis", null))
.withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.HOUR, false, Collections.emptyList()))
.build(),
appenderatorConfig,
new SegmentGenerationMetrics(),
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.granularity.Granularities;
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.segment.RowAdapters;
import org.apache.druid.segment.RowBasedSegment;
@ -76,14 +75,14 @@ public class SinkTest extends InitializedNullHandlingTest
@Test
public void testSwap() throws Exception
{
final DataSchema schema = new DataSchema(
"test",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null),
null
);
final DataSchema schema =
DataSchema.builder()
.withDataSource("test")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(DimensionsSpec.EMPTY)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null))
.build();
final Interval interval = Intervals.of("2013-01-01/2013-01-02");
final String version = DateTimes.nowUtc().toString();
@ -256,18 +255,17 @@ public class SinkTest extends InitializedNullHandlingTest
@Test
public void testGetSinkSignature() throws IndexSizeExceededException
{
final DataSchema schema = new DataSchema(
"test",
new TimestampSpec(null, null, null),
new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new LongDimensionSchema("dimLong")
)),
new AggregatorFactory[]{new CountAggregatorFactory("rows")},
new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null),
null
);
final DataSchema schema =
DataSchema.builder()
.withDataSource("test")
.withTimestamp(new TimestampSpec(null, null, null))
.withDimensions(
new StringDimensionSchema("dim1"),
new LongDimensionSchema("dimLong")
)
.withAggregators(new CountAggregatorFactory("rows"))
.withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.MINUTE, null))
.build();
final Interval interval = Intervals.of("2013-01-01/2013-01-02");
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.jackson.DefaultObjectMapper;
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.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
@ -133,14 +132,11 @@ public class DruidJsonValidatorTest
null,
new TaskResource("rofl", 2),
new IndexTask.IndexIngestionSpec(
new DataSchema(
"foo",
null,
new AggregatorFactory[0],
new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null),
null,
jsonMapper
),
DataSchema.builder()
.withDataSource("foo")
.withGranularity(new UniformGranularitySpec(Granularities.HOUR, Granularities.NONE, null))
.withObjectMapper(jsonMapper)
.build(),
new IndexTask.IndexIOConfig(
new LocalInputSource(new File("lol"), "rofl"),
new JsonInputFormat(null, null, null, null, null),