mirror of https://github.com/apache/druid.git
Preserve column order in DruidSchema, SegmentMetadataQuery. (#12754)
* Preserve column order in DruidSchema, SegmentMetadataQuery. Instead of putting columns in alphabetical order. This is helpful because it makes query order better match ingestion order. It also allows tools, like the reindexing flow in the web console, to more easily do follow-on ingestions using a column order that matches the pre-existing column order. We prefer the order from the latest segments. The logic takes all columns from the latest segments in the order they appear, then adds on columns from older segments after those. * Additional test adjustments. * Adjust imports.
This commit is contained in:
parent
9c925b4f09
commit
edfbcc8455
|
@ -57,8 +57,7 @@ import org.openjdk.jmh.annotations.Warmup;
|
||||||
import org.openjdk.jmh.infra.Blackhole;
|
import org.openjdk.jmh.infra.Blackhole;
|
||||||
|
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.HashMap;
|
import java.util.LinkedHashMap;
|
||||||
import java.util.Map;
|
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
|
||||||
|
@ -113,7 +112,7 @@ public class DruidSchemaInternRowSignatureBenchmark
|
||||||
protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(Iterable<SegmentId> segments)
|
protected Sequence<SegmentAnalysis> runSegmentMetadataQuery(Iterable<SegmentId> segments)
|
||||||
{
|
{
|
||||||
final int numColumns = 1000;
|
final int numColumns = 1000;
|
||||||
Map<String, ColumnAnalysis> columnToAnalysisMap = new HashMap<>();
|
LinkedHashMap<String, ColumnAnalysis> columnToAnalysisMap = new LinkedHashMap<>();
|
||||||
for (int i = 0; i < numColumns; ++i) {
|
for (int i = 0; i < numColumns; ++i) {
|
||||||
columnToAnalysisMap.put(
|
columnToAnalysisMap.put(
|
||||||
"col" + i,
|
"col" + i,
|
||||||
|
|
|
@ -58,8 +58,8 @@ import org.joda.time.Interval;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
|
||||||
|
|
||||||
public class SegmentAnalyzer
|
public class SegmentAnalyzer
|
||||||
{
|
{
|
||||||
|
@ -98,7 +98,8 @@ public class SegmentAnalyzer
|
||||||
// get length and column names from storageAdapter
|
// get length and column names from storageAdapter
|
||||||
final int length = storageAdapter.getNumRows();
|
final int length = storageAdapter.getNumRows();
|
||||||
|
|
||||||
Map<String, ColumnAnalysis> columns = new TreeMap<>();
|
// Use LinkedHashMap to preserve column order.
|
||||||
|
final Map<String, ColumnAnalysis> columns = new LinkedHashMap<>();
|
||||||
|
|
||||||
final RowSignature rowSignature = storageAdapter.getRowSignature();
|
final RowSignature rowSignature = storageAdapter.getRowSignature();
|
||||||
for (String columnName : rowSignature.getColumnNames()) {
|
for (String columnName : rowSignature.getColumnNames()) {
|
||||||
|
|
|
@ -54,17 +54,19 @@ import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
|
||||||
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
|
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
|
||||||
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
|
import org.apache.druid.query.metadata.metadata.SegmentMetadataQuery;
|
||||||
import org.apache.druid.timeline.LogicalSegment;
|
import org.apache.druid.timeline.LogicalSegment;
|
||||||
|
import org.apache.druid.timeline.SegmentId;
|
||||||
import org.joda.time.DateTime;
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import javax.annotation.Nullable;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
import java.util.TreeMap;
|
|
||||||
import java.util.function.BinaryOperator;
|
import java.util.function.BinaryOperator;
|
||||||
|
|
||||||
public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
|
public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAnalysis, SegmentMetadataQuery>
|
||||||
|
@ -108,7 +110,8 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
||||||
ResponseContext context
|
ResponseContext context
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes(config);
|
SegmentMetadataQuery updatedQuery = ((SegmentMetadataQuery) queryPlus.getQuery()).withFinalizedAnalysisTypes(
|
||||||
|
config);
|
||||||
QueryPlus<SegmentAnalysis> updatedQueryPlus = queryPlus.withQuery(updatedQuery);
|
QueryPlus<SegmentAnalysis> updatedQueryPlus = queryPlus.withQuery(updatedQuery);
|
||||||
return new MappedSequence<>(
|
return new MappedSequence<>(
|
||||||
CombiningSequence.create(
|
CombiningSequence.create(
|
||||||
|
@ -135,7 +138,12 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
||||||
@Override
|
@Override
|
||||||
public BinaryOperator<SegmentAnalysis> createMergeFn(Query<SegmentAnalysis> query)
|
public BinaryOperator<SegmentAnalysis> createMergeFn(Query<SegmentAnalysis> query)
|
||||||
{
|
{
|
||||||
return (arg1, arg2) -> mergeAnalyses(arg1, arg2, ((SegmentMetadataQuery) query).isLenientAggregatorMerge());
|
return (arg1, arg2) -> mergeAnalyses(
|
||||||
|
Iterables.getFirst(query.getDataSource().getTableNames(), null),
|
||||||
|
arg1,
|
||||||
|
arg2,
|
||||||
|
((SegmentMetadataQuery) query).isLenientAggregatorMerge()
|
||||||
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
@ -246,8 +254,9 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
||||||
|
|
||||||
@VisibleForTesting
|
@VisibleForTesting
|
||||||
public static SegmentAnalysis mergeAnalyses(
|
public static SegmentAnalysis mergeAnalyses(
|
||||||
final SegmentAnalysis arg1,
|
@Nullable String dataSource,
|
||||||
final SegmentAnalysis arg2,
|
SegmentAnalysis arg1,
|
||||||
|
SegmentAnalysis arg2,
|
||||||
boolean lenientAggregatorMerge
|
boolean lenientAggregatorMerge
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
@ -259,6 +268,19 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
||||||
return arg1;
|
return arg1;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
// Swap arg1, arg2 so the later-ending interval is first. This ensures we prefer the latest column order.
|
||||||
|
// We're preserving it so callers can see columns in their natural order.
|
||||||
|
if (dataSource != null) {
|
||||||
|
final SegmentId id1 = SegmentId.tryParse(dataSource, arg1.getId());
|
||||||
|
final SegmentId id2 = SegmentId.tryParse(dataSource, arg2.getId());
|
||||||
|
|
||||||
|
if (id1 != null && id2 != null && id2.getIntervalEnd().isAfter(id1.getIntervalEnd())) {
|
||||||
|
final SegmentAnalysis tmp = arg1;
|
||||||
|
arg1 = arg2;
|
||||||
|
arg2 = tmp;
|
||||||
|
}
|
||||||
|
}
|
||||||
|
|
||||||
List<Interval> newIntervals = null;
|
List<Interval> newIntervals = null;
|
||||||
if (arg1.getIntervals() != null) {
|
if (arg1.getIntervals() != null) {
|
||||||
newIntervals = new ArrayList<>(arg1.getIntervals());
|
newIntervals = new ArrayList<>(arg1.getIntervals());
|
||||||
|
@ -272,7 +294,7 @@ public class SegmentMetadataQueryQueryToolChest extends QueryToolChest<SegmentAn
|
||||||
|
|
||||||
final Map<String, ColumnAnalysis> leftColumns = arg1.getColumns();
|
final Map<String, ColumnAnalysis> leftColumns = arg1.getColumns();
|
||||||
final Map<String, ColumnAnalysis> rightColumns = arg2.getColumns();
|
final Map<String, ColumnAnalysis> rightColumns = arg2.getColumns();
|
||||||
Map<String, ColumnAnalysis> columns = new TreeMap<>();
|
final LinkedHashMap<String, ColumnAnalysis> columns = new LinkedHashMap<>();
|
||||||
|
|
||||||
Set<String> rightColumnNames = Sets.newHashSet(rightColumns.keySet());
|
Set<String> rightColumnNames = Sets.newHashSet(rightColumns.keySet());
|
||||||
for (Map.Entry<String, ColumnAnalysis> entry : leftColumns.entrySet()) {
|
for (Map.Entry<String, ColumnAnalysis> entry : leftColumns.entrySet()) {
|
||||||
|
|
|
@ -52,9 +52,9 @@ import org.joda.time.Interval;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.TreeMap;
|
|
||||||
import java.util.concurrent.CancellationException;
|
import java.util.concurrent.CancellationException;
|
||||||
import java.util.concurrent.ExecutionException;
|
import java.util.concurrent.ExecutionException;
|
||||||
import java.util.concurrent.TimeUnit;
|
import java.util.concurrent.TimeUnit;
|
||||||
|
@ -98,7 +98,7 @@ public class SegmentMetadataQueryRunnerFactory implements QueryRunnerFactory<Seg
|
||||||
totalSize = analyzedColumns.size() * numRows;
|
totalSize = analyzedColumns.size() * numRows;
|
||||||
}
|
}
|
||||||
|
|
||||||
Map<String, ColumnAnalysis> columns = new TreeMap<>();
|
LinkedHashMap<String, ColumnAnalysis> columns = new LinkedHashMap<>();
|
||||||
ColumnIncluderator includerator = updatedQuery.getToInclude();
|
ColumnIncluderator includerator = updatedQuery.getToInclude();
|
||||||
for (Map.Entry<String, ColumnAnalysis> entry : analyzedColumns.entrySet()) {
|
for (Map.Entry<String, ColumnAnalysis> entry : analyzedColumns.entrySet()) {
|
||||||
final String columnName = entry.getKey();
|
final String columnName = entry.getKey();
|
||||||
|
|
|
@ -26,6 +26,7 @@ import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.query.aggregation.AggregatorFactory;
|
import org.apache.druid.query.aggregation.AggregatorFactory;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
@ -42,7 +43,12 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
|
||||||
*/
|
*/
|
||||||
private final String id;
|
private final String id;
|
||||||
private final List<Interval> interval;
|
private final List<Interval> interval;
|
||||||
private final Map<String, ColumnAnalysis> columns;
|
|
||||||
|
/**
|
||||||
|
* Require LinkedHashMap to emphasize how important column order is. It's used by DruidSchema to keep
|
||||||
|
* SQL column order in line with ingestion column order.
|
||||||
|
*/
|
||||||
|
private final LinkedHashMap<String, ColumnAnalysis> columns;
|
||||||
private final long size;
|
private final long size;
|
||||||
private final long numRows;
|
private final long numRows;
|
||||||
private final Map<String, AggregatorFactory> aggregators;
|
private final Map<String, AggregatorFactory> aggregators;
|
||||||
|
@ -54,7 +60,7 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
|
||||||
public SegmentAnalysis(
|
public SegmentAnalysis(
|
||||||
@JsonProperty("id") String id,
|
@JsonProperty("id") String id,
|
||||||
@JsonProperty("intervals") List<Interval> interval,
|
@JsonProperty("intervals") List<Interval> interval,
|
||||||
@JsonProperty("columns") Map<String, ColumnAnalysis> columns,
|
@JsonProperty("columns") LinkedHashMap<String, ColumnAnalysis> columns,
|
||||||
@JsonProperty("size") long size,
|
@JsonProperty("size") long size,
|
||||||
@JsonProperty("numRows") long numRows,
|
@JsonProperty("numRows") long numRows,
|
||||||
@JsonProperty("aggregators") Map<String, AggregatorFactory> aggregators,
|
@JsonProperty("aggregators") Map<String, AggregatorFactory> aggregators,
|
||||||
|
@ -87,7 +93,7 @@ public class SegmentAnalysis implements Comparable<SegmentAnalysis>
|
||||||
}
|
}
|
||||||
|
|
||||||
@JsonProperty
|
@JsonProperty
|
||||||
public Map<String, ColumnAnalysis> getColumns()
|
public LinkedHashMap<String, ColumnAnalysis> getColumns()
|
||||||
{
|
{
|
||||||
return columns;
|
return columns;
|
||||||
}
|
}
|
||||||
|
|
|
@ -79,6 +79,7 @@ import java.nio.ByteOrder;
|
||||||
import java.util.ArrayList;
|
import java.util.ArrayList;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
import java.util.Iterator;
|
import java.util.Iterator;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
@ -442,7 +443,7 @@ public class IndexIO
|
||||||
{
|
{
|
||||||
MMappedIndex index = legacyHandler.mapDir(inDir);
|
MMappedIndex index = legacyHandler.mapDir(inDir);
|
||||||
|
|
||||||
Map<String, Supplier<ColumnHolder>> columns = new HashMap<>();
|
Map<String, Supplier<ColumnHolder>> columns = new LinkedHashMap<>();
|
||||||
|
|
||||||
for (String dimension : index.getAvailableDimensions()) {
|
for (String dimension : index.getAvailableDimensions()) {
|
||||||
ColumnBuilder builder = new ColumnBuilder()
|
ColumnBuilder builder = new ColumnBuilder()
|
||||||
|
@ -624,7 +625,7 @@ public class IndexIO
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
||||||
Map<String, Supplier<ColumnHolder>> columns = new HashMap<>();
|
Map<String, Supplier<ColumnHolder>> columns = new LinkedHashMap<>();
|
||||||
|
|
||||||
// Register the time column
|
// Register the time column
|
||||||
ByteBuffer timeBuffer = smooshedFiles.mapFile("__time");
|
ByteBuffer timeBuffer = smooshedFiles.mapFile("__time");
|
||||||
|
|
|
@ -20,7 +20,6 @@
|
||||||
package org.apache.druid.segment;
|
package org.apache.druid.segment;
|
||||||
|
|
||||||
import com.google.common.annotations.VisibleForTesting;
|
import com.google.common.annotations.VisibleForTesting;
|
||||||
import com.google.common.collect.Sets;
|
|
||||||
import org.apache.druid.java.util.common.DateTimes;
|
import org.apache.druid.java.util.common.DateTimes;
|
||||||
import org.apache.druid.java.util.common.ISE;
|
import org.apache.druid.java.util.common.ISE;
|
||||||
import org.apache.druid.java.util.common.granularity.Granularities;
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
@ -44,8 +43,9 @@ import org.joda.time.Interval;
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.io.UncheckedIOException;
|
import java.io.UncheckedIOException;
|
||||||
import java.util.HashSet;
|
import java.util.LinkedHashSet;
|
||||||
import java.util.Objects;
|
import java.util.Objects;
|
||||||
|
import java.util.Set;
|
||||||
|
|
||||||
/**
|
/**
|
||||||
*
|
*
|
||||||
|
@ -82,8 +82,14 @@ public class QueryableIndexStorageAdapter implements StorageAdapter
|
||||||
@Override
|
@Override
|
||||||
public Iterable<String> getAvailableMetrics()
|
public Iterable<String> getAvailableMetrics()
|
||||||
{
|
{
|
||||||
HashSet<String> columnNames = Sets.newHashSet(index.getColumnNames());
|
// Use LinkedHashSet to preserve the original order.
|
||||||
return Sets.difference(columnNames, Sets.newHashSet(index.getAvailableDimensions()));
|
final Set<String> columnNames = new LinkedHashSet<>(index.getColumnNames());
|
||||||
|
|
||||||
|
for (final String dimension : index.getAvailableDimensions()) {
|
||||||
|
columnNames.remove(dimension);
|
||||||
|
}
|
||||||
|
|
||||||
|
return columnNames;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
|
|
|
@ -73,6 +73,7 @@ import java.io.IOException;
|
||||||
import java.util.Arrays;
|
import java.util.Arrays;
|
||||||
import java.util.Collection;
|
import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -153,42 +154,44 @@ public class DoubleStorageTest
|
||||||
SegmentAnalysis expectedSegmentAnalysisDouble = new SegmentAnalysis(
|
SegmentAnalysis expectedSegmentAnalysisDouble = new SegmentAnalysis(
|
||||||
SEGMENT_ID.toString(),
|
SEGMENT_ID.toString(),
|
||||||
ImmutableList.of(INTERVAL),
|
ImmutableList.of(INTERVAL),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
TIME_COLUMN,
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
TIME_COLUMN,
|
||||||
ColumnType.LONG,
|
new ColumnAnalysis(
|
||||||
ValueType.LONG.name(),
|
ColumnType.LONG,
|
||||||
false,
|
ValueType.LONG.name(),
|
||||||
false,
|
false,
|
||||||
100,
|
false,
|
||||||
null,
|
100,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
DIM_NAME,
|
),
|
||||||
new ColumnAnalysis(
|
DIM_NAME,
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.name(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.name(),
|
||||||
false,
|
false,
|
||||||
120,
|
false,
|
||||||
1,
|
120,
|
||||||
DIM_VALUE,
|
1,
|
||||||
DIM_VALUE,
|
DIM_VALUE,
|
||||||
null
|
DIM_VALUE,
|
||||||
),
|
null
|
||||||
DIM_FLOAT_NAME,
|
),
|
||||||
new ColumnAnalysis(
|
DIM_FLOAT_NAME,
|
||||||
ColumnType.DOUBLE,
|
new ColumnAnalysis(
|
||||||
ValueType.DOUBLE.name(),
|
ColumnType.DOUBLE,
|
||||||
false,
|
ValueType.DOUBLE.name(),
|
||||||
false,
|
false,
|
||||||
80,
|
false,
|
||||||
null,
|
80,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
), 330,
|
), 330,
|
||||||
MAX_ROWS,
|
MAX_ROWS,
|
||||||
|
@ -201,44 +204,47 @@ public class DoubleStorageTest
|
||||||
SegmentAnalysis expectedSegmentAnalysisFloat = new SegmentAnalysis(
|
SegmentAnalysis expectedSegmentAnalysisFloat = new SegmentAnalysis(
|
||||||
SEGMENT_ID.toString(),
|
SEGMENT_ID.toString(),
|
||||||
ImmutableList.of(INTERVAL),
|
ImmutableList.of(INTERVAL),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
TIME_COLUMN,
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
TIME_COLUMN,
|
||||||
ColumnType.LONG,
|
new ColumnAnalysis(
|
||||||
ValueType.LONG.name(),
|
ColumnType.LONG,
|
||||||
false,
|
ValueType.LONG.name(),
|
||||||
false,
|
false,
|
||||||
100,
|
false,
|
||||||
null,
|
100,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
DIM_NAME,
|
),
|
||||||
new ColumnAnalysis(
|
DIM_NAME,
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.name(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.name(),
|
||||||
false,
|
false,
|
||||||
120,
|
false,
|
||||||
1,
|
120,
|
||||||
DIM_VALUE,
|
1,
|
||||||
DIM_VALUE,
|
DIM_VALUE,
|
||||||
null
|
DIM_VALUE,
|
||||||
),
|
null
|
||||||
DIM_FLOAT_NAME,
|
),
|
||||||
new ColumnAnalysis(
|
DIM_FLOAT_NAME,
|
||||||
ColumnType.FLOAT,
|
new ColumnAnalysis(
|
||||||
ValueType.FLOAT.name(),
|
ColumnType.FLOAT,
|
||||||
false,
|
ValueType.FLOAT.name(),
|
||||||
false,
|
false,
|
||||||
80,
|
false,
|
||||||
null,
|
80,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
), 330,
|
),
|
||||||
|
330,
|
||||||
MAX_ROWS,
|
MAX_ROWS,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
|
|
@ -0,0 +1,89 @@
|
||||||
|
/*
|
||||||
|
* 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.query.metadata;
|
||||||
|
|
||||||
|
import com.fasterxml.jackson.databind.ObjectMapper;
|
||||||
|
import com.google.common.collect.ImmutableList;
|
||||||
|
import com.google.common.collect.ImmutableMap;
|
||||||
|
import org.apache.druid.data.input.impl.TimestampSpec;
|
||||||
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
|
import org.apache.druid.java.util.common.granularity.Granularities;
|
||||||
|
import org.apache.druid.query.aggregation.CountAggregatorFactory;
|
||||||
|
import org.apache.druid.query.metadata.metadata.ColumnAnalysis;
|
||||||
|
import org.apache.druid.query.metadata.metadata.SegmentAnalysis;
|
||||||
|
import org.apache.druid.segment.TestHelper;
|
||||||
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
|
import org.junit.Assert;
|
||||||
|
import org.junit.Test;
|
||||||
|
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
|
|
||||||
|
public class SegmentAnalysisTest
|
||||||
|
{
|
||||||
|
@Test
|
||||||
|
public void testSerde() throws Exception
|
||||||
|
{
|
||||||
|
// Use LinkedHashMap to preserve order.
|
||||||
|
// We'll verify that the order is actually preserved on serde.
|
||||||
|
final LinkedHashMap<String, ColumnAnalysis> columns = new LinkedHashMap<>();
|
||||||
|
columns.put(
|
||||||
|
"b",
|
||||||
|
new ColumnAnalysis(ColumnType.LONG, ColumnType.LONG.asTypeString(), true, true, 0, null, null, null, null)
|
||||||
|
);
|
||||||
|
columns.put(
|
||||||
|
"a",
|
||||||
|
new ColumnAnalysis(ColumnType.FLOAT, ColumnType.FLOAT.asTypeString(), true, true, 0, null, null, null, null)
|
||||||
|
);
|
||||||
|
columns.put(
|
||||||
|
"f",
|
||||||
|
new ColumnAnalysis(ColumnType.STRING, ColumnType.STRING.asTypeString(), true, true, 0, null, null, null, null)
|
||||||
|
);
|
||||||
|
columns.put(
|
||||||
|
"c",
|
||||||
|
new ColumnAnalysis(ColumnType.DOUBLE, ColumnType.DOUBLE.asTypeString(), true, true, 0, null, null, null, null)
|
||||||
|
);
|
||||||
|
|
||||||
|
final SegmentAnalysis analysis = new SegmentAnalysis(
|
||||||
|
"id",
|
||||||
|
Intervals.ONLY_ETERNITY,
|
||||||
|
columns,
|
||||||
|
1,
|
||||||
|
2,
|
||||||
|
ImmutableMap.of("cnt", new CountAggregatorFactory("cnt")),
|
||||||
|
new TimestampSpec(null, null, null),
|
||||||
|
Granularities.SECOND,
|
||||||
|
true
|
||||||
|
);
|
||||||
|
|
||||||
|
final ObjectMapper jsonMapper = TestHelper.makeJsonMapper();
|
||||||
|
final SegmentAnalysis analysis2 = jsonMapper.readValue(
|
||||||
|
jsonMapper.writeValueAsBytes(analysis),
|
||||||
|
SegmentAnalysis.class
|
||||||
|
);
|
||||||
|
|
||||||
|
Assert.assertEquals(analysis, analysis2);
|
||||||
|
|
||||||
|
// Verify column order is preserved.
|
||||||
|
Assert.assertEquals(
|
||||||
|
ImmutableList.copyOf(columns.entrySet()),
|
||||||
|
ImmutableList.copyOf(analysis2.getColumns().entrySet())
|
||||||
|
);
|
||||||
|
}
|
||||||
|
}
|
|
@ -46,6 +46,7 @@ import org.apache.druid.segment.QueryableIndexSegment;
|
||||||
import org.apache.druid.segment.Segment;
|
import org.apache.druid.segment.Segment;
|
||||||
import org.apache.druid.segment.TestIndex;
|
import org.apache.druid.segment.TestIndex;
|
||||||
import org.apache.druid.segment.column.ColumnBuilder;
|
import org.apache.druid.segment.column.ColumnBuilder;
|
||||||
|
import org.apache.druid.segment.column.ColumnHolder;
|
||||||
import org.apache.druid.segment.column.ColumnType;
|
import org.apache.druid.segment.column.ColumnType;
|
||||||
import org.apache.druid.segment.column.ValueType;
|
import org.apache.druid.segment.column.ValueType;
|
||||||
import org.apache.druid.segment.data.ObjectStrategy;
|
import org.apache.druid.segment.data.ObjectStrategy;
|
||||||
|
@ -68,6 +69,7 @@ import java.io.IOException;
|
||||||
import java.net.URL;
|
import java.net.URL;
|
||||||
import java.nio.ByteBuffer;
|
import java.nio.ByteBuffer;
|
||||||
import java.nio.charset.StandardCharsets;
|
import java.nio.charset.StandardCharsets;
|
||||||
|
import java.util.ArrayList;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
|
@ -108,9 +110,20 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest
|
||||||
columns.size()
|
columns.size()
|
||||||
); // All columns including time and empty/null column
|
); // All columns including time and empty/null column
|
||||||
|
|
||||||
for (DimensionSchema schema : TestIndex.DIMENSION_SCHEMAS) {
|
// Verify key order is the same as the underlying segment.
|
||||||
|
// This helps DruidSchema keep things in the proper order when it does SegmentMetadata queries.
|
||||||
|
final List<Map.Entry<String, ColumnAnalysis>> entriesInOrder = new ArrayList<>(columns.entrySet());
|
||||||
|
|
||||||
|
Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, entriesInOrder.get(0).getKey());
|
||||||
|
Assert.assertEquals(ColumnType.LONG, entriesInOrder.get(0).getValue().getTypeSignature());
|
||||||
|
|
||||||
|
// Start from 1: skipping __time
|
||||||
|
for (int i = 0; i < TestIndex.DIMENSION_SCHEMAS.size(); i++) {
|
||||||
|
final DimensionSchema schema = TestIndex.DIMENSION_SCHEMAS.get(i);
|
||||||
|
final Map.Entry<String, ColumnAnalysis> analysisEntry = entriesInOrder.get(i + 1 /* skip __time */);
|
||||||
final String dimension = schema.getName();
|
final String dimension = schema.getName();
|
||||||
final ColumnAnalysis columnAnalysis = columns.get(dimension);
|
Assert.assertEquals(dimension, analysisEntry.getKey());
|
||||||
|
final ColumnAnalysis columnAnalysis = analysisEntry.getValue();
|
||||||
final boolean isString = schema.getColumnType().is(ValueType.STRING);
|
final boolean isString = schema.getColumnType().is(ValueType.STRING);
|
||||||
|
|
||||||
Assert.assertEquals(dimension, schema.getColumnType().toString(), columnAnalysis.getType());
|
Assert.assertEquals(dimension, schema.getColumnType().toString(), columnAnalysis.getType());
|
||||||
|
@ -161,14 +174,20 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest
|
||||||
Assert.assertEquals(SegmentId.dummy("test_1").toString(), analysis.getId());
|
Assert.assertEquals(SegmentId.dummy("test_1").toString(), analysis.getId());
|
||||||
|
|
||||||
final Map<String, ColumnAnalysis> columns = analysis.getColumns();
|
final Map<String, ColumnAnalysis> columns = analysis.getColumns();
|
||||||
Assert.assertEquals(
|
// Verify key order is the same as the underlying segment.
|
||||||
TestIndex.COLUMNS.length + 3,
|
// This helps DruidSchema keep things in the proper order when it does SegmentMetadata queries.
|
||||||
columns.size()
|
final List<Map.Entry<String, ColumnAnalysis>> entriesInOrder = new ArrayList<>(columns.entrySet());
|
||||||
); // All columns including time
|
|
||||||
|
|
||||||
for (DimensionSchema schema : TestIndex.DIMENSION_SCHEMAS) {
|
Assert.assertEquals(ColumnHolder.TIME_COLUMN_NAME, entriesInOrder.get(0).getKey());
|
||||||
|
Assert.assertEquals(ColumnType.LONG, entriesInOrder.get(0).getValue().getTypeSignature());
|
||||||
|
|
||||||
|
// Start from 1: skipping __time
|
||||||
|
for (int i = 0; i < TestIndex.DIMENSION_SCHEMAS.size(); i++) {
|
||||||
|
final DimensionSchema schema = TestIndex.DIMENSION_SCHEMAS.get(i);
|
||||||
|
final Map.Entry<String, ColumnAnalysis> analysisEntry = entriesInOrder.get(i + 1 /* skip __time */);
|
||||||
final String dimension = schema.getName();
|
final String dimension = schema.getName();
|
||||||
final ColumnAnalysis columnAnalysis = columns.get(dimension);
|
Assert.assertEquals(dimension, analysisEntry.getKey());
|
||||||
|
final ColumnAnalysis columnAnalysis = analysisEntry.getValue();
|
||||||
final boolean isString = schema.getColumnType().is(ValueType.STRING);
|
final boolean isString = schema.getColumnType().is(ValueType.STRING);
|
||||||
Assert.assertEquals(dimension, schema.getColumnType().toString(), columnAnalysis.getType());
|
Assert.assertEquals(dimension, schema.getColumnType().toString(), columnAnalysis.getType());
|
||||||
Assert.assertEquals(dimension, 0, columnAnalysis.getSize());
|
Assert.assertEquals(dimension, 0, columnAnalysis.getSize());
|
||||||
|
@ -204,6 +223,7 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest
|
||||||
* *Awesome* method name auto-generated by IntelliJ! I love IntelliJ!
|
* *Awesome* method name auto-generated by IntelliJ! I love IntelliJ!
|
||||||
*
|
*
|
||||||
* @param index
|
* @param index
|
||||||
|
*
|
||||||
* @return
|
* @return
|
||||||
*/
|
*/
|
||||||
private List<SegmentAnalysis> getSegmentAnalysises(Segment index, EnumSet<SegmentMetadataQuery.AnalysisType> analyses)
|
private List<SegmentAnalysis> getSegmentAnalysises(Segment index, EnumSet<SegmentMetadataQuery.AnalysisType> analyses)
|
||||||
|
@ -257,6 +277,7 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest
|
||||||
* (which can happen if an aggregator was removed for a later version), then,
|
* (which can happen if an aggregator was removed for a later version), then,
|
||||||
* analyzing the segment doesn't fail and the result of analysis of the complex column
|
* analyzing the segment doesn't fail and the result of analysis of the complex column
|
||||||
* is reported as an error.
|
* is reported as an error.
|
||||||
|
*
|
||||||
* @throws IOException
|
* @throws IOException
|
||||||
*/
|
*/
|
||||||
@Test
|
@Test
|
||||||
|
@ -317,7 +338,10 @@ public class SegmentAnalyzerTest extends InitializedNullHandlingTest
|
||||||
Assert.assertEquals("error:unknown_complex_invalid_complex_column_type", invalidColumnAnalysis.getErrorMessage());
|
Assert.assertEquals("error:unknown_complex_invalid_complex_column_type", invalidColumnAnalysis.getErrorMessage());
|
||||||
|
|
||||||
// Run a segment metadata query also to verify it doesn't break
|
// Run a segment metadata query also to verify it doesn't break
|
||||||
final List<SegmentAnalysis> results = getSegmentAnalysises(segment, EnumSet.of(SegmentMetadataQuery.AnalysisType.SIZE));
|
final List<SegmentAnalysis> results = getSegmentAnalysises(
|
||||||
|
segment,
|
||||||
|
EnumSet.of(SegmentMetadataQuery.AnalysisType.SIZE)
|
||||||
|
);
|
||||||
for (SegmentAnalysis result : results) {
|
for (SegmentAnalysis result : results) {
|
||||||
Assert.assertTrue(result.getColumns().get(invalid_aggregator).isError());
|
Assert.assertTrue(result.getColumns().get(invalid_aggregator).isError());
|
||||||
}
|
}
|
||||||
|
|
|
@ -46,6 +46,7 @@ import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.stream.Collectors;
|
import java.util.stream.Collectors;
|
||||||
|
@ -77,20 +78,23 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
SegmentAnalysis result = new SegmentAnalysis(
|
SegmentAnalysis result = new SegmentAnalysis(
|
||||||
"testSegment",
|
"testSegment",
|
||||||
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.name(),
|
ColumnType.STRING,
|
||||||
true,
|
ValueType.STRING.name(),
|
||||||
false,
|
true,
|
||||||
10881,
|
false,
|
||||||
1,
|
10881,
|
||||||
"preferred",
|
1,
|
||||||
"preferred",
|
"preferred",
|
||||||
null
|
"preferred",
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
), 71982,
|
),
|
||||||
|
71982,
|
||||||
100,
|
100,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
|
@ -117,7 +121,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
|
@ -131,7 +135,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
|
@ -167,7 +171,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -178,7 +182,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
|
@ -206,7 +210,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -217,7 +221,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -236,7 +240,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
|
@ -250,7 +254,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
ImmutableMap.of(
|
ImmutableMap.of(
|
||||||
|
@ -331,7 +335,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
final SegmentAnalysis analysis1 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -342,7 +346,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
final SegmentAnalysis analysis2 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -353,7 +357,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis3 = new SegmentAnalysis(
|
final SegmentAnalysis analysis3 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -364,7 +368,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis4 = new SegmentAnalysis(
|
final SegmentAnalysis analysis4 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -375,7 +379,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
final SegmentAnalysis analysis5 = new SegmentAnalysis(
|
final SegmentAnalysis analysis5 = new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
null,
|
null,
|
||||||
new HashMap<>(),
|
new LinkedHashMap<>(),
|
||||||
0,
|
0,
|
||||||
0,
|
0,
|
||||||
null,
|
null,
|
||||||
|
@ -395,6 +399,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
{
|
{
|
||||||
return SegmentMetadataQueryQueryToolChest.finalizeAnalysis(
|
return SegmentMetadataQueryQueryToolChest.finalizeAnalysis(
|
||||||
SegmentMetadataQueryQueryToolChest.mergeAnalyses(
|
SegmentMetadataQueryQueryToolChest.mergeAnalyses(
|
||||||
|
null,
|
||||||
analysis1,
|
analysis1,
|
||||||
analysis2,
|
analysis2,
|
||||||
false
|
false
|
||||||
|
@ -406,6 +411,7 @@ public class SegmentMetadataQueryQueryToolChestTest
|
||||||
{
|
{
|
||||||
return SegmentMetadataQueryQueryToolChest.finalizeAnalysis(
|
return SegmentMetadataQueryQueryToolChest.finalizeAnalysis(
|
||||||
SegmentMetadataQueryQueryToolChest.mergeAnalyses(
|
SegmentMetadataQueryQueryToolChest.mergeAnalyses(
|
||||||
|
null,
|
||||||
analysis1,
|
analysis1,
|
||||||
analysis2,
|
analysis2,
|
||||||
true
|
true
|
||||||
|
|
|
@ -69,6 +69,7 @@ import java.util.Collection;
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.HashMap;
|
import java.util.HashMap;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.concurrent.ExecutorService;
|
import java.util.concurrent.ExecutorService;
|
||||||
|
@ -202,42 +203,44 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
expectedSegmentAnalysis1 = new SegmentAnalysis(
|
expectedSegmentAnalysis1 = new SegmentAnalysis(
|
||||||
id1.toString(),
|
id1.toString(),
|
||||||
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"__time",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"__time",
|
||||||
ColumnType.LONG,
|
new ColumnAnalysis(
|
||||||
ValueType.LONG.toString(),
|
ColumnType.LONG,
|
||||||
false,
|
ValueType.LONG.toString(),
|
||||||
false,
|
false,
|
||||||
12090,
|
false,
|
||||||
null,
|
12090,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
"index",
|
),
|
||||||
new ColumnAnalysis(
|
"index",
|
||||||
ColumnType.DOUBLE,
|
new ColumnAnalysis(
|
||||||
ValueType.DOUBLE.toString(),
|
ColumnType.DOUBLE,
|
||||||
false,
|
ValueType.DOUBLE.toString(),
|
||||||
false,
|
false,
|
||||||
9672,
|
false,
|
||||||
null,
|
9672,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
"placement",
|
),
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
preferedSize1,
|
false,
|
||||||
1,
|
preferedSize1,
|
||||||
"preferred",
|
1,
|
||||||
"preferred",
|
"preferred",
|
||||||
null
|
"preferred",
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
overallSize1,
|
overallSize1,
|
||||||
|
@ -250,42 +253,44 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
expectedSegmentAnalysis2 = new SegmentAnalysis(
|
expectedSegmentAnalysis2 = new SegmentAnalysis(
|
||||||
id2.toString(),
|
id2.toString(),
|
||||||
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
ImmutableList.of(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"__time",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"__time",
|
||||||
ColumnType.LONG,
|
new ColumnAnalysis(
|
||||||
ValueType.LONG.toString(),
|
ColumnType.LONG,
|
||||||
false,
|
ValueType.LONG.toString(),
|
||||||
false,
|
false,
|
||||||
12090,
|
false,
|
||||||
null,
|
12090,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
"index",
|
),
|
||||||
new ColumnAnalysis(
|
"index",
|
||||||
ColumnType.DOUBLE,
|
new ColumnAnalysis(
|
||||||
ValueType.DOUBLE.toString(),
|
ColumnType.DOUBLE,
|
||||||
false,
|
ValueType.DOUBLE.toString(),
|
||||||
false,
|
false,
|
||||||
9672,
|
false,
|
||||||
null,
|
9672,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
"placement",
|
),
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
placementSize2,
|
false,
|
||||||
1,
|
placementSize2,
|
||||||
null,
|
1,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
// null_column will be included only for incremental index, which makes a little bigger result than expected
|
// null_column will be included only for incremental index, which makes a little bigger result than expected
|
||||||
|
@ -313,30 +318,32 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
0,
|
false,
|
||||||
0,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
),
|
null
|
||||||
"placementish",
|
),
|
||||||
new ColumnAnalysis(
|
"placementish",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
true,
|
ValueType.STRING.toString(),
|
||||||
false,
|
true,
|
||||||
0,
|
false,
|
||||||
0,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
0,
|
0,
|
||||||
|
@ -385,30 +392,32 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
0,
|
false,
|
||||||
1,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
1,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
),
|
null
|
||||||
"placementish",
|
),
|
||||||
new ColumnAnalysis(
|
"placementish",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
true,
|
ValueType.STRING.toString(),
|
||||||
false,
|
true,
|
||||||
0,
|
false,
|
||||||
9,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
9,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
0,
|
0,
|
||||||
|
@ -457,30 +466,32 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
0,
|
false,
|
||||||
1,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
1,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
),
|
null
|
||||||
"quality_uniques",
|
),
|
||||||
new ColumnAnalysis(
|
"quality_uniques",
|
||||||
ColumnType.ofComplex("hyperUnique"),
|
new ColumnAnalysis(
|
||||||
"hyperUnique",
|
ColumnType.ofComplex("hyperUnique"),
|
||||||
false,
|
"hyperUnique",
|
||||||
true,
|
false,
|
||||||
0,
|
true,
|
||||||
null,
|
0,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
0,
|
0,
|
||||||
|
@ -600,33 +611,35 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
ImmutableList.of(expectedSegmentAnalysis1.getIntervals().get(0)),
|
ImmutableList.of(expectedSegmentAnalysis1.getIntervals().get(0)),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"__time",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"__time",
|
||||||
ColumnType.LONG,
|
new ColumnAnalysis(
|
||||||
ValueType.LONG.toString(),
|
ColumnType.LONG,
|
||||||
false,
|
ValueType.LONG.toString(),
|
||||||
false,
|
false,
|
||||||
12090 * 2,
|
false,
|
||||||
null,
|
12090 * 2,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
"index",
|
),
|
||||||
new ColumnAnalysis(
|
"index",
|
||||||
ColumnType.DOUBLE,
|
new ColumnAnalysis(
|
||||||
ValueType.DOUBLE.toString(),
|
ColumnType.DOUBLE,
|
||||||
false,
|
ValueType.DOUBLE.toString(),
|
||||||
false,
|
false,
|
||||||
9672 * 2,
|
false,
|
||||||
null,
|
9672 * 2,
|
||||||
null,
|
null,
|
||||||
null,
|
null,
|
||||||
null
|
null,
|
||||||
),
|
null
|
||||||
column,
|
),
|
||||||
analysis
|
column,
|
||||||
|
analysis
|
||||||
|
)
|
||||||
),
|
),
|
||||||
expectedSegmentAnalysis1.getSize() + expectedSegmentAnalysis2.getSize(),
|
expectedSegmentAnalysis1.getSize() + expectedSegmentAnalysis2.getSize(),
|
||||||
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
|
expectedSegmentAnalysis1.getNumRows() + expectedSegmentAnalysis2.getNumRows(),
|
||||||
|
@ -668,18 +681,20 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
0,
|
false,
|
||||||
0,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
0,
|
0,
|
||||||
|
@ -732,18 +747,20 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
0,
|
false,
|
||||||
0,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
0,
|
0,
|
||||||
|
@ -792,18 +809,20 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
0,
|
false,
|
||||||
0,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
0,
|
0,
|
||||||
|
@ -852,18 +871,20 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
SegmentAnalysis mergedSegmentAnalysis = new SegmentAnalysis(
|
||||||
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
differentIds ? "merged" : SegmentId.dummy("testSegment").toString(),
|
||||||
null,
|
null,
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
0,
|
false,
|
||||||
0,
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
0,
|
||||||
NullHandling.defaultStringValue(),
|
NullHandling.defaultStringValue(),
|
||||||
null
|
NullHandling.defaultStringValue(),
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
0,
|
0,
|
||||||
|
@ -937,7 +958,10 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
|
|
||||||
TestHelper.assertExpectedObjects(
|
TestHelper.assertExpectedObjects(
|
||||||
ImmutableList.of(bySegmentResult, bySegmentResult),
|
ImmutableList.of(bySegmentResult, bySegmentResult),
|
||||||
myRunner.run(QueryPlus.wrap(testQuery.withOverriddenContext(ImmutableMap.of(QueryContexts.BY_SEGMENT_KEY, true)))),
|
myRunner.run(QueryPlus.wrap(testQuery.withOverriddenContext(ImmutableMap.of(
|
||||||
|
QueryContexts.BY_SEGMENT_KEY,
|
||||||
|
true
|
||||||
|
)))),
|
||||||
"failed SegmentMetadata bySegment query"
|
"failed SegmentMetadata bySegment query"
|
||||||
);
|
);
|
||||||
exec.shutdownNow();
|
exec.shutdownNow();
|
||||||
|
@ -1265,12 +1289,12 @@ public class SegmentMetadataQueryTest extends InitializedNullHandlingTest
|
||||||
.build();
|
.build();
|
||||||
|
|
||||||
final byte[] oneColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()).getCacheStrategy(
|
final byte[] oneColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()).getCacheStrategy(
|
||||||
oneColumnQuery)
|
oneColumnQuery)
|
||||||
.computeCacheKey(
|
.computeCacheKey(
|
||||||
oneColumnQuery);
|
oneColumnQuery);
|
||||||
|
|
||||||
final byte[] twoColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()).getCacheStrategy(
|
final byte[] twoColumnQueryCacheKey = new SegmentMetadataQueryQueryToolChest(new SegmentMetadataQueryConfig()).getCacheStrategy(
|
||||||
twoColumnQuery)
|
twoColumnQuery)
|
||||||
.computeCacheKey(
|
.computeCacheKey(
|
||||||
twoColumnQuery);
|
twoColumnQuery);
|
||||||
|
|
||||||
|
|
|
@ -44,6 +44,7 @@ import org.junit.runner.RunWith;
|
||||||
import org.junit.runners.Parameterized;
|
import org.junit.runners.Parameterized;
|
||||||
|
|
||||||
import java.util.Collections;
|
import java.util.Collections;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
|
|
||||||
@RunWith(Parameterized.class)
|
@RunWith(Parameterized.class)
|
||||||
|
@ -80,7 +81,7 @@ public class SegmentMetadataUnionQueryTest extends InitializedNullHandlingTest
|
||||||
null
|
null
|
||||||
),
|
),
|
||||||
true,
|
true,
|
||||||
},
|
},
|
||||||
new Object[]{
|
new Object[]{
|
||||||
QueryRunnerTestHelper.makeUnionQueryRunner(
|
QueryRunnerTestHelper.makeUnionQueryRunner(
|
||||||
FACTORY,
|
FACTORY,
|
||||||
|
@ -99,18 +100,20 @@ public class SegmentMetadataUnionQueryTest extends InitializedNullHandlingTest
|
||||||
SegmentAnalysis expected = new SegmentAnalysis(
|
SegmentAnalysis expected = new SegmentAnalysis(
|
||||||
QueryRunnerTestHelper.SEGMENT_ID.toString(),
|
QueryRunnerTestHelper.SEGMENT_ID.toString(),
|
||||||
Collections.singletonList(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
Collections.singletonList(Intervals.of("2011-01-12T00:00:00.000Z/2011-04-15T00:00:00.001Z")),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"placement",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
"placement",
|
||||||
ColumnType.STRING,
|
new ColumnAnalysis(
|
||||||
ValueType.STRING.toString(),
|
ColumnType.STRING,
|
||||||
false,
|
ValueType.STRING.toString(),
|
||||||
false,
|
false,
|
||||||
mmap ? 43524 : 43056,
|
false,
|
||||||
1,
|
mmap ? 43524 : 43056,
|
||||||
"preferred",
|
1,
|
||||||
"preferred",
|
"preferred",
|
||||||
null
|
"preferred",
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
mmap ? 805380 : 803324,
|
mmap ? 805380 : 803324,
|
||||||
|
|
|
@ -73,6 +73,7 @@ import java.io.IOException;
|
||||||
import java.util.Comparator;
|
import java.util.Comparator;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Optional;
|
import java.util.Optional;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -783,7 +784,9 @@ public class DruidSchema extends AbstractSchema
|
||||||
DruidTable buildDruidTable(final String dataSource)
|
DruidTable buildDruidTable(final String dataSource)
|
||||||
{
|
{
|
||||||
ConcurrentSkipListMap<SegmentId, AvailableSegmentMetadata> segmentsMap = segmentMetadataInfo.get(dataSource);
|
ConcurrentSkipListMap<SegmentId, AvailableSegmentMetadata> segmentsMap = segmentMetadataInfo.get(dataSource);
|
||||||
final Map<String, ColumnType> columnTypes = new TreeMap<>();
|
|
||||||
|
// Preserve order.
|
||||||
|
final Map<String, ColumnType> columnTypes = new LinkedHashMap<>();
|
||||||
|
|
||||||
if (segmentsMap != null && !segmentsMap.isEmpty()) {
|
if (segmentsMap != null && !segmentsMap.isEmpty()) {
|
||||||
for (AvailableSegmentMetadata availableSegmentMetadata : segmentsMap.values()) {
|
for (AvailableSegmentMetadata availableSegmentMetadata : segmentsMap.values()) {
|
||||||
|
|
|
@ -572,14 +572,6 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
||||||
Pair.of("TYPE_NAME", "TIMESTAMP"),
|
Pair.of("TYPE_NAME", "TIMESTAMP"),
|
||||||
Pair.of("IS_NULLABLE", "NO")
|
Pair.of("IS_NULLABLE", "NO")
|
||||||
),
|
),
|
||||||
row(
|
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
|
||||||
Pair.of("TABLE_NAME", "foo"),
|
|
||||||
Pair.of("COLUMN_NAME", "cnt"),
|
|
||||||
Pair.of("DATA_TYPE", Types.BIGINT),
|
|
||||||
Pair.of("TYPE_NAME", "BIGINT"),
|
|
||||||
Pair.of("IS_NULLABLE", nullNumeric ? "YES" : "NO")
|
|
||||||
),
|
|
||||||
row(
|
row(
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
Pair.of("TABLE_NAME", "foo"),
|
Pair.of("TABLE_NAME", "foo"),
|
||||||
|
@ -604,6 +596,14 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
||||||
Pair.of("TYPE_NAME", "VARCHAR"),
|
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||||
Pair.of("IS_NULLABLE", "YES")
|
Pair.of("IS_NULLABLE", "YES")
|
||||||
),
|
),
|
||||||
|
row(
|
||||||
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
|
Pair.of("TABLE_NAME", "foo"),
|
||||||
|
Pair.of("COLUMN_NAME", "cnt"),
|
||||||
|
Pair.of("DATA_TYPE", Types.BIGINT),
|
||||||
|
Pair.of("TYPE_NAME", "BIGINT"),
|
||||||
|
Pair.of("IS_NULLABLE", nullNumeric ? "YES" : "NO")
|
||||||
|
),
|
||||||
row(
|
row(
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
Pair.of("TABLE_NAME", "foo"),
|
Pair.of("TABLE_NAME", "foo"),
|
||||||
|
@ -663,14 +663,6 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
||||||
Pair.of("TYPE_NAME", "TIMESTAMP"),
|
Pair.of("TYPE_NAME", "TIMESTAMP"),
|
||||||
Pair.of("IS_NULLABLE", "NO")
|
Pair.of("IS_NULLABLE", "NO")
|
||||||
),
|
),
|
||||||
row(
|
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
|
||||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
|
||||||
Pair.of("COLUMN_NAME", "cnt"),
|
|
||||||
Pair.of("DATA_TYPE", Types.BIGINT),
|
|
||||||
Pair.of("TYPE_NAME", "BIGINT"),
|
|
||||||
Pair.of("IS_NULLABLE", nullNumeric ? "YES" : "NO")
|
|
||||||
),
|
|
||||||
row(
|
row(
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||||
|
@ -687,6 +679,14 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
||||||
Pair.of("TYPE_NAME", "VARCHAR"),
|
Pair.of("TYPE_NAME", "VARCHAR"),
|
||||||
Pair.of("IS_NULLABLE", "YES")
|
Pair.of("IS_NULLABLE", "YES")
|
||||||
),
|
),
|
||||||
|
row(
|
||||||
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
|
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||||
|
Pair.of("COLUMN_NAME", "cnt"),
|
||||||
|
Pair.of("DATA_TYPE", Types.BIGINT),
|
||||||
|
Pair.of("TYPE_NAME", "BIGINT"),
|
||||||
|
Pair.of("IS_NULLABLE", nullNumeric ? "YES" : "NO")
|
||||||
|
),
|
||||||
row(
|
row(
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
Pair.of("TABLE_NAME", CalciteTests.FORBIDDEN_DATASOURCE),
|
||||||
|
@ -1181,11 +1181,6 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
||||||
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
||||||
Pair.of("COLUMN_NAME", "__time")
|
Pair.of("COLUMN_NAME", "__time")
|
||||||
),
|
),
|
||||||
row(
|
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
|
||||||
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
|
||||||
Pair.of("COLUMN_NAME", "cnt")
|
|
||||||
),
|
|
||||||
row(
|
row(
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
||||||
|
@ -1201,6 +1196,11 @@ public abstract class DruidAvaticaHandlerTest extends CalciteTestBase
|
||||||
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
||||||
Pair.of("COLUMN_NAME", "dim3")
|
Pair.of("COLUMN_NAME", "dim3")
|
||||||
),
|
),
|
||||||
|
row(
|
||||||
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
|
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
||||||
|
Pair.of("COLUMN_NAME", "cnt")
|
||||||
|
),
|
||||||
row(
|
row(
|
||||||
Pair.of("TABLE_SCHEM", "druid"),
|
Pair.of("TABLE_SCHEM", "druid"),
|
||||||
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
Pair.of("TABLE_NAME", CalciteTests.SOME_DATASOURCE),
|
||||||
|
|
|
@ -126,10 +126,10 @@ public class DruidStatementTest extends CalciteTestBase
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
Lists.newArrayList(
|
Lists.newArrayList(
|
||||||
Lists.newArrayList("__time", "TIMESTAMP", "java.lang.Long"),
|
Lists.newArrayList("__time", "TIMESTAMP", "java.lang.Long"),
|
||||||
Lists.newArrayList("cnt", "BIGINT", "java.lang.Number"),
|
|
||||||
Lists.newArrayList("dim1", "VARCHAR", "java.lang.String"),
|
Lists.newArrayList("dim1", "VARCHAR", "java.lang.String"),
|
||||||
Lists.newArrayList("dim2", "VARCHAR", "java.lang.String"),
|
Lists.newArrayList("dim2", "VARCHAR", "java.lang.String"),
|
||||||
Lists.newArrayList("dim3", "VARCHAR", "java.lang.String"),
|
Lists.newArrayList("dim3", "VARCHAR", "java.lang.String"),
|
||||||
|
Lists.newArrayList("cnt", "BIGINT", "java.lang.Number"),
|
||||||
Lists.newArrayList("m1", "FLOAT", "java.lang.Float"),
|
Lists.newArrayList("m1", "FLOAT", "java.lang.Float"),
|
||||||
Lists.newArrayList("m2", "DOUBLE", "java.lang.Double"),
|
Lists.newArrayList("m2", "DOUBLE", "java.lang.Double"),
|
||||||
Lists.newArrayList("unique_dim1", "OTHER", "java.lang.Object")
|
Lists.newArrayList("unique_dim1", "OTHER", "java.lang.Object")
|
||||||
|
@ -166,9 +166,7 @@ public class DruidStatementTest extends CalciteTestBase
|
||||||
true,
|
true,
|
||||||
Lists.newArrayList(
|
Lists.newArrayList(
|
||||||
new Object[]{""},
|
new Object[]{""},
|
||||||
new Object[]{
|
new Object[]{"1"},
|
||||||
"1"
|
|
||||||
},
|
|
||||||
new Object[]{"10.1"},
|
new Object[]{"10.1"},
|
||||||
new Object[]{"2"},
|
new Object[]{"2"},
|
||||||
new Object[]{"abc"},
|
new Object[]{"abc"},
|
||||||
|
|
|
@ -108,8 +108,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
||||||
+ ")";
|
+ ")";
|
||||||
final String legacyExplanation =
|
final String legacyExplanation =
|
||||||
"DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"list\",\"granularity\":{\"type\":\"all\"}}},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
|
"DruidOuterQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"query\",\"query\":{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"__subquery__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"list\",\"granularity\":{\"type\":\"all\"}}},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[],\"aggregations\":[{\"type\":\"count\",\"name\":\"a0\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{a0:LONG}])\n"
|
||||||
+ " DruidJoinQueryRel(condition=[=(SUBSTRING($3, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"
|
+ " DruidJoinQueryRel(condition=[=(SUBSTRING($2, 1, 1), $8)], joinType=[inner], query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"__join__\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"default\",\"dimension\":\"dim2\",\"outputName\":\"d0\",\"outputType\":\"STRING\"}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n"
|
||||||
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
|
+ " DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n"
|
||||||
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n";
|
+ " DruidQueryRel(query=[{\"queryType\":\"groupBy\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"filter\":{\"type\":\"not\",\"field\":{\"type\":\"selector\",\"dimension\":\"dim1\",\"value\":null}},\"granularity\":{\"type\":\"all\"},\"dimensions\":[{\"type\":\"extraction\",\"dimension\":\"dim1\",\"outputName\":\"d0\",\"outputType\":\"STRING\",\"extractionFn\":{\"type\":\"substring\",\"index\":0,\"length\":1}}],\"limitSpec\":{\"type\":\"NoopLimitSpec\"},\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"}}], signature=[{d0:STRING}])\n";
|
||||||
final String explanation = "["
|
final String explanation = "["
|
||||||
+ "{\"query\":{\"queryType\":\"groupBy\","
|
+ "{\"query\":{\"queryType\":\"groupBy\","
|
||||||
|
@ -153,8 +153,8 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
||||||
|
|
||||||
// Skip vectorization since otherwise the "context" will change for each subtest.
|
// Skip vectorization since otherwise the "context" will change for each subtest.
|
||||||
skipVectorize();
|
skipVectorize();
|
||||||
String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||||
String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
String legacyExplanationWithContext = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":false},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||||
String explanation = "[{"
|
String explanation = "[{"
|
||||||
+ "\"query\":{\"queryType\":\"scan\","
|
+ "\"query\":{\"queryType\":\"scan\","
|
||||||
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
||||||
|
@ -164,7 +164,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
||||||
+ "\"legacy\":false,"
|
+ "\"legacy\":false,"
|
||||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||||
+ "}]";
|
+ "}]";
|
||||||
|
|
||||||
String explanationWithContext = "[{"
|
String explanationWithContext = "[{"
|
||||||
|
@ -176,7 +176,7 @@ public class CalciteExplainQueryTest extends BaseCalciteQueryTest
|
||||||
+ "\"legacy\":false,"
|
+ "\"legacy\":false,"
|
||||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"useNativeQueryExplain\":true,\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||||
+ "}]";
|
+ "}]";
|
||||||
String sql = "EXPLAIN PLAN FOR SELECT * FROM druid.foo";
|
String sql = "EXPLAIN PLAN FOR SELECT * FROM druid.foo";
|
||||||
String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||||
|
|
|
@ -70,10 +70,10 @@ public class CalciteIngestionDmlTest extends BaseCalciteQueryTest
|
||||||
protected static final RowSignature FOO_TABLE_SIGNATURE =
|
protected static final RowSignature FOO_TABLE_SIGNATURE =
|
||||||
RowSignature.builder()
|
RowSignature.builder()
|
||||||
.addTimeColumn()
|
.addTimeColumn()
|
||||||
.add("cnt", ColumnType.LONG)
|
|
||||||
.add("dim1", ColumnType.STRING)
|
.add("dim1", ColumnType.STRING)
|
||||||
.add("dim2", ColumnType.STRING)
|
.add("dim2", ColumnType.STRING)
|
||||||
.add("dim3", ColumnType.STRING)
|
.add("dim3", ColumnType.STRING)
|
||||||
|
.add("cnt", ColumnType.LONG)
|
||||||
.add("m1", ColumnType.FLOAT)
|
.add("m1", ColumnType.FLOAT)
|
||||||
.add("m2", ColumnType.DOUBLE)
|
.add("m2", ColumnType.DOUBLE)
|
||||||
.add("unique_dim1", HyperUniquesAggregatorFactory.TYPE)
|
.add("unique_dim1", HyperUniquesAggregatorFactory.TYPE)
|
||||||
|
|
|
@ -282,10 +282,10 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
||||||
ImmutableList.of(),
|
ImmutableList.of(),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{"__time", "TIMESTAMP", "NO"},
|
new Object[]{"__time", "TIMESTAMP", "NO"},
|
||||||
new Object[]{"cnt", "BIGINT", useDefault ? "NO" : "YES"},
|
|
||||||
new Object[]{"dim1", "VARCHAR", "YES"},
|
new Object[]{"dim1", "VARCHAR", "YES"},
|
||||||
new Object[]{"dim2", "VARCHAR", "YES"},
|
new Object[]{"dim2", "VARCHAR", "YES"},
|
||||||
new Object[]{"dim3", "VARCHAR", "YES"},
|
new Object[]{"dim3", "VARCHAR", "YES"},
|
||||||
|
new Object[]{"cnt", "BIGINT", useDefault ? "NO" : "YES"},
|
||||||
new Object[]{"m1", "FLOAT", useDefault ? "NO" : "YES"},
|
new Object[]{"m1", "FLOAT", useDefault ? "NO" : "YES"},
|
||||||
new Object[]{"m2", "DOUBLE", useDefault ? "NO" : "YES"},
|
new Object[]{"m2", "DOUBLE", useDefault ? "NO" : "YES"},
|
||||||
new Object[]{"unique_dim1", "COMPLEX<hyperUnique>", "YES"}
|
new Object[]{"unique_dim1", "COMPLEX<hyperUnique>", "YES"}
|
||||||
|
@ -313,9 +313,9 @@ public class CalciteQueryTest extends BaseCalciteQueryTest
|
||||||
ImmutableList.of(),
|
ImmutableList.of(),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{"__time", "TIMESTAMP", "NO"},
|
new Object[]{"__time", "TIMESTAMP", "NO"},
|
||||||
new Object[]{"cnt", "BIGINT", useDefault ? "NO" : "YES"},
|
|
||||||
new Object[]{"dim1", "VARCHAR", "YES"},
|
new Object[]{"dim1", "VARCHAR", "YES"},
|
||||||
new Object[]{"dim2", "VARCHAR", "YES"},
|
new Object[]{"dim2", "VARCHAR", "YES"},
|
||||||
|
new Object[]{"cnt", "BIGINT", useDefault ? "NO" : "YES"},
|
||||||
new Object[]{"m1", "FLOAT", useDefault ? "NO" : "YES"},
|
new Object[]{"m1", "FLOAT", useDefault ? "NO" : "YES"},
|
||||||
new Object[]{"m2", "DOUBLE", useDefault ? "NO" : "YES"},
|
new Object[]{"m2", "DOUBLE", useDefault ? "NO" : "YES"},
|
||||||
new Object[]{"unique_dim1", "COMPLEX<hyperUnique>", "YES"}
|
new Object[]{"unique_dim1", "COMPLEX<hyperUnique>", "YES"}
|
||||||
|
|
|
@ -592,9 +592,9 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{timestamp("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1.0f, 1.0d, HLLC_STRING},
|
new Object[]{timestamp("2000-01-01"), "", "a", "[\"a\",\"b\"]", 1L, 1.0f, 1.0d, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-01"), 1L, "1", "a", "", 4.0f, 4.0d, HLLC_STRING},
|
new Object[]{timestamp("2001-01-01"), "1", "a", "", 1L, 4.0f, 4.0d, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-02"), 1L, "def", "abc", NULL_STRING, 5.0f, 5.0d, HLLC_STRING}
|
new Object[]{timestamp("2001-01-02"), "def", "abc", NULL_STRING, 1L, 5.0f, 5.0d, HLLC_STRING}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -1097,12 +1097,12 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{timestamp("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1f, 1.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-01"), "", "a", "[\"a\",\"b\"]", 1L, 1f, 1.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2000-01-02"), 1L, "10.1", NULL_STRING, "[\"b\",\"c\"]", 2f, 2.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-02"), "10.1", NULL_STRING, "[\"b\",\"c\"]", 1L, 2f, 2.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2000-01-03"), 1L, "2", "", "d", 3f, 3.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-03"), "2", "", "d", 1L, 3f, 3.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-01"), 1L, "1", "a", "", 4f, 4.0, HLLC_STRING},
|
new Object[]{timestamp("2001-01-01"), "1", "a", "", 1L, 4f, 4.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-02"), 1L, "def", "abc", NULL_STRING, 5f, 5.0, HLLC_STRING},
|
new Object[]{timestamp("2001-01-02"), "def", "abc", NULL_STRING, 1L, 5f, 5.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-03"), 1L, "abc", NULL_STRING, NULL_STRING, 6f, 6.0, HLLC_STRING}
|
new Object[]{timestamp("2001-01-03"), "abc", NULL_STRING, NULL_STRING, 1L, 6f, 6.0, HLLC_STRING}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -1131,18 +1131,18 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{
|
new Object[]{
|
||||||
timestamp("2000-01-01"),
|
timestamp("2000-01-01"),
|
||||||
1L,
|
|
||||||
"forbidden",
|
"forbidden",
|
||||||
"abcd",
|
"abcd",
|
||||||
|
1L,
|
||||||
9999.0f,
|
9999.0f,
|
||||||
NullHandling.defaultDoubleValue(),
|
NullHandling.defaultDoubleValue(),
|
||||||
"\"AQAAAQAAAALFBA==\""
|
"\"AQAAAQAAAALFBA==\""
|
||||||
},
|
},
|
||||||
new Object[]{
|
new Object[]{
|
||||||
timestamp("2000-01-02"),
|
timestamp("2000-01-02"),
|
||||||
1L,
|
|
||||||
"forbidden",
|
"forbidden",
|
||||||
"a",
|
"a",
|
||||||
|
1L,
|
||||||
1234.0f,
|
1234.0f,
|
||||||
NullHandling.defaultDoubleValue(),
|
NullHandling.defaultDoubleValue(),
|
||||||
"\"AQAAAQAAAALFBA==\""
|
"\"AQAAAQAAAALFBA==\""
|
||||||
|
@ -1271,7 +1271,7 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
skipVectorize();
|
skipVectorize();
|
||||||
|
|
||||||
final String query = "EXPLAIN PLAN FOR SELECT * FROM druid.foo";
|
final String query = "EXPLAIN PLAN FOR SELECT * FROM druid.foo";
|
||||||
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, cnt:LONG, dim1:STRING, dim2:STRING, dim3:STRING, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
final String legacyExplanation = "DruidQueryRel(query=[{\"queryType\":\"scan\",\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},\"intervals\":{\"type\":\"intervals\",\"intervals\":[\"-146136543-09-08T08:23:32.096Z/146140482-04-24T15:36:27.903Z\"]},\"resultFormat\":\"compactedList\",\"columns\":[\"__time\",\"cnt\",\"dim1\",\"dim2\",\"dim3\",\"m1\",\"m2\",\"unique_dim1\"],\"legacy\":false,\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},\"granularity\":{\"type\":\"all\"}}], signature=[{__time:LONG, dim1:STRING, dim2:STRING, dim3:STRING, cnt:LONG, m1:FLOAT, m2:DOUBLE, unique_dim1:COMPLEX<hyperUnique>}])\n";
|
||||||
final String explanation = "[{"
|
final String explanation = "[{"
|
||||||
+ "\"query\":{\"queryType\":\"scan\","
|
+ "\"query\":{\"queryType\":\"scan\","
|
||||||
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
+ "\"dataSource\":{\"type\":\"table\",\"name\":\"foo\"},"
|
||||||
|
@ -1281,7 +1281,7 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
+ "\"legacy\":false,"
|
+ "\"legacy\":false,"
|
||||||
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
+ "\"context\":{\"defaultTimeout\":300000,\"maxScatterGatherBytes\":9223372036854775807,\"sqlCurrentTimestamp\":\"2000-01-01T00:00:00Z\",\"sqlQueryId\":\"dummy\",\"vectorize\":\"false\",\"vectorizeVirtualColumns\":\"false\"},"
|
||||||
+ "\"granularity\":{\"type\":\"all\"}},"
|
+ "\"granularity\":{\"type\":\"all\"}},"
|
||||||
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
+ "\"signature\":[{\"name\":\"__time\",\"type\":\"LONG\"},{\"name\":\"dim1\",\"type\":\"STRING\"},{\"name\":\"dim2\",\"type\":\"STRING\"},{\"name\":\"dim3\",\"type\":\"STRING\"},{\"name\":\"cnt\",\"type\":\"LONG\"},{\"name\":\"m1\",\"type\":\"FLOAT\"},{\"name\":\"m2\",\"type\":\"DOUBLE\"},{\"name\":\"unique_dim1\",\"type\":\"COMPLEX<hyperUnique>\"}]"
|
||||||
+ "}]";
|
+ "}]";
|
||||||
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
final String resources = "[{\"name\":\"foo\",\"type\":\"DATASOURCE\"}]";
|
||||||
|
|
||||||
|
@ -1328,8 +1328,8 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{timestamp("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1.0f, 1.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-01"), "", "a", "[\"a\",\"b\"]", 1L, 1.0f, 1.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2000-01-02"), 1L, "10.1", NULL_STRING, "[\"b\",\"c\"]", 2.0f, 2.0, HLLC_STRING}
|
new Object[]{timestamp("2000-01-02"), "10.1", NULL_STRING, "[\"b\",\"c\"]", 1L, 2.0f, 2.0, HLLC_STRING}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -1354,8 +1354,8 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{timestamp("2000-01-02"), 1L, "10.1", NULL_STRING, "[\"b\",\"c\"]", 2.0f, 2.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-02"), "10.1", NULL_STRING, "[\"b\",\"c\"]", 1L, 2.0f, 2.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2000-01-03"), 1L, "2", "", "d", 3f, 3.0, HLLC_STRING}
|
new Object[]{timestamp("2000-01-03"), "2", "", "d", 1L, 3f, 3.0, HLLC_STRING}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -1429,8 +1429,8 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{timestamp("2001-01-03"), 1L, "abc", NULL_STRING, NULL_STRING, 6f, 6d, HLLC_STRING},
|
new Object[]{timestamp("2001-01-03"), "abc", NULL_STRING, NULL_STRING, 1L, 6f, 6d, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-02"), 1L, "def", "abc", NULL_STRING, 5f, 5d, HLLC_STRING}
|
new Object[]{timestamp("2001-01-02"), "def", "abc", NULL_STRING, 1L, 5f, 5d, HLLC_STRING}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -1455,12 +1455,12 @@ public class CalciteSelectQueryTest extends BaseCalciteQueryTest
|
||||||
.build()
|
.build()
|
||||||
),
|
),
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
new Object[]{timestamp("2000-01-01"), 1L, "", "a", "[\"a\",\"b\"]", 1f, 1.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-01"), "", "a", "[\"a\",\"b\"]", 1L, 1f, 1.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2000-01-02"), 1L, "10.1", NULL_STRING, "[\"b\",\"c\"]", 2f, 2.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-02"), "10.1", NULL_STRING, "[\"b\",\"c\"]", 1L, 2f, 2.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2000-01-03"), 1L, "2", "", "d", 3f, 3.0, HLLC_STRING},
|
new Object[]{timestamp("2000-01-03"), "2", "", "d", 1L, 3f, 3.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-01"), 1L, "1", "a", "", 4f, 4.0, HLLC_STRING},
|
new Object[]{timestamp("2001-01-01"), "1", "a", "", 1L, 4f, 4.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-02"), 1L, "def", "abc", NULL_STRING, 5f, 5.0, HLLC_STRING},
|
new Object[]{timestamp("2001-01-02"), "def", "abc", NULL_STRING, 1L, 5f, 5.0, HLLC_STRING},
|
||||||
new Object[]{timestamp("2001-01-03"), 1L, "abc", NULL_STRING, NULL_STRING, 6f, 6.0, HLLC_STRING}
|
new Object[]{timestamp("2001-01-03"), "abc", NULL_STRING, NULL_STRING, 1L, 6f, 6.0, HLLC_STRING}
|
||||||
)
|
)
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
|
@ -78,6 +78,7 @@ import java.io.File;
|
||||||
import java.io.IOException;
|
import java.io.IOException;
|
||||||
import java.util.EnumSet;
|
import java.util.EnumSet;
|
||||||
import java.util.HashSet;
|
import java.util.HashSet;
|
||||||
|
import java.util.LinkedHashMap;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
import java.util.Set;
|
import java.util.Set;
|
||||||
|
@ -206,7 +207,10 @@ public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||||
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
CalciteTests.createMockQueryLifecycleFactory(walker, conglomerate),
|
||||||
serverView,
|
serverView,
|
||||||
segmentManager,
|
segmentManager,
|
||||||
new MapJoinableFactory(ImmutableSet.of(globalTableJoinable), ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)),
|
new MapJoinableFactory(
|
||||||
|
ImmutableSet.of(globalTableJoinable),
|
||||||
|
ImmutableMap.of(globalTableJoinable.getClass(), GlobalTableDataSource.class)
|
||||||
|
),
|
||||||
PLANNER_CONFIG_DEFAULT,
|
PLANNER_CONFIG_DEFAULT,
|
||||||
new NoopEscalator(),
|
new NoopEscalator(),
|
||||||
new BrokerInternalQueryConfig(),
|
new BrokerInternalQueryConfig(),
|
||||||
|
@ -288,16 +292,16 @@ public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||||
Assert.assertEquals("__time", fields.get(0).getName());
|
Assert.assertEquals("__time", fields.get(0).getName());
|
||||||
Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName());
|
Assert.assertEquals(SqlTypeName.TIMESTAMP, fields.get(0).getType().getSqlTypeName());
|
||||||
|
|
||||||
Assert.assertEquals("cnt", fields.get(1).getName());
|
Assert.assertEquals("dim2", fields.get(1).getName());
|
||||||
Assert.assertEquals(SqlTypeName.BIGINT, fields.get(1).getType().getSqlTypeName());
|
Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(1).getType().getSqlTypeName());
|
||||||
|
|
||||||
Assert.assertEquals("dim1", fields.get(2).getName());
|
Assert.assertEquals("m1", fields.get(2).getName());
|
||||||
Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(2).getType().getSqlTypeName());
|
Assert.assertEquals(SqlTypeName.BIGINT, fields.get(2).getType().getSqlTypeName());
|
||||||
|
|
||||||
Assert.assertEquals("dim2", fields.get(3).getName());
|
Assert.assertEquals("dim1", fields.get(3).getName());
|
||||||
Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getSqlTypeName());
|
Assert.assertEquals(SqlTypeName.VARCHAR, fields.get(3).getType().getSqlTypeName());
|
||||||
|
|
||||||
Assert.assertEquals("m1", fields.get(4).getName());
|
Assert.assertEquals("cnt", fields.get(4).getName());
|
||||||
Assert.assertEquals(SqlTypeName.BIGINT, fields.get(4).getType().getSqlTypeName());
|
Assert.assertEquals(SqlTypeName.BIGINT, fields.get(4).getType().getSqlTypeName());
|
||||||
|
|
||||||
Assert.assertEquals("unique_dim1", fields.get(5).getName());
|
Assert.assertEquals("unique_dim1", fields.get(5).getName());
|
||||||
|
@ -1065,7 +1069,7 @@ public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||||
new TableDataSource(segment.getDataSource()),
|
new TableDataSource(segment.getDataSource()),
|
||||||
new MultipleSpecificSegmentSpec(
|
new MultipleSpecificSegmentSpec(
|
||||||
segmentIterable.stream()
|
segmentIterable.stream()
|
||||||
.map(SegmentId::toDescriptor).collect(Collectors.toList())),
|
.map(SegmentId::toDescriptor).collect(Collectors.toList())),
|
||||||
new AllColumnIncluderator(),
|
new AllColumnIncluderator(),
|
||||||
false,
|
false,
|
||||||
queryContext,
|
queryContext,
|
||||||
|
@ -1094,7 +1098,8 @@ public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||||
|
|
||||||
EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once();
|
EasyMock.expect(factoryMock.factorize()).andReturn(lifecycleMock).once();
|
||||||
// This is the mat of the test, making sure that the query created by the method under test matches the expected query, specifically the operator configured context
|
// This is the mat of the test, making sure that the query created by the method under test matches the expected query, specifically the operator configured context
|
||||||
EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK)).andReturn(null);
|
EasyMock.expect(lifecycleMock.runSimple(expectedMetadataQuery, AllowAllAuthenticator.ALLOW_ALL_RESULT, Access.OK))
|
||||||
|
.andReturn(null);
|
||||||
|
|
||||||
EasyMock.replay(factoryMock, lifecycleMock);
|
EasyMock.replay(factoryMock, lifecycleMock);
|
||||||
|
|
||||||
|
@ -1107,36 +1112,28 @@ public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||||
@Test
|
@Test
|
||||||
public void testSegmentMetadataColumnType()
|
public void testSegmentMetadataColumnType()
|
||||||
{
|
{
|
||||||
|
// Verify order is preserved.
|
||||||
|
final LinkedHashMap<String, ColumnAnalysis> columns = new LinkedHashMap<>();
|
||||||
|
columns.put(
|
||||||
|
"a",
|
||||||
|
new ColumnAnalysis(ColumnType.STRING, ColumnType.STRING.asTypeString(), false, true, 1234, 26, "a", "z", null)
|
||||||
|
);
|
||||||
|
|
||||||
|
columns.put(
|
||||||
|
"count",
|
||||||
|
new ColumnAnalysis(ColumnType.LONG, ColumnType.LONG.asTypeString(), false, true, 1234, 26, "a", "z", null)
|
||||||
|
);
|
||||||
|
|
||||||
|
columns.put(
|
||||||
|
"b",
|
||||||
|
new ColumnAnalysis(ColumnType.DOUBLE, ColumnType.DOUBLE.asTypeString(), false, true, 1234, 26, null, null, null)
|
||||||
|
);
|
||||||
|
|
||||||
RowSignature signature = DruidSchema.analysisToRowSignature(
|
RowSignature signature = DruidSchema.analysisToRowSignature(
|
||||||
new SegmentAnalysis(
|
new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
ImmutableList.of(Intervals.utc(1L, 2L)),
|
ImmutableList.of(Intervals.utc(1L, 2L)),
|
||||||
ImmutableMap.of(
|
columns,
|
||||||
"a",
|
|
||||||
new ColumnAnalysis(
|
|
||||||
ColumnType.STRING,
|
|
||||||
ColumnType.STRING.asTypeString(),
|
|
||||||
false,
|
|
||||||
true,
|
|
||||||
1234,
|
|
||||||
26,
|
|
||||||
"a",
|
|
||||||
"z",
|
|
||||||
null
|
|
||||||
),
|
|
||||||
"count",
|
|
||||||
new ColumnAnalysis(
|
|
||||||
ColumnType.LONG,
|
|
||||||
ColumnType.LONG.asTypeString(),
|
|
||||||
false,
|
|
||||||
true,
|
|
||||||
1234,
|
|
||||||
26,
|
|
||||||
"a",
|
|
||||||
"z",
|
|
||||||
null
|
|
||||||
)
|
|
||||||
),
|
|
||||||
1234,
|
1234,
|
||||||
100,
|
100,
|
||||||
null,
|
null,
|
||||||
|
@ -1147,7 +1144,11 @@ public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||||
);
|
);
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
RowSignature.builder().add("a", ColumnType.STRING).add("count", ColumnType.LONG).build(),
|
RowSignature.builder()
|
||||||
|
.add("a", ColumnType.STRING)
|
||||||
|
.add("count", ColumnType.LONG)
|
||||||
|
.add("b", ColumnType.DOUBLE)
|
||||||
|
.build(),
|
||||||
signature
|
signature
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
@ -1160,30 +1161,32 @@ public class DruidSchemaTest extends DruidSchemaTestCommon
|
||||||
new SegmentAnalysis(
|
new SegmentAnalysis(
|
||||||
"id",
|
"id",
|
||||||
ImmutableList.of(Intervals.utc(1L, 2L)),
|
ImmutableList.of(Intervals.utc(1L, 2L)),
|
||||||
ImmutableMap.of(
|
new LinkedHashMap<>(
|
||||||
"a",
|
ImmutableMap.of(
|
||||||
new ColumnAnalysis(
|
|
||||||
null,
|
|
||||||
ColumnType.STRING.asTypeString(),
|
|
||||||
false,
|
|
||||||
true,
|
|
||||||
1234,
|
|
||||||
26,
|
|
||||||
"a",
|
"a",
|
||||||
"z",
|
new ColumnAnalysis(
|
||||||
null
|
null,
|
||||||
),
|
ColumnType.STRING.asTypeString(),
|
||||||
"count",
|
false,
|
||||||
new ColumnAnalysis(
|
true,
|
||||||
null,
|
1234,
|
||||||
ColumnType.LONG.asTypeString(),
|
26,
|
||||||
false,
|
"a",
|
||||||
true,
|
"z",
|
||||||
1234,
|
null
|
||||||
26,
|
),
|
||||||
"a",
|
"count",
|
||||||
"z",
|
new ColumnAnalysis(
|
||||||
null
|
null,
|
||||||
|
ColumnType.LONG.asTypeString(),
|
||||||
|
false,
|
||||||
|
true,
|
||||||
|
1234,
|
||||||
|
26,
|
||||||
|
"a",
|
||||||
|
"z",
|
||||||
|
null
|
||||||
|
)
|
||||||
)
|
)
|
||||||
),
|
),
|
||||||
1234,
|
1234,
|
||||||
|
|
|
@ -123,13 +123,13 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
private static final String DUMMY_SQL_QUERY_ID = "dummy";
|
private static final String DUMMY_SQL_QUERY_ID = "dummy";
|
||||||
|
|
||||||
private static final List<String> EXPECTED_COLUMNS_FOR_RESULT_FORMAT_TESTS =
|
private static final List<String> EXPECTED_COLUMNS_FOR_RESULT_FORMAT_TESTS =
|
||||||
Arrays.asList("__time", "cnt", "dim1", "dim2", "dim3", "m1", "m2", "unique_dim1", "EXPR$8");
|
Arrays.asList("__time", "dim1", "dim2", "dim3", "cnt", "m1", "m2", "unique_dim1", "EXPR$8");
|
||||||
|
|
||||||
private static final List<String> EXPECTED_TYPES_FOR_RESULT_FORMAT_TESTS =
|
private static final List<String> EXPECTED_TYPES_FOR_RESULT_FORMAT_TESTS =
|
||||||
Arrays.asList("LONG", "LONG", "STRING", "STRING", "STRING", "FLOAT", "DOUBLE", "COMPLEX<hyperUnique>", "STRING");
|
Arrays.asList("LONG", "STRING", "STRING", "STRING", "LONG", "FLOAT", "DOUBLE", "COMPLEX<hyperUnique>", "STRING");
|
||||||
|
|
||||||
private static final List<String> EXPECTED_SQL_TYPES_FOR_RESULT_FORMAT_TESTS =
|
private static final List<String> EXPECTED_SQL_TYPES_FOR_RESULT_FORMAT_TESTS =
|
||||||
Arrays.asList("TIMESTAMP", "BIGINT", "VARCHAR", "VARCHAR", "VARCHAR", "FLOAT", "DOUBLE", "OTHER", "VARCHAR");
|
Arrays.asList("TIMESTAMP", "VARCHAR", "VARCHAR", "VARCHAR", "BIGINT", "FLOAT", "DOUBLE", "OTHER", "VARCHAR");
|
||||||
|
|
||||||
private static QueryRunnerFactoryConglomerate conglomerate;
|
private static QueryRunnerFactoryConglomerate conglomerate;
|
||||||
private static Closer resourceCloser;
|
private static Closer resourceCloser;
|
||||||
|
@ -544,10 +544,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-01T00:00:00.000Z",
|
"2000-01-01T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"",
|
"",
|
||||||
"a",
|
"a",
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
1,
|
||||||
1.0,
|
1.0,
|
||||||
1.0,
|
1.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -555,10 +555,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
),
|
),
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-02T00:00:00.000Z",
|
"2000-01-02T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"10.1",
|
"10.1",
|
||||||
nullStr,
|
nullStr,
|
||||||
"[\"b\",\"c\"]",
|
"[\"b\",\"c\"]",
|
||||||
|
1,
|
||||||
2.0,
|
2.0,
|
||||||
2.0,
|
2.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -655,10 +655,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
EXPECTED_SQL_TYPES_FOR_RESULT_FORMAT_TESTS,
|
EXPECTED_SQL_TYPES_FOR_RESULT_FORMAT_TESTS,
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-01T00:00:00.000Z",
|
"2000-01-01T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"",
|
"",
|
||||||
"a",
|
"a",
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
1,
|
||||||
1.0,
|
1.0,
|
||||||
1.0,
|
1.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -666,10 +666,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
),
|
),
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-02T00:00:00.000Z",
|
"2000-01-02T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"10.1",
|
"10.1",
|
||||||
nullStr,
|
nullStr,
|
||||||
"[\"b\",\"c\"]",
|
"[\"b\",\"c\"]",
|
||||||
|
1,
|
||||||
2.0,
|
2.0,
|
||||||
2.0,
|
2.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -723,10 +723,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-01T00:00:00.000Z",
|
"2000-01-01T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"",
|
"",
|
||||||
"a",
|
"a",
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
1,
|
||||||
1.0,
|
1.0,
|
||||||
1.0,
|
1.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -737,10 +737,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-02T00:00:00.000Z",
|
"2000-01-02T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"10.1",
|
"10.1",
|
||||||
nullStr,
|
nullStr,
|
||||||
"[\"b\",\"c\"]",
|
"[\"b\",\"c\"]",
|
||||||
|
1,
|
||||||
2.0,
|
2.0,
|
||||||
2.0,
|
2.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -771,10 +771,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-01T00:00:00.000Z",
|
"2000-01-01T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"",
|
"",
|
||||||
"a",
|
"a",
|
||||||
"[\"a\",\"b\"]",
|
"[\"a\",\"b\"]",
|
||||||
|
1,
|
||||||
1.0,
|
1.0,
|
||||||
1.0,
|
1.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -785,10 +785,10 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
Arrays.asList(
|
Arrays.asList(
|
||||||
"2000-01-02T00:00:00.000Z",
|
"2000-01-02T00:00:00.000Z",
|
||||||
1,
|
|
||||||
"10.1",
|
"10.1",
|
||||||
nullStr,
|
nullStr,
|
||||||
"[\"b\",\"c\"]",
|
"[\"b\",\"c\"]",
|
||||||
|
1,
|
||||||
2.0,
|
2.0,
|
||||||
2.0,
|
2.0,
|
||||||
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
"org.apache.druid.hll.VersionOneHyperLogLogCollector",
|
||||||
|
@ -1099,8 +1099,8 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
|
|
||||||
Assert.assertEquals(
|
Assert.assertEquals(
|
||||||
ImmutableList.of(
|
ImmutableList.of(
|
||||||
"2000-01-01T00:00:00.000Z,1,,a,\"[\"\"a\"\",\"\"b\"\"]\",1.0,1.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
"2000-01-01T00:00:00.000Z,,a,\"[\"\"a\"\",\"\"b\"\"]\",1,1.0,1.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
||||||
"2000-01-02T00:00:00.000Z,1,10.1,,\"[\"\"b\"\",\"\"c\"\"]\",2.0,2.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
"2000-01-02T00:00:00.000Z,10.1,,\"[\"\"b\"\",\"\"c\"\"]\",1,2.0,2.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
||||||
"",
|
"",
|
||||||
""
|
""
|
||||||
),
|
),
|
||||||
|
@ -1124,8 +1124,8 @@ public class SqlResourceTest extends CalciteTestBase
|
||||||
String.join(",", EXPECTED_COLUMNS_FOR_RESULT_FORMAT_TESTS),
|
String.join(",", EXPECTED_COLUMNS_FOR_RESULT_FORMAT_TESTS),
|
||||||
String.join(",", EXPECTED_TYPES_FOR_RESULT_FORMAT_TESTS),
|
String.join(",", EXPECTED_TYPES_FOR_RESULT_FORMAT_TESTS),
|
||||||
String.join(",", EXPECTED_SQL_TYPES_FOR_RESULT_FORMAT_TESTS),
|
String.join(",", EXPECTED_SQL_TYPES_FOR_RESULT_FORMAT_TESTS),
|
||||||
"2000-01-01T00:00:00.000Z,1,,a,\"[\"\"a\"\",\"\"b\"\"]\",1.0,1.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
"2000-01-01T00:00:00.000Z,,a,\"[\"\"a\"\",\"\"b\"\"]\",1,1.0,1.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
||||||
"2000-01-02T00:00:00.000Z,1,10.1,,\"[\"\"b\"\",\"\"c\"\"]\",2.0,2.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
"2000-01-02T00:00:00.000Z,10.1,,\"[\"\"b\"\",\"\"c\"\"]\",1,2.0,2.0,org.apache.druid.hll.VersionOneHyperLogLogCollector,",
|
||||||
"",
|
"",
|
||||||
""
|
""
|
||||||
),
|
),
|
||||||
|
|
Loading…
Reference in New Issue