backport projections (#17257)

* abstract `IncrementalIndex` cursor stuff to prepare for using different "views" of the data based on the cursor build spec (#17064)

* abstract `IncrementalIndex` cursor stuff to prepare to allow for possibility of using different "views" of the data based on the cursor build spec
changes:
* introduce `IncrementalIndexRowSelector` interface to capture how `IncrementalIndexCursor` and `IncrementalIndexColumnSelectorFactory` read data
* `IncrementalIndex` implements `IncrementalIndexRowSelector`
* move `FactsHolder` interface to separate file
* other minor refactorings

* 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

* Projections prototype (#17214)
This commit is contained in:
Clint Wylie 2024-10-05 10:33:41 -07:00 committed by GitHub
parent 1435b9f4bd
commit 7b3fc4e768
No known key found for this signature in database
GPG Key ID: B5690EEEBB952194
191 changed files with 7470 additions and 2599 deletions

View File

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

View File

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

View File

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

View File

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

View File

@ -278,6 +278,26 @@ public abstract class HllSketchAggregatorFactory extends AggregatorFactory
&& stringEncoding == that.stringEncoding;
}
@Nullable
@Override
public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated)
{
if (this == preAggregated) {
return getCombiningFactory();
}
if (getClass() != preAggregated.getClass()) {
return null;
}
HllSketchAggregatorFactory that = (HllSketchAggregatorFactory) preAggregated;
if (lgK == that.lgK && tgtHllType == that.tgtHllType && stringEncoding == that.stringEncoding && Objects.equals(
fieldName,
that.fieldName
)) {
return getCombiningFactory();
}
return null;
}
@Override
public int hashCode()
{

View File

@ -231,6 +231,42 @@ public class HllSketchAggregatorFactoryTest
Assert.assertArrayEquals(target.getCacheKey(), other.getCacheKey());
}
@Test
public void testCanSubstitute()
{
HllSketchBuildAggregatorFactory factory = new HllSketchBuildAggregatorFactory(
NAME,
FIELD_NAME,
LG_K,
TGT_HLL_TYPE,
STRING_ENCODING,
true,
true
);
HllSketchBuildAggregatorFactory other = new HllSketchBuildAggregatorFactory(
"other name",
FIELD_NAME,
LG_K,
TGT_HLL_TYPE,
STRING_ENCODING,
false,
false
);
HllSketchBuildAggregatorFactory incompatible = new HllSketchBuildAggregatorFactory(
NAME,
"different field",
LG_K,
TGT_HLL_TYPE,
STRING_ENCODING,
false,
false
);
Assert.assertNotNull(other.substituteCombiningFactory(factory));
Assert.assertNotNull(factory.substituteCombiningFactory(other));
Assert.assertNull(factory.substituteCombiningFactory(incompatible));
}
@Test
public void testToString()
{

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -296,7 +296,7 @@ public class ChannelStageOutputReaderTest extends InitializedNullHandlingTest
frameReader = FrameReader.create(adapter.getRowSignature());
frameList = FrameSequenceBuilder.fromCursorFactory(adapter)
.frameType(FrameType.ROW_BASED)
.maxRowsPerFrame(IntMath.divide(index.size(), MAX_FRAMES, RoundingMode.CEILING))
.maxRowsPerFrame(IntMath.divide(index.numRows(), MAX_FRAMES, RoundingMode.CEILING))
.frames()
.toList();
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -165,7 +165,7 @@ public class DatasourceRecordReaderSegmentReaderTest
Assert.assertEquals(18, count);
// Check the index
Assert.assertEquals(9, index.size());
Assert.assertEquals(9, index.numRows());
final IncrementalIndexSegment queryable = new IncrementalIndexSegment(index, SegmentId.dummy("test"));
final List<String> dimensions = index.getDimensionNames(false);
Assert.assertEquals(2, dimensions.size());

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -1913,6 +1913,7 @@ public class CompactionTaskTest
null,
null,
null,
null,
null
);
}

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

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

View File

@ -0,0 +1,212 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.data.input.impl;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Lists;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.query.OrderBy;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.AggregateProjectionMetadata;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
/**
* API type to specify an aggregating projection on {@link org.apache.druid.segment.incremental.IncrementalIndexSchema}
*
* Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of
* projections and need to extract out a base interface from this class.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonTypeName(AggregateProjectionSpec.TYPE_NAME)
public class AggregateProjectionSpec
{
public static final String TYPE_NAME = "aggregate";
private final String name;
private final List<DimensionSchema> groupingColumns;
private final VirtualColumns virtualColumns;
private final AggregatorFactory[] aggregators;
private final List<OrderBy> ordering;
@Nullable
private final String timeColumnName;
@JsonCreator
public AggregateProjectionSpec(
@JsonProperty("name") String name,
@JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns,
@JsonProperty("groupingColumns") @Nullable List<DimensionSchema> groupingColumns,
@JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators
)
{
this.name = name;
if (CollectionUtils.isNullOrEmpty(groupingColumns)) {
throw InvalidInput.exception("groupingColumns must not be null or empty");
}
this.groupingColumns = groupingColumns;
this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns;
// in the future this should be expanded to support user specified ordering, but for now we compute it based on
// the grouping columns, which is consistent with how rollup ordering works for incremental index base table
final ProjectionOrdering ordering = computeOrdering(this.virtualColumns, this.groupingColumns);
this.ordering = ordering.ordering;
this.timeColumnName = ordering.timeColumnName;
this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators;
}
@JsonProperty
public String getName()
{
return name;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public VirtualColumns getVirtualColumns()
{
return virtualColumns;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public List<DimensionSchema> getGroupingColumns()
{
return groupingColumns;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public AggregatorFactory[] getAggregators()
{
return aggregators;
}
@JsonProperty
public List<OrderBy> getOrdering()
{
return ordering;
}
@JsonIgnore
public AggregateProjectionMetadata.Schema toMetadataSchema()
{
return new AggregateProjectionMetadata.Schema(
name,
timeColumnName,
virtualColumns,
groupingColumns.stream().map(DimensionSchema::getName).collect(Collectors.toList()),
aggregators,
ordering
);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
AggregateProjectionSpec that = (AggregateProjectionSpec) o;
return Objects.equals(name, that.name)
&& Objects.equals(groupingColumns, that.groupingColumns)
&& Objects.equals(virtualColumns, that.virtualColumns)
&& Objects.deepEquals(aggregators, that.aggregators)
&& Objects.equals(ordering, that.ordering);
}
@Override
public int hashCode()
{
return Objects.hash(name, groupingColumns, virtualColumns, Arrays.hashCode(aggregators), ordering);
}
@Override
public String toString()
{
return "AggregateProjectionSpec{" +
"name='" + name + '\'' +
", groupingColumns=" + groupingColumns +
", virtualColumns=" + virtualColumns +
", aggregators=" + Arrays.toString(aggregators) +
", ordering=" + ordering +
'}';
}
private static ProjectionOrdering computeOrdering(VirtualColumns virtualColumns, List<DimensionSchema> groupingColumns)
{
final List<OrderBy> ordering = Lists.newArrayListWithCapacity(groupingColumns.size());
String timeColumnName = null;
Granularity granularity = null;
// try to find the __time column equivalent, which might be a time_floor expression to model granularity
// bucketing. The time column is decided as the finest granularity on __time detected. If the projection does
// not have a time-like column, the granularity will be handled as ALL for the projection and all projection
// rows will use a synthetic timestamp of the minimum timestamp of the incremental index
for (final DimensionSchema dimension : groupingColumns) {
ordering.add(OrderBy.ascending(dimension.getName()));
if (ColumnHolder.TIME_COLUMN_NAME.equals(dimension.getName())) {
timeColumnName = dimension.getName();
granularity = Granularities.NONE;
} else {
final VirtualColumn vc = virtualColumns.getVirtualColumn(dimension.getName());
final Granularity maybeGranularity = Granularities.fromVirtualColumn(vc);
if (granularity == null && maybeGranularity != null) {
granularity = maybeGranularity;
timeColumnName = dimension.getName();
} else if (granularity != null && maybeGranularity != null && maybeGranularity.isFinerThan(granularity)) {
granularity = maybeGranularity;
timeColumnName = dimension.getName();
}
}
}
return new ProjectionOrdering(ordering, timeColumnName);
}
private static final class ProjectionOrdering
{
private final List<OrderBy> ordering;
@Nullable
private final String timeColumnName;
private ProjectionOrdering(List<OrderBy> ordering, @Nullable String timeColumnName)
{
this.ordering = ordering;
this.timeColumnName = timeColumnName;
}
}
}

View File

@ -23,9 +23,11 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Query;
import org.apache.druid.query.expression.TimestampFloorExprMacro;
import org.apache.druid.segment.AggregateProjectionMetadata;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.VirtualColumns;
@ -160,4 +162,32 @@ public class Granularities
ExprMacroTable.granularity()
);
}
/**
* Converts a virtual column with a single input time column into a {@link Granularity} if it is a
* {@link TimestampFloorExprMacro.TimestampFloorExpr}.
* <p>
* IMPORTANT - this method DOES NOT VERIFY that the virtual column has a single input that is a time column
* ({@link ColumnHolder#TIME_COLUMN_NAME} or equivalent projection time column as defined by
* {@link AggregateProjectionMetadata.Schema#getTimeColumnName()}). Callers must verify this externally before
* calling this method by examining {@link VirtualColumn#requiredColumns()}.
* <p>
* This method also does not handle other time expressions, or if the virtual column is just an identifier for a
* time column
*/
@Nullable
public static Granularity fromVirtualColumn(VirtualColumn virtualColumn)
{
if (virtualColumn instanceof ExpressionVirtualColumn) {
final ExpressionVirtualColumn expressionVirtualColumn = (ExpressionVirtualColumn) virtualColumn;
final Expr expr = expressionVirtualColumn.getParsedExpression().get();
if (expr instanceof TimestampFloorExprMacro.TimestampFloorExpr) {
final TimestampFloorExprMacro.TimestampFloorExpr gran = (TimestampFloorExprMacro.TimestampFloorExpr) expr;
if (gran.getArg().getBindingIfIdentifier() != null) {
return gran.getGranularity();
}
}
}
return null;
}
}

View File

@ -190,6 +190,12 @@ public class DefaultQueryMetrics<QueryType extends Query<?>> implements QueryMet
setDimension("segment", segmentIdentifier);
}
@Override
public void projection(String projection)
{
setDimension("projection", projection);
}
@Override
public void identity(String identity)
{

View File

@ -88,6 +88,12 @@ public class QueryContexts
public static final String UNCOVERED_INTERVALS_LIMIT_KEY = "uncoveredIntervalsLimit";
public static final String MIN_TOP_N_THRESHOLD = "minTopNThreshold";
public static final String CATALOG_VALIDATION_ENABLED = "catalogValidationEnabled";
// projection context keys
public static final String NO_PROJECTIONS = "noProjections";
public static final String FORCE_PROJECTION = "forceProjections";
public static final String USE_PROJECTION = "useProjection";
// Unique identifier for the query, that is used to map the global shared resources (specifically merge buffers) to the
// query's runtime
public static final String QUERY_RESOURCE_ID = "queryResourceId";

View File

@ -243,6 +243,11 @@ public interface QueryMetrics<QueryType extends Query<?>>
void segment(String segmentIdentifier);
/**
* If a projection was used during segment processing, set its name as the projection dimension
*/
void projection(String projection);
/**
* @deprecated use {@link #filterBundle(FilterBundle.BundleInfo)} instead to collect details about filters which were
* used to construct {@link org.apache.druid.segment.BitmapOffset} or

View File

@ -26,6 +26,8 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.query.PerSegmentQueryOptimizationContext;
import org.apache.druid.segment.ColumnInspector;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.ColumnTypeFactory;
import org.apache.druid.segment.column.ValueType;
@ -365,6 +367,38 @@ public abstract class AggregatorFactory implements Cacheable
throw new UOE("Cannot change output name for AggregatorFactory[%s].", this.getClass().getName());
}
/**
* Check to see if we can make a 'combining' factory of this aggregator that is suitable to process input from a
* selector of values produced by the other {@link AggregatorFactory} representing pre-aggregated data. Typically,
* this means that this and the other aggregator have the same inputs ({@link #requiredFields()}, and the same
* options for how the data was constructed into the intermediary type. If suitable, this method returns a
* 'combining' aggregator factory of this aggregator to use to process the pre-aggregated data which was produced by
* the other aggregator.
* <p>
* This method is used indirectly in service of checking if a
* {@link org.apache.druid.segment.projections.QueryableProjection} can be used instead of the base table during
* {@link org.apache.druid.segment.CursorFactory#makeCursorHolder(CursorBuildSpec)}, which checks if this
* aggregator can be substituted for its combining aggregator if and only if there exists a column that a cursor can
* read which was created by an aggregator that satisfies this method. In other words, this aggregator is the 'query'
* aggregator defined on the {@link CursorBuildSpec}, the argument to this method is the aggregator which created
* some column whose selectors are available to the cursor. If all aggregators on the {@link CursorBuildSpec} can be
* paired with aggregators from the underlying table in the cursor factory, then
* {@link CursorHolder#isPreAggregated()} will be set to true indicating that query engines should use this
* substituted aggregator instead of the original aggregators.
*
* @param preAggregated {@link AggregatorFactory} which produced the partially aggregated values which are
* available in a selector
* @return a "combining" {@link AggregatorFactory} to use with the pre-aggregated selector data
*/
@Nullable
public AggregatorFactory substituteCombiningFactory(AggregatorFactory preAggregated)
{
if (equals(preAggregated.withName(getName()))) {
return getCombiningFactory();
}
return null;
}
/**
* Merges the list of AggregatorFactory[] (presumable from metadata of some segments being merged) and
* returns merged AggregatorFactory[] (for the metadata for merged segment).

View File

@ -454,13 +454,4 @@ public class AggregatorUtil
}
return false;
}
public static List<AggregatorFactory> getCombiningAggregators(List<AggregatorFactory> aggs)
{
List<AggregatorFactory> combining = new ArrayList<>(aggs.size());
for (AggregatorFactory agg : aggs) {
combining.add(agg.getCombiningFactory());
}
return combining;
}
}

View File

@ -20,6 +20,7 @@
package org.apache.druid.query.groupby;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
import com.google.common.base.Suppliers;
import com.google.common.collect.ImmutableList;
@ -60,7 +61,6 @@ import org.apache.druid.query.QueryWatcher;
import org.apache.druid.query.ResourceLimitExceededException;
import org.apache.druid.query.ResultMergeQueryRunner;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.aggregation.PostAggregator;
import org.apache.druid.query.context.ResponseContext;
import org.apache.druid.query.dimension.DefaultDimensionSpec;
@ -509,7 +509,7 @@ public class GroupingEngine
final CursorHolder cursorHolder = closer.register(cursorFactory.makeCursorHolder(buildSpec));
if (cursorHolder.isPreAggregated()) {
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs()));
query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
}
final ColumnInspector inspector = query.getVirtualColumns().wrapInspector(cursorFactory);

View File

@ -151,6 +151,12 @@ public class DefaultSearchQueryMetrics implements SearchQueryMetrics
delegateQueryMetrics.segment(segmentIdentifier);
}
@Override
public void projection(String projection)
{
delegateQueryMetrics.projection(projection);
}
@Override
public void filterBundle(FilterBundle.BundleInfo bundleInfo)
{

View File

@ -20,6 +20,7 @@
package org.apache.druid.query.timeseries;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.inject.Inject;
import org.apache.druid.collections.NonBlockingPool;
@ -38,7 +39,6 @@ import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.Aggregator;
import org.apache.druid.query.aggregation.AggregatorAdapters;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.vector.VectorCursorGranularizer;
import org.apache.druid.segment.ColumnSelectorFactory;
import org.apache.druid.segment.Cursor;
@ -104,7 +104,7 @@ public class TimeseriesQueryEngine
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(makeCursorBuildSpec(query, timeseriesQueryMetrics));
if (cursorHolder.isPreAggregated()) {
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs()));
query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
}
try {
final Sequence<Result<TimeseriesResultValue>> result;

View File

@ -19,6 +19,7 @@
package org.apache.druid.query.topn;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicates;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.collections.ResourceHolder;
@ -30,7 +31,6 @@ import org.apache.druid.query.CursorGranularizer;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.Result;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.AggregatorUtil;
import org.apache.druid.query.extraction.ExtractionFn;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessor;
import org.apache.druid.query.topn.types.TopNColumnAggregatesProcessorFactory;
@ -89,7 +89,7 @@ public class TopNQueryEngine
final CursorBuildSpec buildSpec = makeCursorBuildSpec(query, queryMetrics);
final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec);
if (cursorHolder.isPreAggregated()) {
query = query.withAggregatorSpecs(AggregatorUtil.getCombiningAggregators(query.getAggregatorSpecs()));
query = query.withAggregatorSpecs(Preconditions.checkNotNull(cursorHolder.getAggregatorsForPreAggregated()));
}
final Cursor cursor = cursorHolder.asCursor();
if (cursor == null) {

View File

@ -0,0 +1,463 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonIgnore;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeInfo;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.google.common.collect.Interner;
import com.google.common.collect.Interners;
import com.google.common.collect.Lists;
import org.apache.druid.data.input.impl.AggregateProjectionSpec;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
import org.apache.druid.query.OrderBy;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.projections.Projections;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.Objects;
/**
* Aggregate projection schema and row count information to store in {@link Metadata} which itself is stored inside a
* segment, defining which projections exist for the segment.
* <p>
* Decorated with {@link JsonTypeInfo} annotations as a future-proofing mechanism in the event we add other types of
* projections and need to extract out a base interface from this class.
*/
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type")
@JsonTypeName(AggregateProjectionSpec.TYPE_NAME)
public class AggregateProjectionMetadata
{
private static final Interner<Schema> SCHEMA_INTERNER = Interners.newWeakInterner();
public static final Comparator<AggregateProjectionMetadata> COMPARATOR = (o1, o2) -> {
int rowCompare = Integer.compare(o1.numRows, o2.numRows);
if (rowCompare != 0) {
return rowCompare;
}
return Schema.COMPARATOR.compare(o1.getSchema(), o2.getSchema());
};
private final Schema schema;
private final int numRows;
@JsonCreator
public AggregateProjectionMetadata(
@JsonProperty("schema") Schema schema,
@JsonProperty("numRows") int numRows
)
{
this.schema = SCHEMA_INTERNER.intern(schema);
this.numRows = numRows;
}
@JsonProperty
public Schema getSchema()
{
return schema;
}
@JsonProperty
public int getNumRows()
{
return numRows;
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
AggregateProjectionMetadata that = (AggregateProjectionMetadata) o;
return numRows == that.numRows && Objects.equals(schema, that.schema);
}
@Override
public int hashCode()
{
return Objects.hash(schema, numRows);
}
public static class Schema
{
/**
* It is not likely the best way to find the best matching projections, but it is the one we have for now. This
* comparator is used to sort all the projections in a segment "best" first, where best is defined as fewest grouping
* columns, most virtual columns and aggregators, as an approximation of likely to have the fewest number of rows to
* scan.
*/
public static final Comparator<Schema> COMPARATOR = (o1, o2) -> {
// coarsest granularity first
if (o1.getGranularity().isFinerThan(o2.getGranularity())) {
return 1;
}
if (o2.getGranularity().isFinerThan(o1.getGranularity())) {
return -1;
}
// fewer dimensions first
final int dimsCompare = Integer.compare(
o1.groupingColumns.size(),
o2.groupingColumns.size()
);
if (dimsCompare != 0) {
return dimsCompare;
}
// more metrics first
int metCompare = Integer.compare(o2.aggregators.length, o1.aggregators.length);
if (metCompare != 0) {
return metCompare;
}
// more virtual columns first
final int virtCompare = Integer.compare(
o2.virtualColumns.getVirtualColumns().length,
o1.virtualColumns.getVirtualColumns().length
);
if (virtCompare != 0) {
return virtCompare;
}
return o1.name.compareTo(o2.name);
};
private final String name;
@Nullable
private final String timeColumnName;
private final VirtualColumns virtualColumns;
private final List<String> groupingColumns;
private final AggregatorFactory[] aggregators;
private final List<OrderBy> ordering;
private final List<OrderBy> orderingWithTimeSubstitution;
// computed fields
private final int timeColumnPosition;
private final Granularity granularity;
@JsonCreator
public Schema(
@JsonProperty("name") String name,
@JsonProperty("timeColumnName") @Nullable String timeColumnName,
@JsonProperty("virtualColumns") @Nullable VirtualColumns virtualColumns,
@JsonProperty("groupingColumns") List<String> groupingColumns,
@JsonProperty("aggregators") @Nullable AggregatorFactory[] aggregators,
@JsonProperty("ordering") List<OrderBy> ordering
)
{
this.name = name;
if (CollectionUtils.isNullOrEmpty(groupingColumns)) {
throw InvalidInput.exception("groupingColumns must not be null or empty");
}
this.virtualColumns = virtualColumns == null ? VirtualColumns.EMPTY : virtualColumns;
this.groupingColumns = groupingColumns;
this.aggregators = aggregators == null ? new AggregatorFactory[0] : aggregators;
this.ordering = ordering;
int foundTimePosition = -1;
this.orderingWithTimeSubstitution = Lists.newArrayListWithCapacity(ordering.size());
Granularity granularity = null;
for (int i = 0; i < ordering.size(); i++) {
OrderBy orderBy = ordering.get(i);
if (orderBy.getColumnName().equals(timeColumnName)) {
orderingWithTimeSubstitution.add(new OrderBy(ColumnHolder.TIME_COLUMN_NAME, orderBy.getOrder()));
foundTimePosition = i;
timeColumnName = groupingColumns.get(foundTimePosition);
final VirtualColumn vc = this.virtualColumns.getVirtualColumn(groupingColumns.get(foundTimePosition));
if (vc != null) {
granularity = Granularities.fromVirtualColumn(vc);
} else {
granularity = Granularities.NONE;
}
} else {
orderingWithTimeSubstitution.add(orderBy);
}
}
this.timeColumnName = timeColumnName;
this.timeColumnPosition = foundTimePosition;
this.granularity = granularity == null ? Granularities.ALL : granularity;
}
@JsonProperty
public String getName()
{
return name;
}
@JsonProperty
@Nullable
public String getTimeColumnName()
{
return timeColumnName;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public VirtualColumns getVirtualColumns()
{
return virtualColumns;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public List<String> getGroupingColumns()
{
return groupingColumns;
}
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_DEFAULT)
public AggregatorFactory[] getAggregators()
{
return aggregators;
}
@JsonProperty
public List<OrderBy> getOrdering()
{
return ordering;
}
@JsonIgnore
public List<OrderBy> getOrderingWithTimeColumnSubstitution()
{
return orderingWithTimeSubstitution;
}
@JsonIgnore
public int getTimeColumnPosition()
{
return timeColumnPosition;
}
@JsonIgnore
public Granularity getGranularity()
{
return granularity;
}
/**
* Check if this projection "matches" a {@link CursorBuildSpec} for a query to see if we can use a projection
* instead. For a projection to match, all grouping columns of the build spec must match, virtual columns of the
* build spec must either be available as a physical column on the projection, or the inputs to the virtual column
* must be available on the projection, and all aggregators must be compatible with pre-aggregated columns of the
* projection per {@link AggregatorFactory#substituteCombiningFactory(AggregatorFactory)}. If the projection
* matches, this method returns a {@link Projections.ProjectionMatch} which contains an updated
* {@link CursorBuildSpec} which has the remaining virtual columns from the original build spec which must still be
* computed and the 'combining' aggregator factories to process the pre-aggregated data from the projection, as well
* as a mapping of query column names to projection column names.
*
* @param queryCursorBuildSpec the {@link CursorBuildSpec} that contains the required inputs to build a
* {@link CursorHolder} for a query
* @param physicalColumnChecker Helper utility which can determine if a physical column required by
* queryCursorBuildSpec is available on the projection OR does not exist on the base
* table either
* @return a {@link Projections.ProjectionMatch} if the {@link CursorBuildSpec} matches the projection, which
* contains information such as which
*/
@Nullable
public Projections.ProjectionMatch matches(
CursorBuildSpec queryCursorBuildSpec,
Projections.PhysicalColumnChecker physicalColumnChecker
)
{
if (!queryCursorBuildSpec.isCompatibleOrdering(orderingWithTimeSubstitution)) {
return null;
}
final List<String> queryGrouping = queryCursorBuildSpec.getGroupingColumns();
Projections.ProjectionMatchBuilder matchBuilder = new Projections.ProjectionMatchBuilder();
if (timeColumnName != null) {
matchBuilder.remapColumn(timeColumnName, ColumnHolder.TIME_COLUMN_NAME);
}
if (queryGrouping != null) {
for (String queryColumn : queryGrouping) {
matchBuilder = matchRequiredColumn(
matchBuilder,
queryColumn,
queryCursorBuildSpec.getVirtualColumns(),
physicalColumnChecker
);
if (matchBuilder == null) {
return null;
}
}
}
if (queryCursorBuildSpec.getFilter() != null) {
for (String queryColumn : queryCursorBuildSpec.getFilter().getRequiredColumns()) {
matchBuilder = matchRequiredColumn(
matchBuilder,
queryColumn,
queryCursorBuildSpec.getVirtualColumns(),
physicalColumnChecker
);
if (matchBuilder == null) {
return null;
}
}
}
if (!CollectionUtils.isNullOrEmpty(queryCursorBuildSpec.getAggregators())) {
boolean allMatch = true;
for (AggregatorFactory queryAgg : queryCursorBuildSpec.getAggregators()) {
boolean foundMatch = false;
for (AggregatorFactory projectionAgg : aggregators) {
final AggregatorFactory combining = queryAgg.substituteCombiningFactory(projectionAgg);
if (combining != null) {
matchBuilder.remapColumn(queryAgg.getName(), projectionAgg.getName()).addPreAggregatedAggregator(combining);
foundMatch = true;
}
}
allMatch = allMatch && foundMatch;
}
if (!allMatch) {
return null;
}
}
return matchBuilder.build(queryCursorBuildSpec);
}
/**
* Ensure that the projection has the specified column required by a {@link CursorBuildSpec} in one form or another.
* If the column is a {@link VirtualColumn} on the build spec, ensure that the projection has an equivalent virtual
* column, or has the required inputs to compute the virtual column. If an equivalent virtual column exists, its
* name will be added to {@link Projections.ProjectionMatchBuilder#remapColumn(String, String)} so the query
* virtual column name can be mapped to the projection physical column name. If no equivalent virtual column exists,
* but the inputs are available on the projection to compute it, it will be added to
* {@link Projections.ProjectionMatchBuilder#addReferenceedVirtualColumn(VirtualColumn)}.
* <p>
* Finally, if the column is not a virtual column in the query, it is checked with
* {@link Projections.PhysicalColumnChecker} which true if the column is present on the projection OR if the column
* is NOT present on the base table (meaning missing columns that do not exist anywhere do not disqualify a
* projection from being used).
*
* @param matchBuilder match state to add mappings of query virtual columns to projection physical columns
* and query virtual columns which still must be computed from projection physical
* columns
* @param column Column name to check
* @param queryVirtualColumns {@link VirtualColumns} from the {@link CursorBuildSpec} required by the query
* @param physicalColumnChecker Helper to check if the physical column exists on a projection, or does not exist on
* the base table
* @return {@link Projections.ProjectionMatchBuilder} with updated state per the rules described above, or null
* if the column cannot be matched
*/
@Nullable
private Projections.ProjectionMatchBuilder matchRequiredColumn(
Projections.ProjectionMatchBuilder matchBuilder,
String column,
VirtualColumns queryVirtualColumns,
Projections.PhysicalColumnChecker physicalColumnChecker
)
{
final VirtualColumn buildSpecVirtualColumn = queryVirtualColumns.getVirtualColumn(column);
if (buildSpecVirtualColumn != null) {
// check to see if we have an equivalent virtual column defined in the projection, if so we can
final VirtualColumn projectionEquivalent = virtualColumns.findEquivalent(buildSpecVirtualColumn);
if (projectionEquivalent != null) {
if (!buildSpecVirtualColumn.getOutputName().equals(projectionEquivalent.getOutputName())) {
matchBuilder.remapColumn(
buildSpecVirtualColumn.getOutputName(),
projectionEquivalent.getOutputName()
);
}
return matchBuilder;
}
matchBuilder.addReferenceedVirtualColumn(buildSpecVirtualColumn);
final List<String> requiredInputs = buildSpecVirtualColumn.requiredColumns();
if (requiredInputs.size() == 1 && ColumnHolder.TIME_COLUMN_NAME.equals(requiredInputs.get(0))) {
// special handle time granularity. in the future this should be reworked to push this concept into the
// virtual column and underlying expression itself, but this will do for now
final Granularity virtualGranularity = Granularities.fromVirtualColumn(buildSpecVirtualColumn);
if (virtualGranularity != null) {
if (virtualGranularity.isFinerThan(granularity)) {
return null;
}
return matchBuilder.remapColumn(column, timeColumnName);
} else {
// anything else with __time requires none granularity
if (Granularities.NONE.equals(granularity)) {
return matchBuilder;
}
return null;
}
} else {
for (String required : requiredInputs) {
matchBuilder = matchRequiredColumn(
matchBuilder,
required,
queryVirtualColumns,
physicalColumnChecker
);
if (matchBuilder == null) {
return null;
}
}
return matchBuilder;
}
} else {
if (physicalColumnChecker.check(name, column)) {
return matchBuilder;
}
return null;
}
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
Schema schema = (Schema) o;
return Objects.equals(name, schema.name)
&& Objects.equals(timeColumnName, schema.timeColumnName)
&& Objects.equals(virtualColumns, schema.virtualColumns)
&& Objects.equals(groupingColumns, schema.groupingColumns)
&& Objects.deepEquals(aggregators, schema.aggregators)
&& Objects.equals(ordering, schema.ordering);
}
@Override
public int hashCode()
{
return Objects.hash(
name,
timeColumnName,
virtualColumns,
groupingColumns,
Arrays.hashCode(aggregators),
ordering
);
}
}
}

View File

@ -27,6 +27,7 @@ import org.apache.druid.common.guava.GuavaUtils;
import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Numbers;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.math.expr.Evals;
@ -360,8 +361,42 @@ public class AutoTypeColumnIndexer implements DimensionIndexer<StructuredData, S
return rootLiteralSelector;
}
return new ObjectColumnSelector<StructuredData>()
return new ColumnValueSelector<Object>()
{
@Override
public double getDouble()
{
Object o = StructuredData.unwrap(getObject());
return Numbers.tryParseDouble(o, 0.0);
}
@Override
public float getFloat()
{
Object o = StructuredData.unwrap(getObject());
return Numbers.tryParseFloat(o, 0.0f);
}
@Override
public long getLong()
{
Object o = StructuredData.unwrap(getObject());
return Numbers.tryParseLong(o, 0L);
}
@Override
public boolean isNull()
{
final Object o = StructuredData.unwrap(getObject());
if (o instanceof Number) {
return false;
}
if (o instanceof String) {
return GuavaUtils.tryParseLong((String) o) == null && Doubles.tryParse((String) o) == null;
}
return true;
}
@Override
public void inspectRuntimeShape(RuntimeShapeInspector inspector)
{

View File

@ -75,6 +75,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
SimpleDictionaryMergingIterator.makePeekingComparator();
private final String name;
private final String outputName;
private final IndexSpec indexSpec;
private final SegmentWriteOutMedium segmentWriteOutMedium;
private final Closer closer;
@ -84,9 +85,23 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
@Nullable
private final ColumnType castToType;
private boolean isVariantType = false;
private byte variantTypeByte = 0x00;
/**
* @param name column name
* @param outputName output smoosh file name. if this is a base table column, it will be the equivalent to
* name, however if this merger is for a projection, this will be prefixed with the
* projection name so that multiple projections can store the same column name at
* different smoosh file "paths"
* @param castToType optional mechanism to enforce that all values are a specific type
* @param indexSpec segment level storage options such as compression format and bitmap type
* @param segmentWriteOutMedium temporary storage location to stage segment outputs before finalizing into the segment
* @param closer resource closer if this merger needs to attach any closables that should be cleaned up
* when the segment is finished writing
*/
public AutoTypeColumnMerger(
String name,
String outputName,
@Nullable ColumnType castToType,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
@ -95,6 +110,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
{
this.name = name;
this.outputName = outputName;
this.castToType = castToType;
this.indexSpec = indexSpec;
this.segmentWriteOutMedium = segmentWriteOutMedium;
@ -165,7 +181,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
if (explicitType == null && !forceNested && ((isConstant && constantValue == null) || numMergeIndex == 0)) {
logicalType = ColumnType.STRING;
serializer = new ScalarStringColumnSerializer(
name,
outputName,
indexSpec,
segmentWriteOutMedium,
closer
@ -179,7 +195,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
switch (logicalType.getType()) {
case LONG:
serializer = new ScalarLongColumnSerializer(
name,
outputName,
indexSpec,
segmentWriteOutMedium,
closer
@ -187,7 +203,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
break;
case DOUBLE:
serializer = new ScalarDoubleColumnSerializer(
name,
outputName,
indexSpec,
segmentWriteOutMedium,
closer
@ -195,7 +211,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
break;
case STRING:
serializer = new ScalarStringColumnSerializer(
name,
outputName,
indexSpec,
segmentWriteOutMedium,
closer
@ -203,7 +219,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
break;
case ARRAY:
serializer = new VariantColumnSerializer(
name,
outputName,
logicalType,
null,
indexSpec,
@ -222,6 +238,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
// mixed type column, but only root path, we can use VariantArrayColumnSerializer
// pick the least restrictive type for the logical type
isVariantType = true;
variantTypeByte = rootTypes.getByteValue();
for (ColumnType type : FieldTypeInfo.convertToSet(rootTypes.getByteValue())) {
logicalType = ColumnType.leastRestrictiveType(logicalType, type);
}
@ -230,9 +247,9 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
logicalType = ColumnTypeFactory.getInstance().ofArray(logicalType);
}
serializer = new VariantColumnSerializer(
name,
outputName,
null,
rootTypes.getByteValue(),
variantTypeByte,
indexSpec,
segmentWriteOutMedium,
closer
@ -241,7 +258,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
// all the bells and whistles
logicalType = ColumnType.NESTED_DATA;
serializer = new NestedDataColumnSerializer(
name,
outputName,
indexSpec,
segmentWriteOutMedium,
closer
@ -262,7 +279,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
sortedLookup.getSortedDoubles(),
() -> new ArrayDictionaryMergingIterator(
sortedArrayLookups,
serializer.getGlobalLookup()
serializer.getDictionaryIdLookup()
)
);
stringCardinality = sortedLookup.getStringCardinality();
@ -284,7 +301,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
);
final ArrayDictionaryMergingIterator arrayIterator = new ArrayDictionaryMergingIterator(
sortedArrayLookups,
serializer.getGlobalLookup()
serializer.getDictionaryIdLookup()
);
serializer.serializeDictionaries(
() -> stringIterator,
@ -367,6 +384,76 @@ public class AutoTypeColumnMerger implements DimensionMergerV9
return descriptorBuilder.build();
}
protected DictionaryIdLookup getIdLookup()
{
return serializer.getDictionaryIdLookup();
}
@Override
public void attachParent(DimensionMergerV9 parent, List<IndexableAdapter> projectionAdapters) throws IOException
{
DruidException.conditionalDefensive(
parent instanceof AutoTypeColumnMerger,
"Projection parent dimension must be same type, got [%s]",
parent.getClass()
);
AutoTypeColumnMerger autoParent = (AutoTypeColumnMerger) parent;
logicalType = autoParent.logicalType;
isVariantType = autoParent.isVariantType;
if (autoParent.serializer instanceof ScalarStringColumnSerializer) {
serializer = new ScalarStringColumnSerializer(
outputName,
indexSpec,
segmentWriteOutMedium,
closer
);
} else if (autoParent.serializer instanceof ScalarLongColumnSerializer) {
serializer = new ScalarLongColumnSerializer(
outputName,
indexSpec,
segmentWriteOutMedium,
closer
);
} else if (autoParent.serializer instanceof ScalarDoubleColumnSerializer) {
serializer = new ScalarDoubleColumnSerializer(
outputName,
indexSpec,
segmentWriteOutMedium,
closer
);
} else if (autoParent.serializer instanceof VariantColumnSerializer) {
if (autoParent.isVariantType) {
serializer = new VariantColumnSerializer(
outputName,
null,
variantTypeByte,
indexSpec,
segmentWriteOutMedium,
closer
);
} else {
serializer = new VariantColumnSerializer(
outputName,
logicalType,
null,
indexSpec,
segmentWriteOutMedium,
closer
);
}
} else {
serializer = new NestedDataColumnSerializer(
outputName,
indexSpec,
segmentWriteOutMedium,
closer
);
}
serializer.setDictionaryIdLookup(autoParent.getIdLookup());
serializer.open();
}
public static class ArrayDictionaryMergingIterator implements Iterator<int[]>
{
private static final Comparator<PeekingIterator<int[]>> PEEKING_ITERATOR_COMPARATOR =

View File

@ -26,6 +26,7 @@ import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryMetrics;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.filter.Filter;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -54,10 +55,12 @@ public class CursorBuildSpec
private final VirtualColumns virtualColumns;
@Nullable
private final List<AggregatorFactory> aggregators;
private final List<OrderBy> orderByColumns;
private final List<OrderBy> preferredOrdering;
private final QueryContext queryContext;
private final boolean isAggregate;
@Nullable
private final QueryMetrics<?> queryMetrics;
@ -77,9 +80,10 @@ public class CursorBuildSpec
this.groupingColumns = groupingColumns;
this.virtualColumns = Preconditions.checkNotNull(virtualColumns, "virtualColumns");
this.aggregators = aggregators;
this.orderByColumns = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering");
this.preferredOrdering = Preconditions.checkNotNull(preferredOrdering, "preferredOrdering");
this.queryContext = Preconditions.checkNotNull(queryContext, "queryContext");
this.queryMetrics = queryMetrics;
this.isAggregate = !CollectionUtils.isNullOrEmpty(groupingColumns) || !CollectionUtils.isNullOrEmpty(aggregators);
}
/**
@ -142,7 +146,7 @@ public class CursorBuildSpec
*/
public List<OrderBy> getPreferredOrdering()
{
return orderByColumns;
return preferredOrdering;
}
/**
@ -166,6 +170,39 @@ public class CursorBuildSpec
return queryMetrics;
}
/**
* Returns true if {@link #getGroupingColumns()} is not null or empty and/or {@link #getAggregators()} is not null or
* empty. This method is useful for quickly checking if it is worth considering if a {@link CursorFactory} should
* attempt to produce a {@link CursorHolder} that is {@link CursorHolder#isPreAggregated()} to satisfy the build spec.
*/
public boolean isAggregate()
{
return isAggregate;
}
/**
* Returns true if the supplied ordering matches {@link #getPreferredOrdering()}, meaning that the supplied ordering
* has everything which is in the preferred ordering in the same direction and order. The supplied ordering may have
* additional columns beyond the preferred ordering and still satisify this method.
*/
public boolean isCompatibleOrdering(List<OrderBy> ordering)
{
// if the build spec doesn't prefer an ordering, any order is ok
if (preferredOrdering.isEmpty()) {
return true;
}
// all columns must be present in ordering if the build spec specifies them
if (ordering.size() < preferredOrdering.size()) {
return false;
}
for (int i = 0; i < preferredOrdering.size(); i++) {
if (!ordering.get(i).equals(preferredOrdering.get(i))) {
return false;
}
}
return true;
}
public static class CursorBuildSpecBuilder
{
@Nullable
@ -195,7 +232,7 @@ public class CursorBuildSpec
this.groupingColumns = buildSpec.groupingColumns;
this.virtualColumns = buildSpec.virtualColumns;
this.aggregators = buildSpec.aggregators;
this.preferredOrdering = buildSpec.orderByColumns;
this.preferredOrdering = buildSpec.preferredOrdering;
this.queryContext = buildSpec.queryContext;
this.queryMetrics = buildSpec.queryMetrics;
}

View File

@ -75,6 +75,18 @@ public interface CursorHolder extends Closeable
return false;
}
/**
* Returns a set of replacement {@link AggregatorFactory} if and only if {@link #isPreAggregated()} is true. The
* query engine should replace the query aggregators with these aggregators, which are combining aggregators derived
* from the {@link CursorBuildSpec} passed into {@link CursorFactory#makeCursorHolder(CursorBuildSpec)}. If
* {@link #isPreAggregated()} is not true, this method returns null
*/
@Nullable
default List<AggregatorFactory> getAggregatorsForPreAggregated()
{
return null;
}
/**
* Returns cursor ordering, which may or may not match {@link CursorBuildSpec#getPreferredOrdering()}. If returns
* an empty list then the cursor has no defined ordering.

View File

@ -70,6 +70,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
private static final Logger log = new Logger(DictionaryEncodedColumnMerger.class);
protected final String dimensionName;
protected final String outputName;
protected final ProgressIndicator progress;
protected final Closer closer;
protected final IndexSpec indexSpec;
@ -81,6 +82,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
protected int rowCount = 0;
protected int cardinality = 0;
protected boolean hasNull = false;
protected boolean writeDictionary = true;
@Nullable
protected GenericIndexedWriter<ImmutableBitmap> bitmapWriter;
@ -102,6 +104,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
public DictionaryEncodedColumnMerger(
String dimensionName,
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
@ -110,6 +113,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
)
{
this.dimensionName = dimensionName;
this.outputName = outputName;
this.indexSpec = indexSpec;
this.capabilities = capabilities;
this.segmentWriteOutMedium = segmentWriteOutMedium;
@ -171,8 +175,9 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
numMergeIndex++;
}
String dictFilename = StringUtils.format("%s.dim_values", dimensionName);
String dictFilename = StringUtils.format("%s.dim_values", outputName);
dictionaryWriter = makeDictionaryWriter(dictFilename);
firstDictionaryValue = null;
dictionarySize = 0;
dictionaryWriter.open();
@ -338,7 +343,7 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
long dimStartTime = System.currentTimeMillis();
final BitmapSerdeFactory bitmapSerdeFactory = indexSpec.getBitmapSerdeFactory();
String bmpFilename = StringUtils.format("%s.inverted", dimensionName);
String bmpFilename = StringUtils.format("%s.inverted", outputName);
bitmapWriter = new GenericIndexedWriter<>(
segmentWriteOutMedium,
bmpFilename,
@ -402,11 +407,11 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
{
final CompressionStrategy compressionStrategy = indexSpec.getDimensionCompression();
String filenameBase = StringUtils.format("%s.forward_dim", dimensionName);
String filenameBase = StringUtils.format("%s.forward_dim", outputName);
if (capabilities.hasMultipleValues().isTrue()) {
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
encodedValueSerializer = V3CompressedVSizeColumnarMultiIntsSerializer.create(
dimensionName,
outputName,
segmentWriteOutMedium,
filenameBase,
cardinality,
@ -414,12 +419,12 @@ public abstract class DictionaryEncodedColumnMerger<T extends Comparable<T>> imp
);
} else {
encodedValueSerializer =
new VSizeColumnarMultiIntsSerializer(dimensionName, segmentWriteOutMedium, cardinality);
new VSizeColumnarMultiIntsSerializer(outputName, segmentWriteOutMedium, cardinality);
}
} else {
if (compressionStrategy != CompressionStrategy.UNCOMPRESSED) {
encodedValueSerializer = CompressedVSizeColumnarIntsSerializer.create(
dimensionName,
outputName,
segmentWriteOutMedium,
filenameBase,
cardinality,

View File

@ -106,13 +106,17 @@ public interface DimensionHandler
*
* See {@link DimensionMergerV9} interface for more information.
*
* @param indexSpec Specification object for the index merge
* @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if needed
* @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler
* @param progress ProgressIndicator used by the merging process
* @param outputName Output "file" name for the column to use for serializers, to control where it is
* stored in the segments internal files
* @param indexSpec Specification object for the index merge
* @param segmentWriteOutMedium this SegmentWriteOutMedium object could be used internally in the created merger, if
* needed
* @param capabilities The ColumnCapabilities of the dimension represented by this DimensionHandler
* @param progress ProgressIndicator used by the merging process
* @return A new DimensionMergerV9 object.
*/
DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,

View File

@ -21,6 +21,9 @@ package org.apache.druid.segment;
import org.apache.druid.segment.column.ColumnDescriptor;
import java.io.IOException;
import java.util.List;
/**
* Processing related interface
*
@ -35,4 +38,13 @@ public interface DimensionMergerV9 extends DimensionMerger
* @return ColumnDescriptor that IndexMergerV9 will use to build a column.
*/
ColumnDescriptor makeColumnDescriptor();
/**
* Attaches the {@link DimensionMergerV9} of a "projection" parent column so that stuff like value dictionaries can
* be shared between parent and child
*/
default void attachParent(DimensionMergerV9 parent, List<IndexableAdapter> projectionAdapters) throws IOException
{
// do nothing
}
}

View File

@ -77,6 +77,7 @@ public class DoubleDimensionHandler implements DimensionHandler<Double, Double,
@Override
public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
@ -85,7 +86,7 @@ public class DoubleDimensionHandler implements DimensionHandler<Double, Double,
)
{
return new DoubleDimensionMergerV9(
dimensionName,
outputName,
indexSpec,
segmentWriteOutMedium
);

View File

@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
public class DoubleDimensionMergerV9 extends NumericDimensionMergerV9
{
DoubleDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
DoubleDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
{
super(dimensionName, indexSpec, segmentWriteOutMedium);
super(outputName, indexSpec, segmentWriteOutMedium);
}
@Override
GenericColumnSerializer setupEncodedValueWriter()
{
return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec);
return IndexMergerV9.createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
}
@Override

View File

@ -77,6 +77,7 @@ public class FloatDimensionHandler implements DimensionHandler<Float, Float, Flo
@Override
public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
@ -85,7 +86,7 @@ public class FloatDimensionHandler implements DimensionHandler<Float, Float, Flo
)
{
return new FloatDimensionMergerV9(
dimensionName,
outputName,
indexSpec,
segmentWriteOutMedium
);

View File

@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
public class FloatDimensionMergerV9 extends NumericDimensionMergerV9
{
FloatDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
FloatDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
{
super(dimensionName, indexSpec, segmentWriteOutMedium);
super(outputName, indexSpec, segmentWriteOutMedium);
}
@Override
GenericColumnSerializer setupEncodedValueWriter()
{
return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec);
return IndexMergerV9.createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
}
@Override

View File

@ -48,6 +48,7 @@ import org.apache.druid.java.util.common.io.smoosh.Smoosh;
import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.column.ColumnBuilder;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnConfig;
@ -64,6 +65,7 @@ import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.data.IndexedIterable;
import org.apache.druid.segment.data.ListIndexed;
import org.apache.druid.segment.data.VSizeColumnarMultiInts;
import org.apache.druid.segment.projections.Projections;
import org.apache.druid.segment.serde.ComplexColumnPartSupplier;
import org.apache.druid.segment.serde.FloatNumericColumnSupplier;
import org.apache.druid.segment.serde.LongNumericColumnSupplier;
@ -621,6 +623,7 @@ public class IndexIO
mapper,
timeBuffer,
smooshedFiles,
null,
loadFailed
);
@ -643,6 +646,22 @@ public class IndexIO
smooshedFiles,
loadFailed
);
final Map<String, Map<String, Supplier<ColumnHolder>>> projectionsColumns = new LinkedHashMap<>();
final Metadata metadata = getMetdata(smooshedFiles, mapper, inDir);
if (metadata != null && metadata.getProjections() != null) {
for (AggregateProjectionMetadata projectionSpec : metadata.getProjections()) {
final Map<String, Supplier<ColumnHolder>> projectionColumns = readProjectionColumns(
mapper,
loadFailed,
projectionSpec,
smooshedFiles,
columns,
dataInterval
);
projectionsColumns.put(projectionSpec.getSchema().getName(), projectionColumns);
}
}
final QueryableIndex index = new SimpleQueryableIndex(
dataInterval,
@ -650,30 +669,15 @@ public class IndexIO
segmentBitmapSerdeFactory.getBitmapFactory(),
columns,
smooshedFiles,
lazy
lazy,
metadata,
projectionsColumns
)
{
@Override
public Metadata getMetadata()
{
try {
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
if (metadataBB != null) {
return mapper.readValue(
SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
Metadata.class
);
}
}
catch (JsonParseException | JsonMappingException ex) {
// Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which
// is no longer supported then it is OK to not use the metadata instead of failing segment loading
log.warn(ex, "Failed to load metadata for segment [%s]", inDir);
}
catch (IOException ex) {
log.warn(ex, "Failed to read metadata for segment [%s]", inDir);
}
return null;
return getMetdata(smooshedFiles, mapper, inDir);
}
};
@ -682,6 +686,87 @@ public class IndexIO
return index;
}
private Map<String, Supplier<ColumnHolder>> readProjectionColumns(
ObjectMapper mapper,
SegmentLazyLoadFailCallback loadFailed,
AggregateProjectionMetadata projectionSpec,
SmooshedFileMapper smooshedFiles,
Map<String, Supplier<ColumnHolder>> columns,
Interval dataInterval
) throws IOException
{
final Map<String, Supplier<ColumnHolder>> projectionColumns = new LinkedHashMap<>();
for (String groupingColumn : projectionSpec.getSchema().getGroupingColumns()) {
final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, groupingColumn);
final ByteBuffer colBuffer = smooshedFiles.mapFile(smooshName);
final ColumnHolder parentColumn;
if (columns.containsKey(groupingColumn)) {
parentColumn = columns.get(groupingColumn).get();
} else {
parentColumn = null;
}
registerColumnHolder(
true,
projectionColumns,
groupingColumn,
mapper,
colBuffer,
smooshedFiles,
parentColumn,
loadFailed
);
if (groupingColumn.equals(projectionSpec.getSchema().getTimeColumnName())) {
projectionColumns.put(ColumnHolder.TIME_COLUMN_NAME, projectionColumns.get(groupingColumn));
}
}
for (AggregatorFactory aggregator : projectionSpec.getSchema().getAggregators()) {
final String smooshName = Projections.getProjectionSmooshV9FileName(projectionSpec, aggregator.getName());
final ByteBuffer aggBuffer = smooshedFiles.mapFile(smooshName);
registerColumnHolder(
true,
projectionColumns,
aggregator.getName(),
mapper,
aggBuffer,
smooshedFiles,
null,
loadFailed
);
}
if (projectionSpec.getSchema().getTimeColumnName() == null) {
projectionColumns.put(
ColumnHolder.TIME_COLUMN_NAME,
Projections.makeConstantTimeSupplier(projectionSpec.getNumRows(), dataInterval.getStartMillis())
);
}
return projectionColumns;
}
@Nullable
private Metadata getMetdata(SmooshedFileMapper smooshedFiles, ObjectMapper mapper, File inDir)
{
try {
ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd");
if (metadataBB != null) {
return mapper.readValue(
SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()),
Metadata.class
);
}
}
catch (JsonParseException | JsonMappingException ex) {
// Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which
// is no longer supported then it is OK to not use the metadata instead of failing segment loading
log.warn(ex, "Failed to load metadata for segment [%s]", inDir);
}
catch (IOException ex) {
log.warn(ex, "Failed to read metadata for segment [%s]", inDir);
}
return null;
}
/**
* Return a list of columns that contains given inputs merged. The returned column names are in
* the original order that is used when this segment is created.
@ -740,6 +825,7 @@ public class IndexIO
mapper,
colBuffer,
smooshedFiles,
null,
loadFailed
);
}
@ -752,6 +838,7 @@ public class IndexIO
ObjectMapper mapper,
ByteBuffer colBuffer,
SmooshedFileMapper smooshedFiles,
@Nullable ColumnHolder parentColumn,
SegmentLazyLoadFailCallback loadFailed
) throws IOException
{
@ -767,7 +854,8 @@ public class IndexIO
internedColumnName,
mapper,
colBuffer,
smooshedFiles
smooshedFiles,
parentColumn
);
}
catch (IOException | RuntimeException e) {
@ -782,7 +870,8 @@ public class IndexIO
internedColumnName,
mapper,
colBuffer,
smooshedFiles
smooshedFiles,
parentColumn
);
columns.put(internedColumnName, () -> columnHolder);
}
@ -797,11 +886,12 @@ public class IndexIO
String columnName, // columnName is not used in this method, but used in tests.
ObjectMapper mapper,
ByteBuffer byteBuffer,
SmooshedFileMapper smooshedFiles
SmooshedFileMapper smooshedFiles,
@Nullable ColumnHolder parentColumn
) throws IOException
{
ColumnDescriptor serde = mapper.readValue(SERIALIZER_UTILS.readString(byteBuffer), ColumnDescriptor.class);
return serde.read(byteBuffer, columnConfig, smooshedFiles);
return serde.read(byteBuffer, columnConfig, smooshedFiles, parentColumn);
}
}

View File

@ -354,7 +354,7 @@ public interface IndexMerger
// type of column doesn't have any kind of special per-index encoding that needs to be converted to the "global"
// encoding. E. g. it's always true for subclasses of NumericDimensionMergerV9.
//noinspection ObjectEquality
anySelectorChanged |= convertedDimensionSelector != sourceDimensionSelector;
anySelectorChanged = anySelectorChanged || convertedDimensionSelector != sourceDimensionSelector;
convertedMarkedDimensionSelectors[i] = mergers.get(i).convertSortedSegmentRowValuesToMergedRowValues(
indexNumber,

View File

@ -23,6 +23,7 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import com.google.common.io.Files;
import com.google.common.primitives.Ints;
import com.google.inject.Inject;
@ -51,6 +52,7 @@ import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexAdapter;
import org.apache.druid.segment.loading.MMappedQueryableSegmentizerFactory;
import org.apache.druid.segment.loading.SegmentizerFactory;
import org.apache.druid.segment.projections.Projections;
import org.apache.druid.segment.serde.ColumnPartSerde;
import org.apache.druid.segment.serde.ComplexColumnPartSerde;
import org.apache.druid.segment.serde.ComplexMetricSerde;
@ -64,6 +66,7 @@ import org.apache.druid.segment.serde.LongNumericColumnPartSerdeV2;
import org.apache.druid.segment.serde.NullColumnPartSerde;
import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
import org.apache.druid.segment.writeout.SegmentWriteOutMediumFactory;
import org.apache.druid.utils.CollectionUtils;
import org.joda.time.DateTime;
import org.joda.time.Interval;
@ -138,7 +141,6 @@ public class IndexMergerV9 implements IndexMerger
final DimensionsSpecInspector dimensionsSpecInspector,
final List<String> mergedMetrics,
final Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn,
final boolean fillRowNumConversions,
final IndexSpec indexSpec,
final @Nullable SegmentWriteOutMediumFactory segmentWriteOutMediumFactory
) throws IOException
@ -154,7 +156,7 @@ public class IndexMergerV9 implements IndexMerger
.filter(dim -> !ColumnHolder.TIME_COLUMN_NAME.equals(dim))
.collect(Collectors.toList());
final Metadata segmentMetadata;
Metadata segmentMetadata;
if (metricAggs != null) {
AggregatorFactory[] combiningMetricAggs = new AggregatorFactory[metricAggs.length];
for (int i = 0; i < metricAggs.length; i++) {
@ -215,18 +217,20 @@ public class IndexMergerV9 implements IndexMerger
mergeFormat(adapters, mergedDimensions, metricFormats, dimFormats);
final Map<String, DimensionHandler> handlers = makeDimensionHandlers(mergedDimensions, dimFormats);
final Map<String, DimensionMergerV9> mergersMap = Maps.newHashMapWithExpectedSize(mergedDimensions.size());
final List<DimensionMergerV9> mergers = new ArrayList<>();
for (int i = 0; i < mergedDimensions.size(); i++) {
DimensionHandler handler = handlers.get(mergedDimensions.get(i));
mergers.add(
handler.makeMerger(
indexSpec,
segmentWriteOutMedium,
dimFormats.get(i).toColumnCapabilities(),
progress,
closer
)
DimensionMergerV9 merger = handler.makeMerger(
mergedDimensions.get(i),
indexSpec,
segmentWriteOutMedium,
dimFormats.get(i).toColumnCapabilities(),
progress,
closer
);
mergers.add(merger);
mergersMap.put(mergedDimensions.get(i), merger);
}
/************* Setup Dim Conversions **************/
@ -255,8 +259,7 @@ public class IndexMergerV9 implements IndexMerger
timeAndDimsIterator,
timeWriter,
metricWriters,
mergers,
fillRowNumConversions
mergers
);
/************ Create Inverted Indexes and Finalize Build Columns *************/
@ -291,6 +294,20 @@ public class IndexMergerV9 implements IndexMerger
progress.stopSection(section);
if (segmentMetadata != null && !CollectionUtils.isNullOrEmpty(segmentMetadata.getProjections())) {
segmentMetadata = makeProjections(
v9Smoosher,
segmentMetadata.getProjections(),
adapters,
indexSpec,
segmentWriteOutMedium,
progress,
closer,
mergersMap,
segmentMetadata
);
}
/************* Make index.drd & metadata.drd files **************/
progress.progress();
makeIndexBinary(
@ -332,6 +349,193 @@ public class IndexMergerV9 implements IndexMerger
}
}
private Metadata makeProjections(
final FileSmoosher smoosher,
final List<AggregateProjectionMetadata> projections,
final List<IndexableAdapter> adapters,
final IndexSpec indexSpec,
final SegmentWriteOutMedium segmentWriteOutMedium,
final ProgressIndicator progress,
final Closer closer,
final Map<String, DimensionMergerV9> parentMergers,
final Metadata segmentMetadata
) throws IOException
{
final List<AggregateProjectionMetadata> projectionMetadata = Lists.newArrayListWithCapacity(projections.size());
for (AggregateProjectionMetadata spec : projections) {
final List<IndexableAdapter> projectionAdapters = Lists.newArrayListWithCapacity(adapters.size());
final AggregateProjectionMetadata.Schema projectionSchema = spec.getSchema();
for (IndexableAdapter adapter : adapters) {
projectionAdapters.add(adapter.getProjectionAdapter(projectionSchema.getName()));
}
// we can use the first adapter to get the dimensions and metrics because the projection schema should be
// identical across all segments. This is validated by segment metadata merging
final List<String> dimensions = projectionAdapters.get(0).getDimensionNames(false);
final List<String> metrics = Arrays.stream(projectionSchema.getAggregators())
.map(AggregatorFactory::getName)
.collect(Collectors.toList());
final List<DimensionMergerV9> mergers = new ArrayList<>();
final Map<String, ColumnFormat> columnFormats = Maps.newLinkedHashMapWithExpectedSize(dimensions.size() + metrics.size());
for (String dimension : dimensions) {
final ColumnFormat dimensionFormat = projectionAdapters.get(0).getFormat(dimension);
columnFormats.put(dimension, dimensionFormat);
DimensionHandler handler = dimensionFormat.getColumnHandler(dimension);
DimensionMergerV9 merger = handler.makeMerger(
Projections.getProjectionSmooshV9FileName(spec, dimension),
indexSpec,
segmentWriteOutMedium,
dimensionFormat.toColumnCapabilities(),
progress,
closer
);
if (parentMergers.containsKey(dimension)) {
merger.attachParent(parentMergers.get(dimension), projectionAdapters);
} else {
merger.writeMergedValueDictionary(projectionAdapters);
}
mergers.add(merger);
}
for (String metric : metrics) {
columnFormats.put(metric, projectionAdapters.get(0).getFormat(metric));
}
final GenericColumnSerializer timeWriter;
if (projectionSchema.getTimeColumnName() != null) {
timeWriter = setupTimeWriter(segmentWriteOutMedium, indexSpec);
} else {
timeWriter = null;
}
final ArrayList<GenericColumnSerializer> metricWriters =
setupMetricsWriters(
segmentWriteOutMedium,
metrics,
columnFormats,
indexSpec,
Projections.getProjectionSmooshV9Prefix(spec)
);
Function<List<TransformableRowIterator>, TimeAndDimsIterator> rowMergerFn =
rowIterators -> new RowCombiningTimeAndDimsIterator(rowIterators, projectionSchema.getAggregators(), metrics);
List<TransformableRowIterator> perIndexRowIterators = Lists.newArrayListWithCapacity(projectionAdapters.size());
for (int i = 0; i < projectionAdapters.size(); ++i) {
final IndexableAdapter adapter = projectionAdapters.get(i);
TransformableRowIterator target = adapter.getRows();
perIndexRowIterators.add(IndexMerger.toMergedIndexRowIterator(target, i, mergers));
}
final TimeAndDimsIterator timeAndDimsIterator = rowMergerFn.apply(perIndexRowIterators);
closer.register(timeAndDimsIterator);
int rowCount = 0;
List<IntBuffer> rowNumConversions = new ArrayList<>(projectionAdapters.size());
for (IndexableAdapter adapter : projectionAdapters) {
int[] arr = new int[adapter.getNumRows()];
Arrays.fill(arr, INVALID_ROW);
rowNumConversions.add(IntBuffer.wrap(arr));
}
final String section = "walk through and merge projection[" + projectionSchema.getName() + "] rows";
progress.startSection(section);
long startTime = System.currentTimeMillis();
long time = startTime;
while (timeAndDimsIterator.moveToNext()) {
progress.progress();
TimeAndDimsPointer timeAndDims = timeAndDimsIterator.getPointer();
if (timeWriter != null) {
timeWriter.serialize(timeAndDims.timestampSelector);
}
for (int metricIndex = 0; metricIndex < timeAndDims.getNumMetrics(); metricIndex++) {
metricWriters.get(metricIndex).serialize(timeAndDims.getMetricSelector(metricIndex));
}
for (int dimIndex = 0; dimIndex < timeAndDims.getNumDimensions(); dimIndex++) {
DimensionMergerV9 merger = mergers.get(dimIndex);
if (merger.hasOnlyNulls()) {
continue;
}
merger.processMergedRow(timeAndDims.getDimensionSelector(dimIndex));
}
RowCombiningTimeAndDimsIterator comprisedRows = (RowCombiningTimeAndDimsIterator) timeAndDimsIterator;
for (int originalIteratorIndex = comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(0);
originalIteratorIndex >= 0;
originalIteratorIndex =
comprisedRows.nextCurrentlyCombinedOriginalIteratorIndex(originalIteratorIndex + 1)) {
IntBuffer conversionBuffer = rowNumConversions.get(originalIteratorIndex);
int minRowNum = comprisedRows.getMinCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex);
int maxRowNum = comprisedRows.getMaxCurrentlyCombinedRowNumByOriginalIteratorIndex(originalIteratorIndex);
for (int rowNum = minRowNum; rowNum <= maxRowNum; rowNum++) {
while (conversionBuffer.position() < rowNum) {
conversionBuffer.put(INVALID_ROW);
}
conversionBuffer.put(rowCount);
}
}
if ((++rowCount % 500000) == 0) {
log.debug(
"walked 500,000/%d rows of projection[%s] in %,d millis.",
rowCount,
projectionSchema.getName(),
System.currentTimeMillis() - time
);
time = System.currentTimeMillis();
}
}
for (IntBuffer rowNumConversion : rowNumConversions) {
rowNumConversion.rewind();
}
log.debug(
"completed walk through of %,d rows of projection[%s] in %,d millis.",
rowCount,
projectionSchema.getName(),
System.currentTimeMillis() - startTime
);
progress.stopSection(section);
final String section2 = "build projection[" + projectionSchema.getName() + "] inverted index and columns";
progress.startSection(section2);
if (projectionSchema.getTimeColumnName() != null) {
makeTimeColumn(
smoosher,
progress,
timeWriter,
indexSpec,
Projections.getProjectionSmooshV9FileName(spec, projectionSchema.getTimeColumnName())
);
}
makeMetricsColumns(
smoosher,
progress,
metrics,
columnFormats,
metricWriters,
indexSpec,
Projections.getProjectionSmooshV9Prefix(spec)
);
for (int i = 0; i < dimensions.size(); i++) {
final String dimension = dimensions.get(i);
DimensionMergerV9 merger = mergers.get(i);
merger.writeIndexes(rowNumConversions);
if (!merger.hasOnlyNulls()) {
ColumnDescriptor columnDesc = merger.makeColumnDescriptor();
makeColumn(smoosher, Projections.getProjectionSmooshV9FileName(spec, dimension), columnDesc);
}
}
progress.stopSection(section2);
projectionMetadata.add(new AggregateProjectionMetadata(projectionSchema, rowCount));
}
return segmentMetadata.withProjections(projectionMetadata);
}
private void makeIndexBinary(
final FileSmoosher v9Smoosher,
final List<IndexableAdapter> adapters,
@ -348,7 +552,7 @@ public class IndexMergerV9 implements IndexMerger
columnSet.addAll(mergedMetrics);
Preconditions.checkState(
columnSet.size() == mergedDimensions.size() + mergedMetrics.size(),
"column names are not unique in dims%s and mets%s",
"column names are not unique in dims[%s] and mets[%s]",
mergedDimensions,
mergedMetrics
);
@ -444,6 +648,18 @@ public class IndexMergerV9 implements IndexMerger
final List<GenericColumnSerializer> metWriters,
final IndexSpec indexSpec
) throws IOException
{
makeMetricsColumns(v9Smoosher, progress, mergedMetrics, metricsTypes, metWriters, indexSpec, "");
}
private void makeMetricsColumns(
final FileSmoosher v9Smoosher,
final ProgressIndicator progress,
final List<String> mergedMetrics,
final Map<String, ColumnFormat> metricsTypes,
final List<GenericColumnSerializer> metWriters,
final IndexSpec indexSpec,
final String namePrefix
) throws IOException
{
final String section = "make metric columns";
progress.startSection(section);
@ -483,8 +699,9 @@ public class IndexMergerV9 implements IndexMerger
default:
throw new ISE("Unknown type[%s]", type);
}
makeColumn(v9Smoosher, metric, builder.build());
log.debug("Completed metric column[%s] in %,d millis.", metric, System.currentTimeMillis() - metricStartTime);
final String columnName = namePrefix + metric;
makeColumn(v9Smoosher, columnName, builder.build());
log.debug("Completed metric column[%s] in %,d millis.", columnName, System.currentTimeMillis() - metricStartTime);
}
log.debug("Completed metric columns in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section);
@ -547,6 +764,17 @@ public class IndexMergerV9 implements IndexMerger
final GenericColumnSerializer timeWriter,
final IndexSpec indexSpec
) throws IOException
{
makeTimeColumn(v9Smoosher, progress, timeWriter, indexSpec, ColumnHolder.TIME_COLUMN_NAME);
}
private void makeTimeColumn(
final FileSmoosher v9Smoosher,
final ProgressIndicator progress,
final GenericColumnSerializer timeWriter,
final IndexSpec indexSpec,
final String name
) throws IOException
{
final String section = "make time column";
progress.startSection(section);
@ -557,7 +785,7 @@ public class IndexMergerV9 implements IndexMerger
.setValueType(ValueType.LONG)
.addSerde(createLongColumnPartSerde(timeWriter, indexSpec))
.build();
makeColumn(v9Smoosher, ColumnHolder.TIME_COLUMN_NAME, serdeficator);
makeColumn(v9Smoosher, name, serdeficator);
log.debug("Completed time column in %,d millis.", System.currentTimeMillis() - startTime);
progress.stopSection(section);
}
@ -601,23 +829,19 @@ public class IndexMergerV9 implements IndexMerger
final TimeAndDimsIterator timeAndDimsIterator,
final GenericColumnSerializer timeWriter,
final ArrayList<GenericColumnSerializer> metricWriters,
final List<DimensionMergerV9> mergers,
final boolean fillRowNumConversions
final List<DimensionMergerV9> mergers
) throws IOException
{
final String section = "walk through and merge rows";
progress.startSection(section);
long startTime = System.currentTimeMillis();
List<IntBuffer> rowNumConversions = null;
int rowCount = 0;
if (fillRowNumConversions) {
rowNumConversions = new ArrayList<>(adapters.size());
for (IndexableAdapter adapter : adapters) {
int[] arr = new int[adapter.getNumRows()];
Arrays.fill(arr, INVALID_ROW);
rowNumConversions.add(IntBuffer.wrap(arr));
}
List<IntBuffer> rowNumConversions = new ArrayList<>(adapters.size());
for (IndexableAdapter adapter : adapters) {
int[] arr = new int[adapter.getNumRows()];
Arrays.fill(arr, INVALID_ROW);
rowNumConversions.add(IntBuffer.wrap(arr));
}
long time = System.currentTimeMillis();
@ -656,9 +880,7 @@ public class IndexMergerV9 implements IndexMerger
}
conversionBuffer.put(rowCount);
}
}
} else if (timeAndDimsIterator instanceof MergingRowIterator) {
RowPointer rowPointer = (RowPointer) timeAndDims;
IntBuffer conversionBuffer = rowNumConversions.get(rowPointer.getIndexNum());
@ -668,11 +890,9 @@ public class IndexMergerV9 implements IndexMerger
}
conversionBuffer.put(rowCount);
} else {
if (fillRowNumConversions) {
throw new IllegalStateException(
"Filling row num conversions is supported only with RowCombining and Merging iterators"
);
}
throw new IllegalStateException(
"Filling row num conversions is supported only with RowCombining and Merging iterators"
);
}
if ((++rowCount % 500000) == 0) {
@ -680,10 +900,8 @@ public class IndexMergerV9 implements IndexMerger
time = System.currentTimeMillis();
}
}
if (rowNumConversions != null) {
for (IntBuffer rowNumConversion : rowNumConversions) {
rowNumConversion.rewind();
}
for (IntBuffer rowNumConversion : rowNumConversions) {
rowNumConversion.rewind();
}
log.debug("completed walk through of %,d rows in %,d millis.", rowCount, System.currentTimeMillis() - startTime);
progress.stopSection(section);
@ -711,28 +929,40 @@ public class IndexMergerV9 implements IndexMerger
final Map<String, ColumnFormat> metricsTypes,
final IndexSpec indexSpec
) throws IOException
{
return setupMetricsWriters(segmentWriteOutMedium, mergedMetrics, metricsTypes, indexSpec, "");
}
private ArrayList<GenericColumnSerializer> setupMetricsWriters(
final SegmentWriteOutMedium segmentWriteOutMedium,
final List<String> mergedMetrics,
final Map<String, ColumnFormat> metricsTypes,
final IndexSpec indexSpec,
final String prefix
) throws IOException
{
ArrayList<GenericColumnSerializer> metWriters = Lists.newArrayListWithCapacity(mergedMetrics.size());
for (String metric : mergedMetrics) {
TypeSignature<ValueType> type = metricsTypes.get(metric).getLogicalType();
final String outputName = prefix + metric;
GenericColumnSerializer writer;
switch (type.getType()) {
case LONG:
writer = createLongColumnSerializer(segmentWriteOutMedium, metric, indexSpec);
writer = createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
break;
case FLOAT:
writer = createFloatColumnSerializer(segmentWriteOutMedium, metric, indexSpec);
writer = createFloatColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
break;
case DOUBLE:
writer = createDoubleColumnSerializer(segmentWriteOutMedium, metric, indexSpec);
writer = createDoubleColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
break;
case COMPLEX:
ComplexMetricSerde serde = ComplexMetrics.getSerdeForType(type.getComplexTypeName());
if (serde == null) {
throw new ISE("Unknown type[%s]", type.getComplexTypeName());
}
writer = serde.getSerializer(segmentWriteOutMedium, metric, indexSpec);
writer = serde.getSerializer(segmentWriteOutMedium, outputName, indexSpec);
break;
default:
throw new ISE("Unknown type[%s]", type);
@ -891,7 +1121,7 @@ public class IndexMergerV9 implements IndexMerger
FileUtils.mkdirp(outDir);
log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.size());
log.debug("Starting persist for interval[%s], rows[%,d]", dataInterval, index.numRows());
return multiphaseMerge(
Collections.singletonList(
new IncrementalIndexAdapter(
@ -1179,7 +1409,6 @@ public class IndexMergerV9 implements IndexMerger
new DimensionsSpecInspector(storeEmptyColumns, dimensionsSpec),
mergedMetrics,
rowMergerFn,
true,
indexSpec,
segmentWriteOutMediumFactory
);

View File

@ -64,6 +64,8 @@ public interface IndexableAdapter
TransformableRowIterator getRows();
IndexableAdapter getProjectionAdapter(String projection);
BitmapValues getBitmapValues(String dimension, int dictId);
ColumnCapabilities getCapabilities(String column);

View File

@ -77,6 +77,7 @@ public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
@Override
public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
@ -85,7 +86,7 @@ public class LongDimensionHandler implements DimensionHandler<Long, Long, Long>
)
{
return new LongDimensionMergerV9(
dimensionName,
outputName,
indexSpec,
segmentWriteOutMedium
);

View File

@ -27,15 +27,15 @@ import org.apache.druid.segment.writeout.SegmentWriteOutMedium;
public class LongDimensionMergerV9 extends NumericDimensionMergerV9
{
LongDimensionMergerV9(String dimensionName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
LongDimensionMergerV9(String outputName, IndexSpec indexSpec, SegmentWriteOutMedium segmentWriteOutMedium)
{
super(dimensionName, indexSpec, segmentWriteOutMedium);
super(outputName, indexSpec, segmentWriteOutMedium);
}
@Override
GenericColumnSerializer setupEncodedValueWriter()
{
return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, dimensionName, indexSpec);
return IndexMergerV9.createLongColumnSerializer(segmentWriteOutMedium, outputName, indexSpec);
}
@Override

View File

@ -22,6 +22,7 @@ package org.apache.druid.segment;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.error.DruidException;
import org.apache.druid.guice.annotations.PublicApi;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.granularity.Granularity;
@ -56,6 +57,8 @@ public class Metadata
private final Boolean rollup;
@Nullable
private final List<OrderBy> ordering;
@Nullable
private final List<AggregateProjectionMetadata> projections;
public Metadata(
@JsonProperty("container") @Nullable Map<String, Object> container,
@ -63,7 +66,8 @@ public class Metadata
@JsonProperty("timestampSpec") @Nullable TimestampSpec timestampSpec,
@JsonProperty("queryGranularity") @Nullable Granularity queryGranularity,
@JsonProperty("rollup") @Nullable Boolean rollup,
@JsonProperty("ordering") @Nullable List<OrderBy> ordering
@JsonProperty("ordering") @Nullable List<OrderBy> ordering,
@JsonProperty("projections") @Nullable List<AggregateProjectionMetadata> projections
)
{
this.container = container == null ? new ConcurrentHashMap<>() : container;
@ -72,6 +76,7 @@ public class Metadata
this.queryGranularity = queryGranularity;
this.rollup = rollup;
this.ordering = ordering;
this.projections = projections;
}
@JsonProperty
@ -124,6 +129,27 @@ public class Metadata
return ordering;
}
@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_EMPTY)
public List<AggregateProjectionMetadata> getProjections()
{
return projections;
}
public Metadata withProjections(List<AggregateProjectionMetadata> projections)
{
return new Metadata(
container,
aggregators,
timestampSpec,
queryGranularity,
rollup,
ordering,
projections
);
}
public Metadata putAll(@Nullable Map<String, Object> other)
{
if (other != null) {
@ -155,6 +181,7 @@ public class Metadata
List<Granularity> gransToMerge = new ArrayList<>();
List<Boolean> rollupToMerge = new ArrayList<>();
List<List<OrderBy>> orderingsToMerge = new ArrayList<>();
List<List<AggregateProjectionMetadata>> projectionsToMerge = new ArrayList<>();
for (Metadata metadata : toBeMerged) {
if (metadata != null) {
@ -176,6 +203,7 @@ public class Metadata
}
orderingsToMerge.add(metadata.getOrdering());
projectionsToMerge.add(metadata.getProjections());
mergedContainer.putAll(metadata.container);
} else {
//if metadata and hence aggregators and queryGranularity for some segment being merged are unknown then
@ -204,6 +232,7 @@ public class Metadata
Granularity.mergeGranularities(gransToMerge);
final List<OrderBy> mergedOrdering = mergeOrderings(orderingsToMerge);
validateProjections(projectionsToMerge);
Boolean rollup = null;
if (rollupToMerge != null && !rollupToMerge.isEmpty()) {
@ -227,7 +256,8 @@ public class Metadata
mergedTimestampSpec,
mergedGranularity,
rollup,
mergedOrdering
mergedOrdering,
projectionsToMerge.get(0) // we're going to replace this later with updated rowcount
);
}
@ -246,13 +276,14 @@ public class Metadata
Objects.equals(timestampSpec, metadata.timestampSpec) &&
Objects.equals(queryGranularity, metadata.queryGranularity) &&
Objects.equals(rollup, metadata.rollup) &&
Objects.equals(ordering, metadata.ordering);
Objects.equals(ordering, metadata.ordering) &&
Objects.equals(projections, metadata.projections);
}
@Override
public int hashCode()
{
return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup);
return Objects.hash(container, Arrays.hashCode(aggregators), timestampSpec, queryGranularity, rollup, ordering, projections);
}
@Override
@ -265,6 +296,7 @@ public class Metadata
", queryGranularity=" + queryGranularity +
", rollup=" + rollup +
", ordering=" + ordering +
", projections=" + projections +
'}';
}
@ -308,4 +340,34 @@ public class Metadata
mergedOrdering.add(orderBy);
}
}
public static void validateProjections(List<List<AggregateProjectionMetadata>> projectionsToMerge)
{
final Map<String, AggregateProjectionMetadata> projectionsMap = new HashMap<>();
// dedupe by name, fail if somehow incompatible projections are defined
int nullCount = 0;
int expectedSize = -1;
for (List<AggregateProjectionMetadata> projections : projectionsToMerge) {
if (projections == null) {
nullCount++;
continue;
}
if (expectedSize < 0) {
expectedSize = projections.size();
} else if (projections.size() != expectedSize) {
throw DruidException.defensive("Unable to merge projections: mismatched projections count");
}
for (AggregateProjectionMetadata projection : projections) {
AggregateProjectionMetadata prev = projectionsMap.putIfAbsent(projection.getSchema().getName(), projection);
if (prev != null && !prev.getSchema().equals(projection.getSchema())) {
throw DruidException.defensive("Unable to merge projections: mismatched projections [%s] and [%s]", prev, projection);
}
}
}
if (nullCount > 0) {
if (nullCount != projectionsToMerge.size()) {
throw DruidException.defensive("Unable to merge projections: some projections were null");
}
}
}
}

View File

@ -77,6 +77,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler<Structu
@Override
public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
@ -84,7 +85,7 @@ public class NestedCommonFormatColumnHandler implements DimensionHandler<Structu
Closer closer
)
{
return new AutoTypeColumnMerger(name, castTo, indexSpec, segmentWriteOutMedium, closer);
return new AutoTypeColumnMerger(name, outputName, castTo, indexSpec, segmentWriteOutMedium, closer);
}
@Override

View File

@ -73,6 +73,7 @@ public class NestedDataColumnHandlerV4 implements DimensionHandler<StructuredDat
@Override
public DimensionMergerV9 makeMerger(
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium,
ColumnCapabilities capabilities,
@ -80,7 +81,7 @@ public class NestedDataColumnHandlerV4 implements DimensionHandler<StructuredDat
Closer closer
)
{
return new NestedDataColumnMergerV4(name, indexSpec, segmentWriteOutMedium, closer);
return new NestedDataColumnMergerV4(outputName, indexSpec, segmentWriteOutMedium, closer);
}
@Override

View File

@ -31,19 +31,19 @@ import java.util.List;
*/
public abstract class NumericDimensionMergerV9 implements DimensionMergerV9
{
protected final String dimensionName;
protected final String outputName;
protected final IndexSpec indexSpec;
protected final SegmentWriteOutMedium segmentWriteOutMedium;
protected final GenericColumnSerializer serializer;
NumericDimensionMergerV9(
String dimensionName,
String outputName,
IndexSpec indexSpec,
SegmentWriteOutMedium segmentWriteOutMedium
)
{
this.dimensionName = dimensionName;
this.outputName = outputName;
this.indexSpec = indexSpec;
this.segmentWriteOutMedium = segmentWriteOutMedium;

View File

@ -24,6 +24,7 @@ import org.apache.druid.query.OrderBy;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.Indexed;
import org.apache.druid.segment.projections.QueryableProjection;
import org.joda.time.Interval;
import javax.annotation.Nullable;
@ -85,4 +86,16 @@ public interface QueryableIndex extends Closeable, ColumnInspector
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed
@Override
void close();
@Nullable
default QueryableProjection<QueryableIndex> getProjection(CursorBuildSpec cursorBuildSpec)
{
return null;
}
@Nullable
default QueryableIndex getProjectionQueryableIndex(String name)
{
return null;
}
}

View File

@ -20,13 +20,19 @@
package org.apache.druid.segment;
import org.apache.druid.query.OrderBy;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.column.ColumnCapabilities;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.data.Offset;
import org.apache.druid.segment.projections.QueryableProjection;
import org.apache.druid.segment.vector.VectorColumnSelectorFactory;
import org.apache.druid.segment.vector.VectorOffset;
import javax.annotation.Nullable;
import java.util.LinkedHashSet;
import java.util.List;
public class QueryableIndexCursorFactory implements CursorFactory
{
@ -40,6 +46,46 @@ public class QueryableIndexCursorFactory implements CursorFactory
@Override
public CursorHolder makeCursorHolder(CursorBuildSpec spec)
{
QueryableProjection<QueryableIndex> projection = index.getProjection(spec);
if (projection != null) {
return new QueryableIndexCursorHolder(projection.getRowSelector(), projection.getCursorBuildSpec())
{
@Override
protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset(
ColumnCache columnCache,
Offset baseOffset
)
{
return projection.wrapColumnSelectorFactory(
super.makeColumnSelectorFactoryForOffset(columnCache, baseOffset)
);
}
@Override
protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset(
ColumnCache columnCache,
VectorOffset baseOffset
)
{
return projection.wrapVectorColumnSelectorFactory(
super.makeVectorColumnSelectorFactoryForOffset(columnCache, baseOffset)
);
}
@Override
public boolean isPreAggregated()
{
return true;
}
@Nullable
@Override
public List<AggregatorFactory> getAggregatorsForPreAggregated()
{
return projection.getCursorBuildSpec().getAggregators();
}
};
}
return new QueryableIndexCursorHolder(index, CursorBuildSpec.builder(spec).build());
}

View File

@ -214,11 +214,9 @@ public class QueryableIndexCursorHolder implements CursorHolder
}
final Offset baseCursorOffset = offset.clone();
final ColumnSelectorFactory columnSelectorFactory = new QueryableIndexColumnSelectorFactory(
virtualColumns,
Cursors.getTimeOrdering(ordering),
baseCursorOffset.getBaseReadableOffset(),
columnCache
final ColumnSelectorFactory columnSelectorFactory = makeColumnSelectorFactoryForOffset(
columnCache,
baseCursorOffset
);
// filterBundle will only be null if the filter itself is null, otherwise check to see if the filter
// needs to use a value matcher
@ -327,7 +325,20 @@ public class QueryableIndexCursorHolder implements CursorHolder
}
private VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset(
protected ColumnSelectorFactory makeColumnSelectorFactoryForOffset(
ColumnCache columnCache,
Offset baseOffset
)
{
return new QueryableIndexColumnSelectorFactory(
virtualColumns,
Cursors.getTimeOrdering(ordering),
baseOffset.getBaseReadableOffset(),
columnCache
);
}
protected VectorColumnSelectorFactory makeVectorColumnSelectorFactoryForOffset(
ColumnCache columnCache,
VectorOffset baseOffset
)

View File

@ -262,6 +262,14 @@ public class QueryableIndexIndexableAdapter implements IndexableAdapter
return new RowIteratorImpl();
}
@Override
public IndexableAdapter getProjectionAdapter(String projection)
{
QueryableIndex projectionIndex = input.getProjectionQueryableIndex(projection);
DruidException.conditionalDefensive(projectionIndex != null, "Projection[%s] was not found", projection);
return new QueryableIndexIndexableAdapter(projectionIndex);
}
/**
* On {@link #moveToNext()} and {@link #mark()}, this class copies all column values into a set of {@link
* SettableColumnValueSelector} instances. Alternative approach was to save only offset in column and use the same

View File

@ -0,0 +1,73 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/
package org.apache.druid.segment;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.query.dimension.DimensionSpec;
import org.apache.druid.segment.column.ColumnCapabilities;
import javax.annotation.Nullable;
import java.util.Map;
public class RemapColumnSelectorFactory implements ColumnSelectorFactory
{
private final ColumnSelectorFactory delegate;
private final Map<String, String> remap;
public RemapColumnSelectorFactory(ColumnSelectorFactory delegate, Map<String, String> remap)
{
this.delegate = delegate;
this.remap = remap;
}
@Override
public DimensionSelector makeDimensionSelector(DimensionSpec dimensionSpec)
{
DimensionSpec remapDimensionSpec = dimensionSpec.withDimension(remap.getOrDefault(dimensionSpec.getDimension(), dimensionSpec.getDimension()));
return delegate.makeDimensionSelector(remapDimensionSpec);
}
@Override
public ColumnValueSelector makeColumnValueSelector(String columnName)
{
return delegate.makeColumnValueSelector(remap.getOrDefault(columnName, columnName));
}
@Override
@Nullable
public ColumnCapabilities getColumnCapabilities(String column)
{
return delegate.getColumnCapabilities(remap.getOrDefault(column, column));
}
@Nullable
@Override
public RowIdSupplier getRowIdSupplier()
{
return delegate.getRowIdSupplier();
}
@Nullable
@Override
public ExpressionType getType(String name)
{
return delegate.getType(remap.getOrDefault(name, name));
}
}

Some files were not shown because too many files have changed in this diff Show More