Skip to content

Commit

Permalink
Remove null and empty fields from native queries (#12634)
Browse files Browse the repository at this point in the history
* Remove null and empty fields from native queries

* Test fixes

* Attempted IT fix.

* Revisions from review comments

* Build fixes resulting from changes suggested by reviews

* IT fix for changed segment size
  • Loading branch information
paul-rogers authored Jun 16, 2022
1 parent f050069 commit 893759d
Show file tree
Hide file tree
Showing 39 changed files with 209 additions and 164 deletions.
9 changes: 9 additions & 0 deletions codestyle/spotbugs-exclude.xml
Original file line number Diff line number Diff line change
Expand Up @@ -46,7 +46,10 @@
<Or>
<Class name="org.apache.druid.query.scan.ScanQuery$ScanRowsLimitJsonIncludeFilter"/>
<Class name="org.apache.druid.query.scan.ScanQuery$ScanTimeOrderJsonIncludeFilter"/>
<Class name="org.apache.druid.query.scan.ScanQuery$BatchSizeJsonIncludeFilter"/>
<Class name="org.apache.druid.query.scan.ScanQuery$IsLegacyJsonIncludeFilter"/>
<Class name="org.apache.druid.query.groupby.orderby.DefaultLimitSpec$LimitJsonIncludeFilter"/>
<Class name="org.apache.druid.segment.VirtualColumns$JsonIncludeFilter"/>
</Or>
</And>
</Match>
Expand All @@ -56,6 +59,12 @@
<Class name="org.apache.druid.server.AsyncQueryForwardingServlet" />
</And>
</Match>
<Match>
<And>
<Bug pattern="URF_UNREAD_PUBLIC_OR_PROTECTED_FIELD" />
<Class name="org.apache.druid.sql.calcite.planner.CapturedState" />
</And>
</Match>

<Bug pattern="AT_OPERATION_SEQUENCE_ON_CONCURRENT_ABSTRACTION"/>
<Bug pattern="BC_UNCONFIRMED_CAST"/>
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -90,7 +90,6 @@ private static Response.ResponseBuilder responseBuilder(HttpVersion version, Htt
return builder;
}


MockHttpClient httpClient;
HttpPostEmitter emitter;

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -164,7 +164,6 @@ public void testRunParallelWithDynamicPartitioningMatchCompactionState() throws
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
CompactionState expectedState = new CompactionState(
new DynamicPartitionsSpec(null, Long.MAX_VALUE),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
Expand Down Expand Up @@ -211,7 +210,6 @@ public void testRunParallelWithHashPartitioningMatchCompactionState() throws Exc
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
Assert.assertSame(HashBasedNumberedShardSpec.class, segment.getShardSpec().getClass());
CompactionState expectedState = new CompactionState(
new HashedPartitionsSpec(null, 3, null),
Expand Down Expand Up @@ -259,7 +257,6 @@ public void testRunParallelWithRangePartitioning() throws Exception
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass());
CompactionState expectedState = new CompactionState(
new SingleDimensionPartitionsSpec(7, null, "dim", false),
Expand Down Expand Up @@ -310,7 +307,6 @@ public void testRunParallelWithMultiDimensionRangePartitioning() throws Exceptio
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass());
CompactionState expectedState = new CompactionState(
new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false),
Expand Down Expand Up @@ -358,7 +354,6 @@ public void testRunParallelWithRangePartitioningWithSingleTask() throws Exceptio
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
Assert.assertSame(SingleDimensionShardSpec.class, segment.getShardSpec().getClass());
CompactionState expectedState = new CompactionState(
new SingleDimensionPartitionsSpec(7, null, "dim", false),
Expand Down Expand Up @@ -409,7 +404,6 @@ public void testRunParallelWithMultiDimensionRangePartitioningWithSingleTask() t
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
Assert.assertSame(DimensionRangeShardSpec.class, segment.getShardSpec().getClass());
CompactionState expectedState = new CompactionState(
new DimensionRangePartitionsSpec(7, null, Arrays.asList("dim1", "dim2"), false),
Expand Down Expand Up @@ -490,7 +484,6 @@ public void testRunCompactionWithFilterShouldStoreInState() throws Exception
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
CompactionState expectedState = new CompactionState(
new DynamicPartitionsSpec(null, Long.MAX_VALUE),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
Expand Down Expand Up @@ -545,7 +538,6 @@ public void testRunCompactionWithNewMetricsShouldStoreInState() throws Exception
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
CompactionState expectedState = new CompactionState(
new DynamicPartitionsSpec(null, Long.MAX_VALUE),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -202,7 +202,6 @@ public static CompactionState getDefaultCompactionState(Granularity segmentGranu
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
return new CompactionState(
new DynamicPartitionsSpec(5000000, Long.MAX_VALUE),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
Expand Down Expand Up @@ -355,7 +354,6 @@ public void testRunWithHashPartitioning() throws Exception
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
CompactionState expectedState = new CompactionState(
new HashedPartitionsSpec(null, 3, null),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
Expand Down Expand Up @@ -648,7 +646,6 @@ public void testCompactionWithFilterInTransformSpec() throws Exception
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
CompactionState expectedCompactionState = new CompactionState(
new DynamicPartitionsSpec(5000000, Long.MAX_VALUE),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
Expand Down Expand Up @@ -710,7 +707,6 @@ public void testCompactionWithNewMetricInMetricsSpec() throws Exception
expectedLongSumMetric.put("type", "longSum");
expectedLongSumMetric.put("name", "val");
expectedLongSumMetric.put("fieldName", "val");
expectedLongSumMetric.put("expression", null);
CompactionState expectedCompactionState = new CompactionState(
new DynamicPartitionsSpec(5000000, Long.MAX_VALUE),
new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("ts", "dim"))),
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -115,9 +115,6 @@ protected void finished(Description description)
}
};




@Rule
public final TestRule timeout = new DeadlockDetectingTimeout(60, TimeUnit.SECONDS);

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -117,7 +117,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
@BeforeMethod
public void setup() throws Exception
{
// Set comapction slot to 5
// Set compaction slot to 5
updateCompactionTaskSlot(0.5, 10, null);
fullDatasourceName = "wikipedia_index_test_" + UUID.randomUUID() + config.getExtraDatasourceNameSuffix();
}
Expand Down Expand Up @@ -458,8 +458,8 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception
fullDatasourceName,
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
0,
14906,
14905,
14762,
14761,
0,
2,
2,
Expand All @@ -476,7 +476,7 @@ public void testAutoCompactionDutySubmitAndVerifyCompaction() throws Exception
fullDatasourceName,
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
0,
23372,
23156,
0,
0,
3,
Expand Down Expand Up @@ -592,16 +592,16 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception
getAndAssertCompactionStatus(
fullDatasourceName,
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
14906,
14905,
14762,
14761,
0,
2,
2,
0,
1,
1,
0);
Assert.assertEquals(compactionResource.getCompactionProgress(fullDatasourceName).get("remainingSegmentSize"), "14906");
Assert.assertEquals(compactionResource.getCompactionProgress(fullDatasourceName).get("remainingSegmentSize"), "14762");
// Run compaction again to compact the remaining day
// Remaining day compacted (1 new segment). Now both days compacted (2 total)
forceTriggerAutoCompaction(2);
Expand All @@ -612,7 +612,7 @@ public void testAutoCompactionDutyCanUpdateTaskSlots() throws Exception
fullDatasourceName,
AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING,
0,
23372,
23156,
0,
0,
3,
Expand Down Expand Up @@ -645,7 +645,7 @@ public void testAutoCompactionDutyWithSegmentGranularityAndWithDropExistingTrue(
// Four data segments (two months) and 10 tombstones for remaining months
// 3d compaction: SEMESTER: 5 rows @ 2013-08-31 (two segments), 5 rows @ 2013-09-01 (two segments),
// 2 compactions were generated for year 2013; one for each semester to be compacted of the whole year.
//
//
loadData(INDEX_TASK);

try (final Closeable ignored = unloader(fullDatasourceName)) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

package org.apache.druid.query;

import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonInclude.Include;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
Expand Down Expand Up @@ -101,6 +103,7 @@ public DataSource getDataSource()

@JsonProperty
@Override
@JsonInclude(Include.NON_DEFAULT)
public boolean isDescending()
{
return descending;
Expand Down Expand Up @@ -165,6 +168,7 @@ public DateTimeZone getTimezone()

@Override
@JsonProperty
@JsonInclude(Include.NON_DEFAULT)
public Map<String, Object> getContext()
{
return context.getMergedParams();
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,8 @@

import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonInclude.Include;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
Expand All @@ -33,6 +35,7 @@
import org.apache.druid.segment.join.JoinType;

import javax.annotation.Nullable;

import java.util.HashSet;
import java.util.List;
import java.util.Objects;
Expand Down Expand Up @@ -176,6 +179,7 @@ public JoinType getJoinType()

@JsonProperty
@Nullable
@JsonInclude(Include.NON_NULL)
public DimFilter getLeftFilter()
{
return leftFilter;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -34,7 +34,7 @@
public class QueryDataSource implements DataSource
{
@JsonProperty
private final Query query;
private final Query<?> query;

@JsonCreator
public QueryDataSource(@JsonProperty("query") Query query)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -21,6 +21,7 @@

import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
Expand Down Expand Up @@ -204,13 +205,15 @@ public String getName()

@JsonProperty
@Nullable
@JsonInclude(JsonInclude.Include.NON_EMPTY)
public Set<String> getFields()
{
return fields;
}

@JsonProperty
@Nullable
@JsonInclude(JsonInclude.Include.NON_NULL)
public String getAccumulatorIdentifier()
{
return accumulatorId;
Expand Down Expand Up @@ -260,13 +263,15 @@ public String getCombineExpressionString()

@JsonProperty("compare")
@Nullable
@JsonInclude(JsonInclude.Include.NON_NULL)
public String getCompareExpressionString()
{
return compareExpressionString;
}

@JsonProperty("finalize")
@Nullable
@JsonInclude(JsonInclude.Include.NON_NULL)
public String getFinalizeExpressionString()
{
return finalizeExpressionString;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,6 +19,8 @@

package org.apache.druid.query.aggregation;

import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonInclude.Include;
import org.apache.druid.guice.annotations.ExtensionPoint;
import org.apache.druid.java.util.common.Cacheable;
import org.apache.druid.segment.ColumnInspector;
Expand All @@ -45,6 +47,7 @@ public interface PostAggregator extends Cacheable
Object compute(Map<String, Object> combinedAggregators);

@Nullable
@JsonInclude(Include.NON_NULL)
String getName();

/**
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.apache.druid.query.aggregation;


import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
Expand Down Expand Up @@ -234,13 +235,15 @@ public String getName()

@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_NULL)
public String getFieldName()
{
return fieldName;
}

@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_NULL)
public String getExpression()
{
return expression;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -20,6 +20,7 @@
package org.apache.druid.query.aggregation;


import com.fasterxml.jackson.annotation.JsonInclude;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Supplier;
Expand Down Expand Up @@ -212,13 +213,15 @@ public String getName()

@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_NULL)
public String getFieldName()
{
return fieldName;
}

@Nullable
@JsonProperty
@JsonInclude(JsonInclude.Include.NON_NULL)
public String getExpression()
{
return expression;
Expand Down
Loading

0 comments on commit 893759d

Please sign in to comment.