mirror of https://github.com/apache/druid.git
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:
parent
6ed8632420
commit
aa6336c5cf
|
@ -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(),
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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"),
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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);
|
||||
|
||||
|
|
|
@ -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
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -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()
|
||||
);
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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());
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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",
|
||||
|
|
|
@ -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),
|
||||
|
|
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
||||
|
|
|
@ -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();
|
||||
|
||||
|
|
|
@ -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(
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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)
|
||||
|
|
|
@ -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()
|
||||
|
|
|
@ -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
|
||||
),
|
||||
|
|
|
@ -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
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -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
|
||||
);
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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,
|
||||
|
|
|
@ -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(),
|
||||
|
|
|
@ -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();
|
||||
|
|
|
@ -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),
|
||||
|
|
Loading…
Reference in New Issue