mirror of https://github.com/apache/druid.git
JOIN or UNNEST queries over tombstone segment can fail (#14021)
Join,Unnest queries over tombstone segment can fail
This commit is contained in:
parent
b11c0bc249
commit
92912a6a2b
|
@ -66,9 +66,6 @@ public class ScanQueryEngine
|
||||||
@Nullable final QueryMetrics<?> queryMetrics
|
@Nullable final QueryMetrics<?> queryMetrics
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
if (segment.asQueryableIndex() != null && segment.asQueryableIndex().isFromTombstone()) {
|
|
||||||
return Sequences.empty();
|
|
||||||
}
|
|
||||||
|
|
||||||
// "legacy" should be non-null due to toolChest.mergeResults
|
// "legacy" should be non-null due to toolChest.mergeResults
|
||||||
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");
|
final boolean legacy = Preconditions.checkNotNull(query.isLegacy(), "Expected non-null 'legacy' parameter");
|
||||||
|
@ -87,6 +84,10 @@ public class ScanQueryEngine
|
||||||
);
|
);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
if (adapter.isFromTombstone()) {
|
||||||
|
return Sequences.empty();
|
||||||
|
}
|
||||||
|
|
||||||
final List<String> allColumns = new ArrayList<>();
|
final List<String> allColumns = new ArrayList<>();
|
||||||
|
|
||||||
if (query.getColumns() != null && !query.getColumns().isEmpty()) {
|
if (query.getColumns() != null && !query.getColumns().isEmpty()) {
|
||||||
|
|
|
@ -70,12 +70,4 @@ public interface QueryableIndex extends Closeable, ColumnInspector
|
||||||
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed
|
//@Deprecated // This is still required for SimpleQueryableIndex. It should not go away until SimpleQueryableIndex is fixed
|
||||||
@Override
|
@Override
|
||||||
void close();
|
void close();
|
||||||
|
|
||||||
/**
|
|
||||||
* @return true if this index was created from a tombstone or false otherwise
|
|
||||||
*/
|
|
||||||
default boolean isFromTombstone()
|
|
||||||
{
|
|
||||||
return false;
|
|
||||||
}
|
|
||||||
}
|
}
|
||||||
|
|
|
@ -119,4 +119,12 @@ public interface StorageAdapter extends CursorFactory, ColumnInspector
|
||||||
{
|
{
|
||||||
return false;
|
return false;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
/**
|
||||||
|
* @return true if this index was created from a tombstone or false otherwise
|
||||||
|
*/
|
||||||
|
default boolean isFromTombstone()
|
||||||
|
{
|
||||||
|
return false;
|
||||||
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -36,6 +36,7 @@ import org.apache.druid.timeline.SegmentId;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
|
|
||||||
import java.io.File;
|
import java.io.File;
|
||||||
import java.util.List;
|
import java.util.List;
|
||||||
import java.util.Map;
|
import java.util.Map;
|
||||||
|
@ -120,15 +121,16 @@ public class TombstoneSegmentizerFactory implements SegmentizerFactory
|
||||||
throw new UnsupportedOperationException();
|
throw new UnsupportedOperationException();
|
||||||
}
|
}
|
||||||
|
|
||||||
// mark this index to indicate that it comes from a tombstone:
|
|
||||||
@Override
|
|
||||||
public boolean isFromTombstone()
|
|
||||||
{
|
|
||||||
return true;
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex);
|
final QueryableIndexStorageAdapter storageAdapter = new QueryableIndexStorageAdapter(queryableIndex)
|
||||||
|
{
|
||||||
|
@Override
|
||||||
|
public boolean isFromTombstone()
|
||||||
|
{
|
||||||
|
return true;
|
||||||
|
}
|
||||||
|
};
|
||||||
|
|
||||||
Segment segmentObject = new Segment()
|
Segment segmentObject = new Segment()
|
||||||
{
|
{
|
||||||
|
|
|
@ -19,27 +19,41 @@
|
||||||
|
|
||||||
package org.apache.druid.segment;
|
package org.apache.druid.segment;
|
||||||
|
|
||||||
import org.apache.druid.collections.bitmap.BitmapFactory;
|
import org.apache.druid.java.util.common.granularity.Granularity;
|
||||||
import org.apache.druid.segment.column.ColumnHolder;
|
import org.apache.druid.java.util.common.guava.Sequence;
|
||||||
|
import org.apache.druid.query.QueryMetrics;
|
||||||
|
import org.apache.druid.query.filter.Filter;
|
||||||
|
import org.apache.druid.segment.column.ColumnCapabilities;
|
||||||
import org.apache.druid.segment.data.Indexed;
|
import org.apache.druid.segment.data.Indexed;
|
||||||
|
import org.joda.time.DateTime;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
import org.junit.Assert;
|
import org.junit.Assert;
|
||||||
import org.junit.Test;
|
import org.junit.Test;
|
||||||
|
|
||||||
import javax.annotation.Nullable;
|
import javax.annotation.Nullable;
|
||||||
import java.util.List;
|
|
||||||
import java.util.Map;
|
|
||||||
|
|
||||||
|
|
||||||
public class SimpleQueryableIndexTest
|
public class TombstoneSegmentStorageAdapterTest
|
||||||
{
|
{
|
||||||
@Test
|
@Test
|
||||||
public void testTombstoneDefaultInterface()
|
public void testTombstoneDefaultInterface()
|
||||||
{
|
{
|
||||||
QueryableIndex qi = new QueryableIndex()
|
StorageAdapter sa = new StorageAdapter()
|
||||||
{
|
{
|
||||||
@Override
|
@Override
|
||||||
public Interval getDataInterval()
|
public Sequence<Cursor> makeCursors(
|
||||||
|
@Nullable Filter filter,
|
||||||
|
Interval interval,
|
||||||
|
VirtualColumns virtualColumns,
|
||||||
|
Granularity gran,
|
||||||
|
boolean descending,
|
||||||
|
@Nullable QueryMetrics<?> queryMetrics)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public Interval getInterval()
|
||||||
{
|
{
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -50,6 +64,12 @@ public class SimpleQueryableIndexTest
|
||||||
return 0;
|
return 0;
|
||||||
}
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DateTime getMaxIngestedEventTime()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public Indexed<String> getAvailableDimensions()
|
public Indexed<String> getAvailableDimensions()
|
||||||
{
|
{
|
||||||
|
@ -57,7 +77,46 @@ public class SimpleQueryableIndexTest
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
@Override
|
||||||
public BitmapFactory getBitmapFactoryForDimensions()
|
public Iterable<String> getAvailableMetrics()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public int getDimensionCardinality(String column)
|
||||||
|
{
|
||||||
|
return 0;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DateTime getMinTime()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Override
|
||||||
|
public DateTime getMaxTime()
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public Comparable getMinValue(String column)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public Comparable getMaxValue(String column)
|
||||||
|
{
|
||||||
|
return null;
|
||||||
|
}
|
||||||
|
|
||||||
|
@Nullable
|
||||||
|
@Override
|
||||||
|
public ColumnCapabilities getColumnCapabilities(String column)
|
||||||
{
|
{
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
@ -69,33 +128,9 @@ public class SimpleQueryableIndexTest
|
||||||
return null;
|
return null;
|
||||||
}
|
}
|
||||||
|
|
||||||
@Override
|
|
||||||
public Map<String, DimensionHandler> getDimensionHandlers()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public void close()
|
|
||||||
{
|
|
||||||
|
|
||||||
}
|
|
||||||
|
|
||||||
@Override
|
|
||||||
public List<String> getColumnNames()
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
|
|
||||||
@Nullable
|
|
||||||
@Override
|
|
||||||
public ColumnHolder getColumnHolder(String columnName)
|
|
||||||
{
|
|
||||||
return null;
|
|
||||||
}
|
|
||||||
};
|
};
|
||||||
|
|
||||||
Assert.assertFalse(qi.isFromTombstone());
|
Assert.assertFalse(sa.isFromTombstone());
|
||||||
}
|
}
|
||||||
|
|
||||||
}
|
}
|
|
@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableMap;
|
||||||
import org.apache.druid.java.util.common.Intervals;
|
import org.apache.druid.java.util.common.Intervals;
|
||||||
import org.apache.druid.segment.QueryableIndex;
|
import org.apache.druid.segment.QueryableIndex;
|
||||||
import org.apache.druid.segment.Segment;
|
import org.apache.druid.segment.Segment;
|
||||||
|
import org.apache.druid.segment.StorageAdapter;
|
||||||
import org.apache.druid.timeline.DataSegment;
|
import org.apache.druid.timeline.DataSegment;
|
||||||
import org.apache.druid.timeline.partition.TombstoneShardSpec;
|
import org.apache.druid.timeline.partition.TombstoneShardSpec;
|
||||||
import org.joda.time.Interval;
|
import org.joda.time.Interval;
|
||||||
|
@ -55,14 +56,17 @@ public class TombstoneSegmentizerFactoryTest
|
||||||
|
|
||||||
QueryableIndex queryableIndex = segment.asQueryableIndex();
|
QueryableIndex queryableIndex = segment.asQueryableIndex();
|
||||||
Assert.assertNotNull(queryableIndex);
|
Assert.assertNotNull(queryableIndex);
|
||||||
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getNumRows());
|
assertThrows(UnsupportedOperationException.class, queryableIndex::getNumRows);
|
||||||
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getAvailableDimensions());
|
assertThrows(UnsupportedOperationException.class, queryableIndex::getAvailableDimensions);
|
||||||
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getBitmapFactoryForDimensions());
|
assertThrows(UnsupportedOperationException.class, queryableIndex::getBitmapFactoryForDimensions);
|
||||||
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getMetadata());
|
assertThrows(UnsupportedOperationException.class, queryableIndex::getMetadata);
|
||||||
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getDimensionHandlers());
|
assertThrows(UnsupportedOperationException.class, queryableIndex::getDimensionHandlers);
|
||||||
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnNames());
|
assertThrows(UnsupportedOperationException.class, queryableIndex::getColumnNames);
|
||||||
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnHolder(null));
|
assertThrows(UnsupportedOperationException.class, () -> queryableIndex.getColumnHolder(null));
|
||||||
Assert.assertTrue(queryableIndex.isFromTombstone());
|
|
||||||
|
StorageAdapter storageAdapter = segment.asStorageAdapter();
|
||||||
|
Assert.assertNotNull(storageAdapter);
|
||||||
|
Assert.assertTrue(storageAdapter.isFromTombstone());
|
||||||
|
|
||||||
}
|
}
|
||||||
}
|
}
|
||||||
|
|
|
@ -275,11 +275,7 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
)
|
)
|
||||||
{
|
{
|
||||||
|
|
||||||
// Short-circuit when the index comes from a tombstone (it has no data by definition),
|
|
||||||
// check for null also since no all segments (higher level ones) will have QueryableIndex...
|
|
||||||
if (segment.asQueryableIndex() != null && segment.asQueryableIndex().isFromTombstone()) {
|
|
||||||
return new NoopQueryRunner<>();
|
|
||||||
}
|
|
||||||
|
|
||||||
final SpecificSegmentSpec segmentSpec = new SpecificSegmentSpec(segmentDescriptor);
|
final SpecificSegmentSpec segmentSpec = new SpecificSegmentSpec(segmentDescriptor);
|
||||||
final SegmentId segmentId = segment.getId();
|
final SegmentId segmentId = segment.getId();
|
||||||
|
@ -290,6 +286,13 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
if (segmentId == null || segmentInterval == null) {
|
if (segmentId == null || segmentInterval == null) {
|
||||||
return new ReportTimelineMissingSegmentQueryRunner<>(segmentDescriptor);
|
return new ReportTimelineMissingSegmentQueryRunner<>(segmentDescriptor);
|
||||||
}
|
}
|
||||||
|
|
||||||
|
StorageAdapter storageAdapter = segment.asStorageAdapter();
|
||||||
|
// Short-circuit when the index comes from a tombstone (it has no data by definition),
|
||||||
|
// check for null also since no all segments (higher level ones) will have QueryableIndex...
|
||||||
|
if (storageAdapter.isFromTombstone()) {
|
||||||
|
return new NoopQueryRunner<>();
|
||||||
|
}
|
||||||
String segmentIdString = segmentId.toString();
|
String segmentIdString = segmentId.toString();
|
||||||
|
|
||||||
MetricsEmittingQueryRunner<T> metricsEmittingQueryRunnerInner = new MetricsEmittingQueryRunner<>(
|
MetricsEmittingQueryRunner<T> metricsEmittingQueryRunnerInner = new MetricsEmittingQueryRunner<>(
|
||||||
|
@ -300,7 +303,6 @@ public class ServerManager implements QuerySegmentWalker
|
||||||
queryMetrics -> queryMetrics.segment(segmentIdString)
|
queryMetrics -> queryMetrics.segment(segmentIdString)
|
||||||
);
|
);
|
||||||
|
|
||||||
StorageAdapter storageAdapter = segment.asStorageAdapter();
|
|
||||||
long segmentMaxTime = storageAdapter.getMaxTime().getMillis();
|
long segmentMaxTime = storageAdapter.getMaxTime().getMillis();
|
||||||
long segmentMinTime = storageAdapter.getMinTime().getMillis();
|
long segmentMinTime = storageAdapter.getMinTime().getMillis();
|
||||||
Interval actualDataInterval = Intervals.utc(segmentMinTime, segmentMaxTime + 1);
|
Interval actualDataInterval = Intervals.utc(segmentMinTime, segmentMaxTime + 1);
|
||||||
|
|
|
@ -83,7 +83,7 @@ public class SegmentLocalCacheLoaderTest
|
||||||
Assert.assertEquals(interval, segment.getDataInterval());
|
Assert.assertEquals(interval, segment.getDataInterval());
|
||||||
Assert.assertNotNull(segment.asStorageAdapter());
|
Assert.assertNotNull(segment.asStorageAdapter());
|
||||||
|
|
||||||
Assert.assertTrue(segment.asQueryableIndex().isFromTombstone());
|
Assert.assertTrue(segment.asStorageAdapter().isFromTombstone());
|
||||||
|
|
||||||
Assert.assertEquals(interval, segment.asQueryableIndex().getDataInterval());
|
Assert.assertEquals(interval, segment.asQueryableIndex().getDataInterval());
|
||||||
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getMetadata());
|
Assert.assertThrows(UnsupportedOperationException.class, () -> segment.asQueryableIndex().getMetadata());
|
||||||
|
|
Loading…
Reference in New Issue