From de40d81b29cb25e092c84dc673401ad9432851d1 Mon Sep 17 00:00:00 2001 From: Gian Merlino Date: Tue, 6 Aug 2024 06:40:21 -0700 Subject: [PATCH 1/6] SQL: Add ProjectableFilterableTable to SegmentsTable. (#16841) * SQL: Add ProjectableFilterableTable to SegmentsTable. This allows us to skip serialization of expensive fields such as shard_spec, dimensions, metrics, and last_compaction_state, if those fields are not actually being queried. * Restructure logic to avoid unnecessary toString() as well. --- .../sql/calcite/planner/QueryHandler.java | 4 +- .../sql/calcite/schema/SystemSchema.java | 191 ++++++++++++------ .../sql/calcite/schema/SystemSchemaTest.java | 77 ++++++- 3 files changed, 201 insertions(+), 71 deletions(-) diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java index b543be135cc..82dd6afe8c9 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/planner/QueryHandler.java @@ -47,6 +47,7 @@ import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; import org.apache.calcite.rex.RexBuilder; import org.apache.calcite.rex.RexNode; +import org.apache.calcite.schema.ProjectableFilterableTable; import org.apache.calcite.schema.ScannableTable; import org.apache.calcite.sql.SqlExplain; import org.apache.calcite.sql.SqlNode; @@ -285,7 +286,8 @@ public abstract class QueryHandler extends SqlStatementHandler.BaseStatementHand { if (node instanceof TableScan) { RelOptTable table = node.getTable(); - if (table.unwrap(ScannableTable.class) != null && table.unwrap(DruidTable.class) == null) { + if ((table.unwrap(ScannableTable.class) != null || table.unwrap(ProjectableFilterableTable.class) != null) + && table.unwrap(DruidTable.class) == null) { found.add(table); return; } diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java index 1706620f79c..695b34b2772 100644 --- a/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java +++ b/sql/src/main/java/org/apache/druid/sql/calcite/schema/SystemSchema.java @@ -32,6 +32,8 @@ import com.google.common.collect.Maps; import com.google.common.collect.Sets; import com.google.common.util.concurrent.Futures; import com.google.inject.Inject; +import it.unimi.dsi.fastutil.ints.IntOpenHashSet; +import it.unimi.dsi.fastutil.ints.IntSet; import org.apache.calcite.DataContext; import org.apache.calcite.linq4j.DefaultEnumerable; import org.apache.calcite.linq4j.Enumerable; @@ -39,6 +41,8 @@ import org.apache.calcite.linq4j.Enumerator; import org.apache.calcite.linq4j.Linq4j; import org.apache.calcite.rel.type.RelDataType; import org.apache.calcite.rel.type.RelDataTypeFactory; +import org.apache.calcite.rex.RexNode; +import org.apache.calcite.schema.ProjectableFilterableTable; import org.apache.calcite.schema.ScannableTable; import org.apache.calcite.schema.Table; import org.apache.calcite.schema.impl.AbstractSchema; @@ -68,6 +72,7 @@ import org.apache.druid.java.util.http.client.response.InputStreamFullResponseHo import org.apache.druid.rpc.indexing.OverlordClient; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.column.ValueType; import org.apache.druid.segment.metadata.AvailableSegmentMetadata; import org.apache.druid.server.DruidNode; import org.apache.druid.server.security.Access; @@ -99,6 +104,7 @@ import java.util.Map.Entry; import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; +import java.util.stream.IntStream; public class SystemSchema extends AbstractSchema { @@ -157,6 +163,23 @@ public class SystemSchema extends AbstractSchema .add("replication_factor", ColumnType.LONG) .build(); + /** + * List of [0..n) where n is the size of {@link #SEGMENTS_SIGNATURE}. + */ + private static final int[] SEGMENTS_PROJECT_ALL = IntStream.range(0, SEGMENTS_SIGNATURE.size()).toArray(); + + /** + * Fields in {@link #SEGMENTS_SIGNATURE} that are serialized with {@link ObjectMapper#writeValueAsString(Object)}. + */ + private static final IntSet SEGMENTS_JSON_FIELDS = new IntOpenHashSet( + new int[]{ + SEGMENTS_SIGNATURE.indexOf("shard_spec"), + SEGMENTS_SIGNATURE.indexOf("dimensions"), + SEGMENTS_SIGNATURE.indexOf("metrics"), + SEGMENTS_SIGNATURE.indexOf("last_compaction_state") + } + ); + static final RowSignature SERVERS_SIGNATURE = RowSignature .builder() .add("server", ColumnType.STRING) @@ -241,7 +264,7 @@ public class SystemSchema extends AbstractSchema /** * This table contains row per segment from metadata store as well as served segments. */ - static class SegmentsTable extends AbstractTable implements ScannableTable + static class SegmentsTable extends AbstractTable implements ProjectableFilterableTable { private final DruidSchema druidSchema; private final ObjectMapper jsonMapper; @@ -274,7 +297,11 @@ public class SystemSchema extends AbstractSchema } @Override - public Enumerable scan(DataContext root) + public Enumerable scan( + final DataContext root, + final List filters, + @Nullable final int[] projects + ) { // get available segments from druidSchema final Map availableSegmentMetadata = @@ -327,35 +354,30 @@ public class SystemSchema extends AbstractSchema // is_active is true for published segments that are not overshadowed or else they should be realtime boolean isActive = isPublished ? !val.isOvershadowed() : val.isRealtime(); - try { - return new Object[]{ - segment.getId(), - segment.getDataSource(), - segment.getInterval().getStart().toString(), - segment.getInterval().getEnd().toString(), - segment.getSize(), - segment.getVersion(), - (long) segment.getShardSpec().getPartitionNum(), - numReplicas, - numRows, - isActive ? IS_ACTIVE_TRUE : IS_ACTIVE_FALSE, - isPublished ? IS_PUBLISHED_TRUE : IS_PUBLISHED_FALSE, - isAvailable, - isRealtime, - val.isOvershadowed() ? IS_OVERSHADOWED_TRUE : IS_OVERSHADOWED_FALSE, - segment.getShardSpec() == null ? null : jsonMapper.writeValueAsString(segment.getShardSpec()), - segment.getDimensions() == null ? null : jsonMapper.writeValueAsString(segment.getDimensions()), - segment.getMetrics() == null ? null : jsonMapper.writeValueAsString(segment.getMetrics()), - segment.getLastCompactionState() == null ? null : jsonMapper.writeValueAsString(segment.getLastCompactionState()), - // If the segment is unpublished, we won't have this information yet. - // If the value is null, the load rules might have not evaluated yet, and we don't know the replication factor. - // This should be automatically updated in the next Coordinator poll. - val.getReplicationFactor() == null ? REPLICATION_FACTOR_UNKNOWN : (long) val.getReplicationFactor() - }; - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } + return new Object[]{ + segment.getId(), + segment.getDataSource(), + segment.getInterval().getStart(), + segment.getInterval().getEnd(), + segment.getSize(), + segment.getVersion(), + (long) segment.getShardSpec().getPartitionNum(), + numReplicas, + numRows, + isActive ? IS_ACTIVE_TRUE : IS_ACTIVE_FALSE, + isPublished ? IS_PUBLISHED_TRUE : IS_PUBLISHED_FALSE, + isAvailable, + isRealtime, + val.isOvershadowed() ? IS_OVERSHADOWED_TRUE : IS_OVERSHADOWED_FALSE, + segment.getShardSpec(), + segment.getDimensions(), + segment.getMetrics(), + segment.getLastCompactionState(), + // If the segment is unpublished, we won't have this information yet. + // If the value is null, the load rules might have not evaluated yet, and we don't know the replication factor. + // This should be automatically updated in the next Coordinator poll. + val.getReplicationFactor() == null ? REPLICATION_FACTOR_UNKNOWN : (long) val.getReplicationFactor() + }; }); // If druid.centralizedDatasourceSchema.enabled is set on the Coordinator, all the segments in this loop @@ -369,45 +391,43 @@ public class SystemSchema extends AbstractSchema if (segmentsAlreadySeen.contains(val.getKey())) { return null; } + final DataSegment segment = val.getValue().getSegment(); final PartialSegmentData partialSegmentData = partialSegmentDataMap.get(val.getKey()); final long numReplicas = partialSegmentData == null ? 0L : partialSegmentData.getNumReplicas(); - try { - return new Object[]{ - val.getKey(), - val.getKey().getDataSource(), - val.getKey().getInterval().getStart().toString(), - val.getKey().getInterval().getEnd().toString(), - val.getValue().getSegment().getSize(), - val.getKey().getVersion(), - (long) val.getValue().getSegment().getShardSpec().getPartitionNum(), - numReplicas, - val.getValue().getNumRows(), - // is_active is true for unpublished segments iff they are realtime - val.getValue().isRealtime() /* is_active */, - // is_published is false for unpublished segments - IS_PUBLISHED_FALSE, - // is_available is assumed to be always true for segments announced by historicals or realtime tasks - IS_AVAILABLE_TRUE, - val.getValue().isRealtime(), - IS_OVERSHADOWED_FALSE, - // there is an assumption here that unpublished segments are never overshadowed - val.getValue().getSegment().getShardSpec() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getShardSpec()), - val.getValue().getSegment().getDimensions() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getDimensions()), - val.getValue().getSegment().getMetrics() == null ? null : jsonMapper.writeValueAsString(val.getValue().getSegment().getMetrics()), - null, // unpublished segments from realtime tasks will not be compacted yet - REPLICATION_FACTOR_UNKNOWN // If the segment is unpublished, we won't have this information yet. - }; - } - catch (JsonProcessingException e) { - throw new RuntimeException(e); - } + return new Object[]{ + val.getKey(), + val.getKey().getDataSource(), + val.getKey().getInterval().getStart(), + val.getKey().getInterval().getEnd(), + segment.getSize(), + val.getKey().getVersion(), + (long) segment.getShardSpec().getPartitionNum(), + numReplicas, + val.getValue().getNumRows(), + // is_active is true for unpublished segments iff they are realtime + val.getValue().isRealtime() /* is_active */, + // is_published is false for unpublished segments + IS_PUBLISHED_FALSE, + // is_available is assumed to be always true for segments announced by historicals or realtime tasks + IS_AVAILABLE_TRUE, + val.getValue().isRealtime(), + IS_OVERSHADOWED_FALSE, + // there is an assumption here that unpublished segments are never overshadowed + segment.getShardSpec(), + segment.getDimensions(), + segment.getMetrics(), + null, // unpublished segments from realtime tasks will not be compacted yet + REPLICATION_FACTOR_UNKNOWN // If the segment is unpublished, we won't have this information yet. + }; }); final Iterable allSegments = Iterables.unmodifiableIterable( Iterables.concat(publishedSegments, availableSegments) ); - return Linq4j.asEnumerable(allSegments).where(Objects::nonNull); + return Linq4j.asEnumerable(allSegments) + .where(Objects::nonNull) + .select(row -> projectSegmentsRow(row, projects, jsonMapper)); } private Iterator getAuthorizedPublishedSegments( @@ -638,7 +658,10 @@ public class SystemSchema extends AbstractSchema /** * Returns a row for all node types which don't serve data. The returned row contains only static information. */ - private static Object[] buildRowForNonDataServerWithLeadership(DiscoveryDruidNode discoveryDruidNode, boolean isLeader) + private static Object[] buildRowForNonDataServerWithLeadership( + DiscoveryDruidNode discoveryDruidNode, + boolean isLeader + ) { final DruidNode node = discoveryDruidNode.getDruidNode(); return new Object[]{ @@ -775,7 +798,7 @@ public class SystemSchema extends AbstractSchema for (DataSegment segment : authorizedServerSegments) { Object[] row = new Object[serverSegmentsTableSize]; row[0] = druidServer.getHost(); - row[1] = segment.getId(); + row[1] = segment.getId().toString(); rows.add(row); } } @@ -1138,4 +1161,44 @@ public class SystemSchema extends AbstractSchema throw new ForbiddenException("Insufficient permission to view servers: " + stateAccess.toMessage()); } } + + /** + * Project a row using "projects" from {@link SegmentsTable#scan(DataContext, List, int[])}. + * + * Also, fix up types so {@link ColumnType#STRING} are transformed to Strings if they aren't yet. This defers + * computation of {@link ObjectMapper#writeValueAsString(Object)} or {@link Object#toString()} until we know we + * actually need it. + */ + private static Object[] projectSegmentsRow( + final Object[] row, + @Nullable final int[] projects, + final ObjectMapper jsonMapper + ) + { + final int[] nonNullProjects = projects == null ? SEGMENTS_PROJECT_ALL : projects; + final Object[] projectedRow = new Object[nonNullProjects.length]; + + for (int i = 0; i < nonNullProjects.length; i++) { + final Object o = row[nonNullProjects[i]]; + + if (SEGMENTS_SIGNATURE.getColumnType(nonNullProjects[i]).get().is(ValueType.STRING) + && o != null + && !(o instanceof String)) { + // Delay calling toString() or ObjectMapper#writeValueAsString() until we know we actually need this field. + if (SEGMENTS_JSON_FIELDS.contains(nonNullProjects[i])) { + try { + projectedRow[i] = jsonMapper.writeValueAsString(o); + } + catch (JsonProcessingException e) { + throw new RuntimeException(e); + } + } else { + projectedRow[i] = o.toString(); + } + } else { + projectedRow[i] = o; + } + } + return projectedRow; + } } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java index 992d91dabe8..de5ffbb5f19 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/schema/SystemSchemaTest.java @@ -19,6 +19,7 @@ package org.apache.druid.sql.calcite.schema; +import com.fasterxml.jackson.core.JsonProcessingException; import com.fasterxml.jackson.core.type.TypeReference; import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.collect.ImmutableList; @@ -579,7 +580,7 @@ public class SystemSchemaTest extends CalciteTestBase EasyMock.replay(client, request, responseHolder, responseHandler, metadataView); DataContext dataContext = createDataContext(Users.SUPER); - final List rows = segmentsTable.scan(dataContext).toList(); + final List rows = segmentsTable.scan(dataContext, Collections.emptyList(), null).toList(); rows.sort((Object[] row1, Object[] row2) -> ((Comparable) row1[0]).compareTo(row2[0])); // total segments = 8 @@ -717,6 +718,74 @@ public class SystemSchemaTest extends CalciteTestBase verifyTypes(rows, SystemSchema.SEGMENTS_SIGNATURE); } + @Test + public void testSegmentsTableWithProjection() throws JsonProcessingException + { + final SegmentsTable segmentsTable = new SegmentsTable(druidSchema, metadataView, new ObjectMapper(), authMapper); + final Set publishedSegments = new HashSet<>(Arrays.asList( + new SegmentStatusInCluster(publishedCompactedSegment1, true, 2, null, false), + new SegmentStatusInCluster(publishedCompactedSegment2, false, 0, null, false), + new SegmentStatusInCluster(publishedUncompactedSegment3, false, 2, null, false), + new SegmentStatusInCluster(segment1, true, 2, null, false), + new SegmentStatusInCluster(segment2, false, 0, null, false) + )); + + EasyMock.expect(metadataView.getSegments()).andReturn(publishedSegments.iterator()).once(); + + EasyMock.replay(client, request, responseHolder, responseHandler, metadataView); + DataContext dataContext = createDataContext(Users.SUPER); + final List rows = segmentsTable.scan( + dataContext, + Collections.emptyList(), + new int[]{ + SystemSchema.SEGMENTS_SIGNATURE.indexOf("last_compaction_state"), + SystemSchema.SEGMENTS_SIGNATURE.indexOf("segment_id") + } + ).toList(); + rows.sort((Object[] row1, Object[] row2) -> ((Comparable) row1[1]).compareTo(row2[1])); + + // total segments = 8 + // segments test1, test2 are published and available + // segment test3 is served by historical but unpublished or unused + // segments test4, test5 are not published but available (realtime segments) + // segment test2 is both published and served by a realtime server. + + Assert.assertEquals(8, rows.size()); + + Assert.assertNull(null, rows.get(0)[0]); + Assert.assertEquals("test1_2010-01-01T00:00:00.000Z_2011-01-01T00:00:00.000Z_version1", rows.get(0)[1]); + + Assert.assertNull(null, rows.get(1)[0]); + Assert.assertEquals("test2_2011-01-01T00:00:00.000Z_2012-01-01T00:00:00.000Z_version2", rows.get(1)[1]); + + Assert.assertNull(null, rows.get(2)[0]); + Assert.assertEquals("test3_2012-01-01T00:00:00.000Z_2013-01-01T00:00:00.000Z_version3_2", rows.get(2)[1]); + + Assert.assertNull(null, rows.get(3)[0]); + Assert.assertEquals("test4_2014-01-01T00:00:00.000Z_2015-01-01T00:00:00.000Z_version4", rows.get(3)[1]); + + Assert.assertNull(null, rows.get(4)[0]); + Assert.assertEquals("test5_2015-01-01T00:00:00.000Z_2016-01-01T00:00:00.000Z_version5", rows.get(4)[1]); + + Assert.assertEquals(mapper.writeValueAsString(expectedCompactionState), rows.get(5)[0]); + Assert.assertEquals("wikipedia1_2007-01-01T00:00:00.000Z_2008-01-01T00:00:00.000Z_version1", rows.get(5)[1]); + + Assert.assertEquals(mapper.writeValueAsString(expectedCompactionState), rows.get(6)[0]); + Assert.assertEquals("wikipedia2_2008-01-01T00:00:00.000Z_2009-01-01T00:00:00.000Z_version2", rows.get(6)[1]); + + Assert.assertNull(null, rows.get(7)[0]); + Assert.assertEquals("wikipedia3_2009-01-01T00:00:00.000Z_2010-01-01T00:00:00.000Z_version3", rows.get(7)[1]); + + // Verify value types. + verifyTypes( + rows, + RowSignature.builder() + .add("last_compaction_state", ColumnType.STRING) + .add("segment_id", ColumnType.STRING) + .build() + ); + } + private void verifyRow( Object[] row, String segmentId, @@ -1519,11 +1588,7 @@ public class SystemSchemaTest extends CalciteTestBase expectedClass = Double.class; break; case STRING: - if (signature.getColumnName(i).equals("segment_id")) { - expectedClass = SegmentId.class; - } else { - expectedClass = String.class; - } + expectedClass = String.class; break; default: throw new IAE("Don't know what class to expect for valueType[%s]", columnType); From aa49be61ea1cad99e3ad4fb85b0b0c5dfec2d033 Mon Sep 17 00:00:00 2001 From: Kashif Faraz Date: Tue, 6 Aug 2024 06:59:13 -0700 Subject: [PATCH 2/6] Do not create ZK paths if not needed (#16816) Background: ZK-based segment loading has been completely disabled in #15705 . ZK `servedSegmentsPath` has been deprecated since Druid 0.7.1, #1182 . This legacy path has been replaced by the `liveSegmentsPath` and is not used in the code anymore. Changes: - Never create ZK loadQueuePath as it is never used. - Never create ZK servedSegmentsPath as it is never used. - Do not create ZK liveSegmentsPath if announcement on ZK is disabled - Fix up tests --- docs/configuration/index.md | 1 - docs/design/zookeeper.md | 6 +- .../druid/curator/ZkEnablementConfig.java | 4 +- .../apache/druid/guice/ServerViewModule.java | 4 - .../server/coordination/ZkCoordinator.java | 134 +++-------------- .../server/initialization/ZkPathsConfig.java | 26 +--- .../initialization/ZkPathsConfigTest.java | 2 - .../coordination/ZkCoordinatorTest.java | 141 +++++------------- .../server/coordinator/simulate/README.md | 1 - 9 files changed, 67 insertions(+), 252 deletions(-) diff --git a/docs/configuration/index.md b/docs/configuration/index.md index 3b3c2711d3b..6c09e07634f 100644 --- a/docs/configuration/index.md +++ b/docs/configuration/index.md @@ -160,7 +160,6 @@ Druid interacts with ZooKeeper through a set of standard path configurations. We |`druid.zk.paths.announcementsPath`|Druid service announcement path.|`${druid.zk.paths.base}/announcements`| |`druid.zk.paths.liveSegmentsPath`|Current path for where Druid services announce their segments.|`${druid.zk.paths.base}/segments`| |`druid.zk.paths.coordinatorPath`|Used by the Coordinator for leader election.|`${druid.zk.paths.base}/coordinator`| -|`druid.zk.paths.servedSegmentsPath`|Deprecated. Legacy path for where Druid services announce their segments.|`${druid.zk.paths.base}/servedSegments`| The indexing service also uses its own set of paths. These configs can be included in the common configuration. diff --git a/docs/design/zookeeper.md b/docs/design/zookeeper.md index 50241bd3d9d..eb8bea57741 100644 --- a/docs/design/zookeeper.md +++ b/docs/design/zookeeper.md @@ -53,7 +53,7 @@ ${druid.zk.paths.coordinatorPath}/_COORDINATOR ## Segment "publishing" protocol from Historical and Realtime -The `announcementsPath` and `servedSegmentsPath` are used for this. +The `announcementsPath` and `liveSegmentsPath` are used for this. All [Historical](../design/historical.md) processes publish themselves on the `announcementsPath`, specifically, they will create an ephemeral znode at @@ -64,13 +64,13 @@ ${druid.zk.paths.announcementsPath}/${druid.host} Which signifies that they exist. They will also subsequently create a permanent znode at ``` -${druid.zk.paths.servedSegmentsPath}/${druid.host} +${druid.zk.paths.liveSegmentsPath}/${druid.host} ``` And as they load up segments, they will attach ephemeral znodes that look like ``` -${druid.zk.paths.servedSegmentsPath}/${druid.host}/_segment_identifier_ +${druid.zk.paths.liveSegmentsPath}/${druid.host}/_segment_identifier_ ``` Processes like the [Coordinator](../design/coordinator.md) and [Broker](../design/broker.md) can then watch these paths to see which processes are currently serving which segments. diff --git a/server/src/main/java/org/apache/druid/curator/ZkEnablementConfig.java b/server/src/main/java/org/apache/druid/curator/ZkEnablementConfig.java index 0e4eb31785b..e1f9c2b4271 100644 --- a/server/src/main/java/org/apache/druid/curator/ZkEnablementConfig.java +++ b/server/src/main/java/org/apache/druid/curator/ZkEnablementConfig.java @@ -37,7 +37,7 @@ public class ZkEnablementConfig @JsonCreator public ZkEnablementConfig(@JsonProperty("enabled") Boolean enabled) { - this.enabled = enabled == null ? true : enabled.booleanValue(); + this.enabled = enabled == null || enabled; } public boolean isEnabled() @@ -48,6 +48,6 @@ public class ZkEnablementConfig public static boolean isEnabled(Properties properties) { String value = properties.getProperty(PROP_KEY_ENABLED); - return value == null ? true : Boolean.parseBoolean(value); + return value == null || Boolean.parseBoolean(value); } } diff --git a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java index 5b36f4a1d77..ee5ac7d939b 100644 --- a/server/src/main/java/org/apache/druid/guice/ServerViewModule.java +++ b/server/src/main/java/org/apache/druid/guice/ServerViewModule.java @@ -41,10 +41,6 @@ public class ServerViewModule implements Module public static final String SERVERVIEW_TYPE_HTTP = "http"; public static final String SERVERVIEW_TYPE_BATCH = "batch"; - // this value should be consistent with the default implementation used in - // {@code ServerInventoryViewProvider} & {@code FilteredServerInventoryViewProvider} - public static final String DEFAULT_SERVERVIEW_TYPE = "http"; - @Override public void configure(Binder binder) { diff --git a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java index 49356d05975..72f6affe82f 100644 --- a/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java +++ b/server/src/main/java/org/apache/druid/server/coordination/ZkCoordinator.java @@ -19,29 +19,25 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.databind.ObjectMapper; import com.google.common.base.Throwables; import com.google.inject.Inject; import org.apache.curator.framework.CuratorFramework; -import org.apache.curator.framework.recipes.cache.ChildData; -import org.apache.curator.framework.recipes.cache.PathChildrenCache; import org.apache.curator.utils.ZKPaths; -import org.apache.druid.java.util.common.concurrent.Execs; import org.apache.druid.java.util.common.lifecycle.LifecycleStart; import org.apache.druid.java.util.common.lifecycle.LifecycleStop; import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.loading.SegmentLoaderConfig; +import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import org.apache.druid.server.initialization.ZkPathsConfig; -import javax.annotation.Nullable; import java.io.IOException; -import java.util.concurrent.ExecutorService; /** - * We are gradually migrating to {@link org.apache.druid.server.http.SegmentListerResource} for driving segment - * loads/drops on data server processes. + * Creates paths for announcing served segments on Zookeeper. * - * However, this class is still the default mechanism as of this writing (2020-12-03). + * @deprecated as Druid has already migrated to HTTP-based segment loading and + * will soon migrate to HTTP-based inventory view using {@code SegmentListerResource}. + * + * @see org.apache.druid.server.http.SegmentListerResource */ @Deprecated public class ZkCoordinator @@ -50,36 +46,25 @@ public class ZkCoordinator private final Object lock = new Object(); - private final DataSegmentChangeHandler dataSegmentChangeHandler; - private final ObjectMapper jsonMapper; private final ZkPathsConfig zkPaths; private final DruidServerMetadata me; private final CuratorFramework curator; + private final BatchDataSegmentAnnouncerConfig announcerConfig; - @Nullable - private volatile PathChildrenCache loadQueueCache; private volatile boolean started = false; - private final ExecutorService segmentLoadUnloadService; @Inject public ZkCoordinator( - SegmentLoadDropHandler loadDropHandler, - ObjectMapper jsonMapper, ZkPathsConfig zkPaths, DruidServerMetadata me, CuratorFramework curator, - SegmentLoaderConfig config + BatchDataSegmentAnnouncerConfig announcerConfig ) { - this.dataSegmentChangeHandler = loadDropHandler; - this.jsonMapper = jsonMapper; this.zkPaths = zkPaths; this.me = me; this.curator = curator; - this.segmentLoadUnloadService = Execs.multiThreaded( - config.getNumLoadingThreads(), - "ZKCoordinator--%d" - ); + this.announcerConfig = announcerConfig; } @LifecycleStart @@ -92,40 +77,17 @@ public class ZkCoordinator log.info("Starting zkCoordinator for server[%s]", me.getName()); - final String loadQueueLocation = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName()); - final String servedSegmentsLocation = ZKPaths.makePath(zkPaths.getServedSegmentsPath(), me.getName()); - final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName()); + if (announcerConfig.isSkipSegmentAnnouncementOnZk()) { + log.info("Skipping zkPath creation as segment announcement on ZK is disabled."); + started = true; + return; + } - loadQueueCache = new PathChildrenCache( - curator, - loadQueueLocation, - true, - true, - Execs.singleThreaded("ZkCoordinator") - ); + final String liveSegmentsLocation = ZKPaths.makePath(zkPaths.getLiveSegmentsPath(), me.getName()); + log.info("Creating zkPath[%s] for announcing live segments.", liveSegmentsLocation); try { - curator.newNamespaceAwareEnsurePath(loadQueueLocation).ensure(curator.getZookeeperClient()); - curator.newNamespaceAwareEnsurePath(servedSegmentsLocation).ensure(curator.getZookeeperClient()); curator.newNamespaceAwareEnsurePath(liveSegmentsLocation).ensure(curator.getZookeeperClient()); - - loadQueueCache.getListenable().addListener( - (client, event) -> { - final ChildData child = event.getData(); - switch (event.getType()) { - case CHILD_ADDED: - childAdded(child); - break; - case CHILD_REMOVED: - log.info("zNode[%s] was removed", event.getData().getPath()); - break; - default: - log.info("Ignoring event[%s]", event); - } - } - - ); - loadQueueCache.start(); } catch (Exception e) { Throwables.propagateIfPossible(e, IOException.class); @@ -136,54 +98,6 @@ public class ZkCoordinator } } - private void childAdded(ChildData child) - { - segmentLoadUnloadService.submit(() -> { - final String path = child.getPath(); - DataSegmentChangeRequest request = new SegmentChangeRequestNoop(); - try { - final DataSegmentChangeRequest finalRequest = jsonMapper.readValue( - child.getData(), - DataSegmentChangeRequest.class - ); - - finalRequest.go( - dataSegmentChangeHandler, - () -> { - try { - curator.delete().guaranteed().forPath(path); - log.info("Completed request [%s]", finalRequest.asString()); - } - catch (Exception e) { - try { - curator.delete().guaranteed().forPath(path); - } - catch (Exception e1) { - log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); - } - log.error(e, "Exception while removing zNode[%s]", path); - throw new RuntimeException(e); - } - } - ); - } - catch (Exception e) { - // Something went wrong in either deserializing the request using jsonMapper or when invoking it - try { - curator.delete().guaranteed().forPath(path); - } - catch (Exception e1) { - log.error(e1, "Failed to delete zNode[%s], but ignoring exception.", path); - } - - log.makeAlert(e, "Segment load/unload: uncaught exception.") - .addData("node", path) - .addData("nodeProperties", request) - .emit(); - } - }); - } - @LifecycleStop public void stop() { @@ -193,21 +107,7 @@ public class ZkCoordinator return; } - try { - loadQueueCache.close(); - } - catch (Exception e) { - throw new RuntimeException(e); - } - finally { - loadQueueCache = null; - started = false; - } + started = false; } } - - public boolean isStarted() - { - return started; - } } diff --git a/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java b/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java index 24eef17ec42..a47764c6502 100644 --- a/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java +++ b/server/src/main/java/org/apache/druid/server/initialization/ZkPathsConfig.java @@ -31,15 +31,10 @@ public class ZkPathsConfig @JsonProperty private String announcementsPath; @JsonProperty - @Deprecated - private String servedSegmentsPath; - @JsonProperty private String liveSegmentsPath; @JsonProperty private String coordinatorPath; @JsonProperty - private String loadQueuePath; - @JsonProperty private String connectorPath; public String getBase() @@ -57,12 +52,12 @@ public class ZkPathsConfig return (null == announcementsPath) ? defaultPath("announcements") : announcementsPath; } + /** + * Path to announce served segments on. + * + * @deprecated Use HTTP-based segment discovery instead. + */ @Deprecated - public String getServedSegmentsPath() - { - return (null == servedSegmentsPath) ? defaultPath("servedSegments") : servedSegmentsPath; - } - public String getLiveSegmentsPath() { return (null == liveSegmentsPath) ? defaultPath("segments") : liveSegmentsPath; @@ -78,11 +73,6 @@ public class ZkPathsConfig return defaultPath("overlord"); } - public String getLoadQueuePath() - { - return (null == loadQueuePath) ? defaultPath("loadQueue") : loadQueuePath; - } - public String getConnectorPath() { return (null == connectorPath) ? defaultPath("connector") : connectorPath; @@ -116,9 +106,7 @@ public class ZkPathsConfig this.getConnectorPath().equals(otherConfig.getConnectorPath()) && this.getLiveSegmentsPath().equals(otherConfig.getLiveSegmentsPath()) && this.getCoordinatorPath().equals(otherConfig.getCoordinatorPath()) && - this.getLoadQueuePath().equals(otherConfig.getLoadQueuePath()) && - this.getPropertiesPath().equals(otherConfig.getPropertiesPath()) && - this.getServedSegmentsPath().equals(otherConfig.getServedSegmentsPath())) { + this.getPropertiesPath().equals(otherConfig.getPropertiesPath())) { return true; } return false; @@ -130,10 +118,8 @@ public class ZkPathsConfig int result = base != null ? base.hashCode() : 0; result = 31 * result + (propertiesPath != null ? propertiesPath.hashCode() : 0); result = 31 * result + (announcementsPath != null ? announcementsPath.hashCode() : 0); - result = 31 * result + (servedSegmentsPath != null ? servedSegmentsPath.hashCode() : 0); result = 31 * result + (liveSegmentsPath != null ? liveSegmentsPath.hashCode() : 0); result = 31 * result + (coordinatorPath != null ? coordinatorPath.hashCode() : 0); - result = 31 * result + (loadQueuePath != null ? loadQueuePath.hashCode() : 0); result = 31 * result + (connectorPath != null ? connectorPath.hashCode() : 0); return result; } diff --git a/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java b/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java index 70ec19064f2..146e5adf326 100644 --- a/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java +++ b/server/src/test/java/org/apache/druid/initialization/ZkPathsConfigTest.java @@ -56,10 +56,8 @@ public class ZkPathsConfigTest extends JsonConfigTesterBase propertyValues.put(StringUtils.format("%s.base", CONFIG_PREFIX), base); propertyValues.put(StringUtils.format("%s.propertiesPath", CONFIG_PREFIX), ZKPaths.makePath(base, "properties")); propertyValues.put(StringUtils.format("%s.announcementsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "announcements")); - propertyValues.put(StringUtils.format("%s.servedSegmentsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "servedSegments")); propertyValues.put(StringUtils.format("%s.liveSegmentsPath", CONFIG_PREFIX), ZKPaths.makePath(base, "segments")); propertyValues.put(StringUtils.format("%s.coordinatorPath", CONFIG_PREFIX), ZKPaths.makePath(base, "coordinator")); - propertyValues.put(StringUtils.format("%s.loadQueuePath", CONFIG_PREFIX), ZKPaths.makePath(base, "loadQueue")); propertyValues.put(StringUtils.format("%s.connectorPath", CONFIG_PREFIX), ZKPaths.makePath(base, "connector")); ZkPathsConfig zkPathsConfigObj = zkPathsConfig.get(); diff --git a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java index a9f7772e59d..c822cfb6d0d 100644 --- a/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java +++ b/server/src/test/java/org/apache/druid/server/coordination/ZkCoordinatorTest.java @@ -19,34 +19,16 @@ package org.apache.druid.server.coordination; -import com.fasterxml.jackson.databind.ObjectMapper; -import com.google.common.collect.ImmutableMap; +import org.apache.curator.framework.CuratorFramework; +import org.apache.curator.utils.EnsurePath; import org.apache.curator.utils.ZKPaths; -import org.apache.druid.curator.CuratorTestBase; -import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.emitter.EmittingLogger; -import org.apache.druid.segment.IndexIO; -import org.apache.druid.segment.TestHelper; -import org.apache.druid.segment.loading.SegmentLoaderConfig; -import org.apache.druid.server.SegmentManager; +import org.apache.druid.server.initialization.BatchDataSegmentAnnouncerConfig; import org.apache.druid.server.initialization.ZkPathsConfig; -import org.apache.druid.server.metrics.NoopServiceEmitter; -import org.apache.druid.timeline.DataSegment; -import org.apache.druid.timeline.partition.NoneShardSpec; -import org.apache.zookeeper.CreateMode; import org.easymock.EasyMock; -import org.junit.After; -import org.junit.Before; import org.junit.Test; -import java.util.Arrays; -import java.util.concurrent.CountDownLatch; - -/** - */ -public class ZkCoordinatorTest extends CuratorTestBase +public class ZkCoordinatorTest { - private final ObjectMapper jsonMapper = TestHelper.makeJsonMapper(); private final DruidServerMetadata me = new DruidServerMetadata( "dummyServer", "dummyHost", @@ -65,100 +47,55 @@ public class ZkCoordinatorTest extends CuratorTestBase } }; - @Before - public void setUp() throws Exception + @Test(timeout = 60_000L) + public void testSegmentPathIsCreatedIfZkAnnouncementIsEnabled() throws Exception { - setupServerAndCurator(); - curator.start(); - curator.blockUntilConnected(); - } - - @After - public void tearDown() - { - tearDownServerAndCurator(); + testSegmentPathCreated(true); } @Test(timeout = 60_000L) - public void testLoadDrop() throws Exception + public void testSegmentPathIsNotCreatedIfZkAnnouncementIsDisabled() throws Exception { - EmittingLogger.registerEmitter(new NoopServiceEmitter()); - DataSegment segment = new DataSegment( - "test", - Intervals.of("P1d/2011-04-02"), - "v0", - ImmutableMap.of("version", "v0", "interval", Intervals.of("P1d/2011-04-02"), "cacheDir", "/no"), - Arrays.asList("dim1", "dim2", "dim3"), - Arrays.asList("metric1", "metric2"), - NoneShardSpec.instance(), - IndexIO.CURRENT_VERSION_ID, - 123L + testSegmentPathCreated(false); + } + + private void testSegmentPathCreated(boolean announceSegmentsOnZk) throws Exception + { + final String liveSegmentsPath = ZKPaths.makePath( + zkPaths.getLiveSegmentsPath(), + me.getName() ); - CountDownLatch loadLatch = new CountDownLatch(1); - CountDownLatch dropLatch = new CountDownLatch(1); + final EnsurePath mockEnsurePath = EasyMock.mock(EnsurePath.class); + final CuratorFramework mockCurator = EasyMock.mock(CuratorFramework.class); - SegmentLoadDropHandler segmentLoadDropHandler = new SegmentLoadDropHandler( - new SegmentLoaderConfig(), - EasyMock.createNiceMock(DataSegmentAnnouncer.class), - EasyMock.createNiceMock(SegmentManager.class) - ) - { - @Override - public void addSegment(DataSegment s, DataSegmentChangeCallback callback) - { - if (segment.getId().equals(s.getId())) { - loadLatch.countDown(); - callback.execute(); - } - } + if (announceSegmentsOnZk) { + EasyMock.expect(mockCurator.newNamespaceAwareEnsurePath(liveSegmentsPath)) + .andReturn(mockEnsurePath).once(); - @Override - public void removeSegment(DataSegment s, DataSegmentChangeCallback callback) - { - if (segment.getId().equals(s.getId())) { - dropLatch.countDown(); - callback.execute(); - } - } - }; + EasyMock.expect(mockCurator.getZookeeperClient()) + .andReturn(null).once(); - ZkCoordinator zkCoordinator = new ZkCoordinator( - segmentLoadDropHandler, - jsonMapper, + mockEnsurePath.ensure(EasyMock.anyObject()); + EasyMock.expectLastCall().once(); + } + + EasyMock.replay(mockCurator, mockEnsurePath); + final ZkCoordinator zkCoordinator = new ZkCoordinator( zkPaths, me, - curator, - new SegmentLoaderConfig() + mockCurator, + new BatchDataSegmentAnnouncerConfig() { + @Override + public boolean isSkipSegmentAnnouncementOnZk() + { + return !announceSegmentsOnZk; + } + } ); + zkCoordinator.start(); - - String segmentZkPath = ZKPaths.makePath(zkPaths.getLoadQueuePath(), me.getName(), segment.getId().toString()); - - curator - .create() - .creatingParentsIfNeeded() - .withMode(CreateMode.EPHEMERAL) - .forPath(segmentZkPath, jsonMapper.writeValueAsBytes(new SegmentChangeRequestLoad(segment))); - - loadLatch.await(); - - while (curator.checkExists().forPath(segmentZkPath) != null) { - Thread.sleep(100); - } - - curator - .create() - .creatingParentsIfNeeded() - .withMode(CreateMode.EPHEMERAL) - .forPath(segmentZkPath, jsonMapper.writeValueAsBytes(new SegmentChangeRequestDrop(segment))); - - dropLatch.await(); - - while (curator.checkExists().forPath(segmentZkPath) != null) { - Thread.sleep(100); - } - + EasyMock.verify(); zkCoordinator.stop(); } } diff --git a/server/src/test/java/org/apache/druid/server/coordinator/simulate/README.md b/server/src/test/java/org/apache/druid/server/coordinator/simulate/README.md index 16c7ee185bc..9e827083284 100644 --- a/server/src/test/java/org/apache/druid/server/coordinator/simulate/README.md +++ b/server/src/test/java/org/apache/druid/server/coordinator/simulate/README.md @@ -63,7 +63,6 @@ of the coordinator in these situations. interfaces to communicate with external dependencies have been provided as simple in-memory implementations: - communication with metadata store: `SegmentMetadataManager`, `MetadataRuleManager` - communication with historicals: `HttpClient`, `ServerInventoryView` - - `CuratorFramework`: provided as a mock as simulations of `CuratorLoadQueuePeon` are not supported yet 4. __Inventory__: The coordinator maintains an inventory view of the cluster state. Simulations can choose from two modes of inventory update - auto and manual. In auto update mode, any change made to the cluster is immediately reflected in the inventory view. In manual update mode, the inventory must be explicitly synchronized with the From 593c3b21503903478ca276acbaa1913710a416ca Mon Sep 17 00:00:00 2001 From: Vishesh Garg Date: Tue, 6 Aug 2024 20:58:08 +0530 Subject: [PATCH 3/6] Do not support non-idempotent aggregator in MSQ compaction (#16846) This PR adds checks for verification of DataSourceCompactionConfig and CompactionTask with msq engine to ensure: each aggregator in metricsSpec is idempotent metricsSpec is non-null when rollup is set to true Unit tests and existing compaction ITs have been updated accordingly. --- .../msq/indexing/MSQCompactionRunner.java | 56 +------ .../msq/indexing/MSQCompactionRunnerTest.java | 90 +++++------ .../common/task/CompactionRunner.java | 5 +- .../indexing/common/task/CompactionTask.java | 36 ++--- .../common/task/NativeCompactionRunner.java | 3 +- .../common/task/CompactionTaskTest.java | 48 ++---- .../duty/ITAutoCompactionTest.java | 146 +++++++++--------- .../indexing/ClientCompactionRunnerInfo.java | 36 ++++- .../segment/indexing/CombinedDataSchema.java | 64 -------- .../ClientCompactionRunnerInfoTest.java | 59 ++++++- .../segment/indexing/DataSchemaTest.java | 20 --- ...rdinatorCompactionConfigsResourceTest.java | 2 +- 12 files changed, 225 insertions(+), 340 deletions(-) delete mode 100644 server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java index 7b4d3235dc0..efc2cbb2afb 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/indexing/MSQCompactionRunner.java @@ -66,7 +66,6 @@ import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; -import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.virtual.ExpressionVirtualColumn; import org.apache.druid.server.coordinator.CompactionConfigValidationResult; @@ -120,15 +119,14 @@ public class MSQCompactionRunner implements CompactionRunner *
    *
  • partitionsSpec of type HashedParititionsSpec.
  • *
  • maxTotalRows in DynamicPartitionsSpec.
  • - *
  • rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.
  • - *
  • queryGranularity set to ALL in granularitySpec.
  • - *
  • Each metric has output column name same as the input name.
  • + *
  • rollup in granularitySpec set to false when metricsSpec is specified or true when it's null. + * Null is treated as true if metricsSpec exist and false if empty.
  • + *
  • any metric is non-idempotent, i.e. it defines some aggregatorFactory 'A' s.t. 'A != A.combiningFactory()'.
  • *
*/ @Override public CompactionConfigValidationResult validateCompactionTask( - CompactionTask compactionTask, - Map intervalToDataSchemaMap + CompactionTask compactionTask ) { List validationResults = new ArrayList<>(); @@ -144,57 +142,13 @@ public class MSQCompactionRunner implements CompactionRunner )); } validationResults.add(ClientCompactionRunnerInfo.validateMaxNumTasksForMSQ(compactionTask.getContext())); - validationResults.add(validateRolledUpSegments(intervalToDataSchemaMap)); + validationResults.add(ClientCompactionRunnerInfo.validateMetricsSpecForMSQ(compactionTask.getMetricsSpec())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() .orElse(CompactionConfigValidationResult.success()); } - /** - * Valides that there are no rolled-up segments where either: - *
    - *
  • aggregator factory differs from its combining factory
  • - *
  • input col name is different from the output name (non-idempotent)
  • - *
- */ - private CompactionConfigValidationResult validateRolledUpSegments(Map intervalToDataSchemaMap) - { - for (Map.Entry intervalDataSchema : intervalToDataSchemaMap.entrySet()) { - if (intervalDataSchema.getValue() instanceof CombinedDataSchema) { - CombinedDataSchema combinedDataSchema = (CombinedDataSchema) intervalDataSchema.getValue(); - if (combinedDataSchema.hasRolledUpSegments()) { - for (AggregatorFactory aggregatorFactory : combinedDataSchema.getAggregators()) { - // This is a conservative check as existing rollup may have been idempotent but the aggregator provided in - // compaction spec isn't. This would get properly compacted yet fails in the below pre-check. - if ( - !( - aggregatorFactory.getClass().equals(aggregatorFactory.getCombiningFactory().getClass()) && - ( - aggregatorFactory.requiredFields().isEmpty() || - (aggregatorFactory.requiredFields().size() == 1 && - aggregatorFactory.requiredFields() - .get(0) - .equals(aggregatorFactory.getName())) - ) - ) - ) { - // MSQ doesn't support rolling up already rolled-up segments when aggregate column name is different from - // the aggregated column name. This is because the aggregated values would then get overwritten by new - // values and the existing values would be lost. Note that if no rollup is specified in an index spec, - // the default value is true. - return CompactionConfigValidationResult.failure( - "MSQ: Rolled-up segments in compaction interval[%s].", - intervalDataSchema.getKey() - ); - } - } - } - } - } - return CompactionConfigValidationResult.success(); - } - @Override public CurrentSubTaskHolder getCurrentSubTaskHolder() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java index 6c5d1957265..d868ddf20e5 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/indexing/MSQCompactionRunnerTest.java @@ -42,7 +42,6 @@ import org.apache.druid.indexing.common.task.CompactionTask; import org.apache.druid.indexing.common.task.TuningConfigBuilder; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; -import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.GranularityType; import org.apache.druid.math.expr.ExprMacroTable; @@ -60,7 +59,6 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.data.CompressionFactory; import org.apache.druid.segment.data.CompressionStrategy; import org.apache.druid.segment.data.RoaringBitmapSerdeFactory; -import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec; import org.apache.druid.segment.transform.TransformSpec; @@ -131,7 +129,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -144,7 +142,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -157,7 +155,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -170,7 +168,7 @@ public class MSQCompactionRunnerTest null, null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -183,7 +181,7 @@ public class MSQCompactionRunnerTest new ClientCompactionTaskGranularitySpec(null, Granularities.ALL, null), null ); - Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); + Assert.assertTrue(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); } @Test @@ -196,7 +194,41 @@ public class MSQCompactionRunnerTest new ClientCompactionTaskGranularitySpec(null, null, false), AGGREGATORS.toArray(new AggregatorFactory[0]) ); - Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask, Collections.emptyMap()).isValid()); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + public void testRollupTrueWithoutMetricsSpecIsInValid() + { + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + Collections.emptyMap(), + new ClientCompactionTaskGranularitySpec(null, null, true), + null + ); + Assert.assertFalse(MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask).isValid()); + } + + @Test + public void testMSQEngineWithUnsupportedMetricsSpecIsInValid() + { + // Aggregators having different input and ouput column names are unsupported. + final String inputColName = "added"; + final String outputColName = "sum_added"; + CompactionTask compactionTask = createCompactionTask( + new DynamicPartitionsSpec(3, null), + null, + Collections.emptyMap(), + new ClientCompactionTaskGranularitySpec(null, null, null), + new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} + ); + CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask(compactionTask); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "MSQ: Non-idempotent aggregator[sum_added] not supported in 'metricsSpec'.", + validationResult.getReason() + ); } @Test @@ -345,48 +377,6 @@ public class MSQCompactionRunnerTest Assert.assertEquals(WorkerAssignmentStrategy.MAX, actualMSQSpec.getAssignmentStrategy()); } - @Test - public void testIntervalsWithRolledUpSegmentsAndNonIdempotentAggregatorFails() - { - final String inputColName = "added"; - final String outputColName = "sum_added"; - CompactionTask compactionTask = createCompactionTask( - null, - null, - Collections.emptyMap(), - null, - new AggregatorFactory[]{ - new LongSumAggregatorFactory( - outputColName, - inputColName - ) - } - ); - CombinedDataSchema dataSchema = new CombinedDataSchema( - DATA_SOURCE, - new TimestampSpec(TIMESTAMP_COLUMN, null, null), - new DimensionsSpec(DIMENSIONS), - new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)}, - new UniformGranularitySpec( - SEGMENT_GRANULARITY.getDefaultGranularity(), - null, - false, - Collections.singletonList(COMPACTION_INTERVAL) - ), - null, - true - ); - CompactionConfigValidationResult validationResult = MSQ_COMPACTION_RUNNER.validateCompactionTask( - compactionTask, - Collections.singletonMap(COMPACTION_INTERVAL, dataSchema) - ); - Assert.assertFalse(validationResult.isValid()); - Assert.assertEquals(validationResult.getReason(), StringUtils.format( - "MSQ: Rolled-up segments in compaction interval[%s].", - COMPACTION_INTERVAL - )); - } - private CompactionTask createCompactionTask( @Nullable PartitionsSpec partitionsSpec, @Nullable DimFilter dimFilter, diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java index 0abaeed8eb2..8d30a60d04e 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionRunner.java @@ -57,9 +57,6 @@ public interface CompactionRunner * Checks if the provided compaction config is supported by the runner. * The same validation is done at {@link org.apache.druid.msq.indexing.MSQCompactionRunner#validateCompactionTask} */ - CompactionConfigValidationResult validateCompactionTask( - CompactionTask compactionTask, - Map intervalToDataSchemaMap - ); + CompactionConfigValidationResult validateCompactionTask(CompactionTask compactionTask); } diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java index 68320387845..8659eb0f397 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/CompactionTask.java @@ -77,7 +77,6 @@ import org.apache.druid.segment.IndexSpec; import org.apache.druid.segment.QueryableIndex; import org.apache.druid.segment.column.ColumnHolder; import org.apache.druid.segment.incremental.AppendableIndexSpec; -import org.apache.druid.segment.indexing.CombinedDataSchema; import org.apache.druid.segment.indexing.DataSchema; import org.apache.druid.segment.indexing.TuningConfig; import org.apache.druid.segment.indexing.granularity.GranularitySpec; @@ -460,13 +459,11 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg transformSpec, metricsSpec, granularitySpec, - getMetricBuilder(), - compactionRunner + getMetricBuilder() ); registerResourceCloserOnAbnormalExit(compactionRunner.getCurrentSubTaskHolder()); - CompactionConfigValidationResult supportsCompactionConfig = - compactionRunner.validateCompactionTask(this, intervalDataSchemas); + CompactionConfigValidationResult supportsCompactionConfig = compactionRunner.validateCompactionTask(this); if (!supportsCompactionConfig.isValid()) { throw InvalidInput.exception("Compaction spec not supported. Reason[%s].", supportsCompactionConfig.getReason()); } @@ -488,8 +485,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @Nullable final ClientCompactionTaskTransformSpec transformSpec, @Nullable final AggregatorFactory[] metricsSpec, @Nullable final ClientCompactionTaskGranularitySpec granularitySpec, - final ServiceMetricEvent.Builder metricBuilder, - CompactionRunner compactionRunner + final ServiceMetricEvent.Builder metricBuilder ) throws IOException { final Iterable timelineSegments = retrieveRelevantTimelineHolders( @@ -553,8 +549,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg metricsSpec, granularitySpec == null ? new ClientCompactionTaskGranularitySpec(segmentGranularityToUse, null, null) - : granularitySpec.withSegmentGranularity(segmentGranularityToUse), - compactionRunner + : granularitySpec.withSegmentGranularity(segmentGranularityToUse) ); intervalDataSchemaMap.put(interval, dataSchema); } @@ -579,8 +574,7 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg dimensionsSpec, transformSpec, metricsSpec, - granularitySpec, - compactionRunner + granularitySpec ); return Collections.singletonMap(segmentProvider.interval, dataSchema); } @@ -610,17 +604,13 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg @Nullable DimensionsSpec dimensionsSpec, @Nullable ClientCompactionTaskTransformSpec transformSpec, @Nullable AggregatorFactory[] metricsSpec, - @Nonnull ClientCompactionTaskGranularitySpec granularitySpec, - @Nullable CompactionRunner compactionRunner + @Nonnull ClientCompactionTaskGranularitySpec granularitySpec ) { // Check index metadata & decide which values to propagate (i.e. carry over) for rollup & queryGranularity final ExistingSegmentAnalyzer existingSegmentAnalyzer = new ExistingSegmentAnalyzer( segments, - // For MSQ, always need rollup to check if there are some rollup segments already present. - compactionRunner instanceof NativeCompactionRunner - ? (granularitySpec.isRollup() == null) - : true, + granularitySpec.isRollup() == null, granularitySpec.getQueryGranularity() == null, dimensionsSpec == null, metricsSpec == null @@ -675,14 +665,13 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg finalMetricsSpec = metricsSpec; } - return new CombinedDataSchema( + return new DataSchema( dataSource, new TimestampSpec(ColumnHolder.TIME_COLUMN_NAME, "millis", null), finalDimensionsSpec, finalMetricsSpec, uniformGranularitySpec, - transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null), - existingSegmentAnalyzer.hasRolledUpSegments() + transformSpec == null ? null : new TransformSpec(transformSpec.getFilter(), null) ); } @@ -759,7 +748,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg // For processRollup: private boolean rollup = true; - private boolean hasRolledUpSegments = false; // For processQueryGranularity: private Granularity queryGranularity; @@ -827,11 +815,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg return rollup; } - public boolean hasRolledUpSegments() - { - return hasRolledUpSegments; - } - public Granularity getQueryGranularity() { if (!needQueryGranularity) { @@ -921,7 +904,6 @@ public class CompactionTask extends AbstractBatchIndexTask implements PendingSeg // Pick rollup value if all segments being compacted have the same, non-null, value otherwise set it to false final Boolean isIndexRollup = index.getMetadata().isRollup(); rollup = rollup && Boolean.valueOf(true).equals(isIndexRollup); - hasRolledUpSegments = hasRolledUpSegments || Boolean.valueOf(true).equals(isIndexRollup); } private void processQueryGranularity(final QueryableIndex index) diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java index 5aa7af71451..2074d14f0f9 100644 --- a/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java +++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/task/NativeCompactionRunner.java @@ -85,8 +85,7 @@ public class NativeCompactionRunner implements CompactionRunner @Override public CompactionConfigValidationResult validateCompactionTask( - CompactionTask compactionTask, - Map intervalToDataSchemaMap + CompactionTask compactionTask ) { return CompactionConfigValidationResult.success(); diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index 3a386bc4aa7..f9849b1483d 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -749,8 +749,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -811,8 +810,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -874,8 +872,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -938,8 +935,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1009,8 +1005,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1060,8 +1055,7 @@ public class CompactionTaskTest null, customMetricsSpec, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1104,8 +1098,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1155,8 +1148,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); NativeCompactionRunner.createIngestionSpecs( @@ -1186,8 +1178,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); NativeCompactionRunner.createIngestionSpecs( @@ -1228,8 +1219,7 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(new PeriodGranularity(Period.months(3), null, null), null, null), - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1273,8 +1263,7 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, new PeriodGranularity(Period.months(3), null, null), null), - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( dataSchemasForIntervals, @@ -1319,8 +1308,7 @@ public class CompactionTaskTest new PeriodGranularity(Period.months(3), null, null), null ), - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1367,8 +1355,7 @@ public class CompactionTaskTest null, null, null, - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1413,8 +1400,7 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1459,8 +1445,7 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, null, true), - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( @@ -1490,8 +1475,7 @@ public class CompactionTaskTest null, null, new ClientCompactionTaskGranularitySpec(null, null, null), - METRIC_BUILDER, - null + METRIC_BUILDER ); final List ingestionSpecs = NativeCompactionRunner.createIngestionSpecs( diff --git a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java index d09bced3313..8f070f33405 100644 --- a/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java +++ b/integration-tests/src/test/java/org/apache/druid/tests/coordinator/duty/ITAutoCompactionTest.java @@ -80,7 +80,6 @@ import org.testng.annotations.DataProvider; import org.testng.annotations.Guice; import org.testng.annotations.Test; -import javax.annotation.Nullable; import java.io.Closeable; import java.io.IOException; import java.io.InputStream; @@ -184,7 +183,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest new HllSketchBuildAggregatorFactory("HLLSketchBuild", "user", 12, TgtHllType.HLL_4.name(), null, false, false), new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", 128, 1000000000L, null) }, - false + false, + CompactionEngine.NATIVE ); // should now only have 1 row after compaction // added = null, count = 3, sum_added = 93.0 @@ -286,7 +286,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest ), new DoublesSketchAggregatorFactory("quantilesDoublesSketch", "delta", 128, 1000000000L, null) }, - false + false, + CompactionEngine.NATIVE ); // should now only have 1 row after compaction // added = null, count = 3, sum_added = 93 @@ -328,8 +329,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test(dataProvider = "engine") - public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics(CompactionEngine engine) throws Exception + @Test() + public void testAutoCompactionOnlyRowsWithoutMetricShouldAddNewMetrics() throws Exception { // added = 31, count = null, sum_added = null loadData(INDEX_TASK_WITHOUT_ROLLUP_FOR_PRESERVE_METRICS); @@ -357,7 +358,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest null, new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, false, - engine + CompactionEngine.NATIVE ); // should now only have 1 row after compaction // added = null, count = 2, sum_added = 62 @@ -480,7 +481,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest new UserCompactionTaskDimensionsConfig(DimensionsSpec.getDefaultSchemas(ImmutableList.of("language"))), null, new AggregatorFactory[] {new CountAggregatorFactory("count"), new LongSumAggregatorFactory("sum_added", "added")}, - false + false, + CompactionEngine.NATIVE ); // should now only have 1 row after compaction // added = null, count = 4, sum_added = 124 @@ -521,7 +523,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1)); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, Period.days(1), CompactionEngine.NATIVE); //...compacted into 1 new segment for 1 day. 1 day compacted and 1 day skipped/remains uncompacted. (3 total) forceTriggerAutoCompaction(3); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -539,7 +541,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest 0, 1, 1); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, CompactionEngine.NATIVE); //...compacted into 1 new segment for the remaining one day. 2 day compacted and 0 day uncompacted. (2 total) forceTriggerAutoCompaction(2); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -651,7 +653,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest verifySegmentsCount(4); verifyQuery(INDEX_QUERIES_RESOURCE); - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET); + submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, CompactionEngine.NATIVE); // ...should remains unchanged (4 total) forceTriggerAutoCompaction(4); verifyQuery(INDEX_QUERIES_RESOURCE); @@ -863,7 +865,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest Granularity newGranularity = Granularities.YEAR; // Set dropExisting to true // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); List expectedIntervalAfterCompaction = new ArrayList<>(); for (String interval : intervalsBeforeCompaction) { @@ -881,7 +889,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest // "interval": "2013-01-01T00:00:00.000Z/2014-01-01T00:00:00.000Z", newGranularity = Granularities.MONTH; // Set dropExisting to true - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), true, engine); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + true, + engine + ); // Since dropExisting is set to true... // Again data is only in two days @@ -950,7 +964,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest Granularity newGranularity = Granularities.YEAR; // Set dropExisting to false - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + false, + CompactionEngine.NATIVE + ); LOG.info("Auto compaction test with YEAR segment granularity"); @@ -967,7 +987,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest newGranularity = Granularities.DAY; // Set dropExisting to false - submitCompactionConfig(1000, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false); + submitCompactionConfig( + 1000, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + false, + CompactionEngine.NATIVE + ); LOG.info("Auto compaction test with DAY segment granularity"); @@ -1169,7 +1195,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest Granularity newGranularity = Granularities.YEAR; // Set dropExisting to false - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false); + submitCompactionConfig( + MAX_ROWS_PER_SEGMENT_COMPACTED, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + false, + CompactionEngine.NATIVE + ); List expectedIntervalAfterCompaction = new ArrayList<>(); // We wil have one segment with interval of 2013-01-01/2014-01-01 (compacted with YEAR) @@ -1195,7 +1227,13 @@ public class ITAutoCompactionTest extends AbstractIndexerTest newGranularity = Granularities.MONTH; // Set dropExisting to false - submitCompactionConfig(MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(newGranularity, null, null), false); + submitCompactionConfig( + MAX_ROWS_PER_SEGMENT_COMPACTED, + NO_SKIP_OFFSET, + new UserCompactionTaskGranularityConfig(newGranularity, null, null), + false, + CompactionEngine.NATIVE + ); // Since dropExisting is set to true... // This will submit a single compaction task for interval of 2013-01-01/2014-01-01 with MONTH granularity expectedIntervalAfterCompaction = new ArrayList<>(); @@ -1239,7 +1277,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest MAX_ROWS_PER_SEGMENT_COMPACTED, NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(Granularities.WEEK, null, null), - false + false, + CompactionEngine.NATIVE ); // Before compaction, we have segments with the interval 2013-08-01/2013-09-01 and 2013-09-01/2013-10-01 // We will compact the latest segment, 2013-09-01/2013-10-01, to WEEK. @@ -1319,8 +1358,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest } } - @Test(dataProvider = "engine") - public void testAutoCompactionDutyWithRollup(CompactionEngine engine) throws Exception + @Test() + public void testAutoCompactionDutyWithRollup() throws Exception { final ISOChronology chrono = ISOChronology.getInstance(DateTimes.inferTzFromString("America/Los_Angeles")); Map specs = ImmutableMap.of("%%GRANULARITYSPEC%%", new UniformGranularitySpec(Granularities.DAY, Granularities.DAY, false, ImmutableList.of(new Interval("2013-08-31/2013-09-02", chrono)))); @@ -1337,7 +1376,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest NO_SKIP_OFFSET, new UserCompactionTaskGranularityConfig(null, null, true), false, - engine + CompactionEngine.NATIVE ); forceTriggerAutoCompaction(2); queryAndResultFields = ImmutableMap.of( @@ -1470,7 +1509,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest null, new UserCompactionTaskTransformConfig(new SelectorDimFilter("page", "Striker Eureka", null)), null, - false + false, + CompactionEngine.NATIVE ); forceTriggerAutoCompaction(2); @@ -1517,7 +1557,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest null, null, new AggregatorFactory[] {new DoubleSumAggregatorFactory("double_sum_added", "added"), new LongSumAggregatorFactory("long_sum_added", "added")}, - false + false, + CompactionEngine.NATIVE ); forceTriggerAutoCompaction(2); @@ -1577,7 +1618,8 @@ public class ITAutoCompactionTest extends AbstractIndexerTest null, null, null, - false + false, + CompactionEngine.NATIVE ); // Compact the MONTH segment forceTriggerAutoCompaction(2); @@ -1679,57 +1721,31 @@ public class ITAutoCompactionTest extends AbstractIndexerTest queryHelper.testQueriesFromString(queryResponseTemplate); } - private void submitCompactionConfig(Integer maxRowsPerSegment, Period skipOffsetFromLatest) - throws Exception - { - submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, null); - } - private void submitCompactionConfig( Integer maxRowsPerSegment, Period skipOffsetFromLatest, - @Nullable CompactionEngine engine + CompactionEngine engine ) throws Exception { submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, null, engine); } - private void submitCompactionConfig( - Integer maxRowsPerSegment, - Period skipOffsetFromLatest, - UserCompactionTaskGranularityConfig granularitySpec - ) throws Exception - { - submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, null); - } - - private void submitCompactionConfig( Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec, - @Nullable CompactionEngine engine + CompactionEngine engine ) throws Exception { submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, false, engine); } - private void submitCompactionConfig( - Integer maxRowsPerSegment, - Period skipOffsetFromLatest, - UserCompactionTaskGranularityConfig granularitySpec, - boolean dropExisting - ) throws Exception - { - submitCompactionConfig(maxRowsPerSegment, skipOffsetFromLatest, granularitySpec, dropExisting, null); - } - private void submitCompactionConfig( Integer maxRowsPerSegment, Period skipOffsetFromLatest, UserCompactionTaskGranularityConfig granularitySpec, boolean dropExisting, - @Nullable CompactionEngine engine + CompactionEngine engine ) throws Exception { submitCompactionConfig( @@ -1744,28 +1760,6 @@ public class ITAutoCompactionTest extends AbstractIndexerTest ); } - private void submitCompactionConfig( - Integer maxRowsPerSegment, - Period skipOffsetFromLatest, - UserCompactionTaskGranularityConfig granularitySpec, - UserCompactionTaskDimensionsConfig dimensionsSpec, - UserCompactionTaskTransformConfig transformSpec, - AggregatorFactory[] metricsSpec, - boolean dropExisting - ) throws Exception - { - submitCompactionConfig( - maxRowsPerSegment, - skipOffsetFromLatest, - granularitySpec, - dimensionsSpec, - transformSpec, - metricsSpec, - dropExisting, - null - ); - } - private void submitCompactionConfig( Integer maxRowsPerSegment, Period skipOffsetFromLatest, @@ -1774,7 +1768,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest UserCompactionTaskTransformConfig transformSpec, AggregatorFactory[] metricsSpec, boolean dropExisting, - @Nullable CompactionEngine engine + CompactionEngine engine ) throws Exception { submitCompactionConfig( @@ -1799,7 +1793,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest UserCompactionTaskTransformConfig transformSpec, AggregatorFactory[] metricsSpec, boolean dropExisting, - @Nullable CompactionEngine engine + CompactionEngine engine ) throws Exception { DataSourceCompactionConfig dataSourceCompactionConfig = new DataSourceCompactionConfig( diff --git a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java index 0f92d99db10..806b35e9481 100644 --- a/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java +++ b/server/src/main/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfo.java @@ -32,6 +32,7 @@ import org.apache.druid.server.coordinator.DataSourceCompactionConfig; import javax.annotation.Nullable; import java.util.ArrayList; +import java.util.Arrays; import java.util.List; import java.util.Map; import java.util.Objects; @@ -102,9 +103,8 @@ public class ClientCompactionRunnerInfo *
    *
  • partitionsSpec of type HashedParititionsSpec.
  • *
  • maxTotalRows in DynamicPartitionsSpec.
  • - *
  • rollup set to false in granularitySpec when metricsSpec is specified. Null is treated as true.
  • - *
  • queryGranularity set to ALL in granularitySpec.
  • - *
  • Each metric has output column name same as the input name.
  • + *
  • rollup in granularitySpec set to false when metricsSpec is specified or true when it's empty.
  • + *
  • any metric is non-idempotent, i.e. it defines some aggregatorFactory 'A' s.t. 'A != A.combiningFactory()'.
  • *
*/ private static CompactionConfigValidationResult compactionConfigSupportedByMSQEngine(DataSourceCompactionConfig newConfig) @@ -120,6 +120,7 @@ public class ClientCompactionRunnerInfo )); } validationResults.add(validateMaxNumTasksForMSQ(newConfig.getTaskContext())); + validationResults.add(validateMetricsSpecForMSQ(newConfig.getMetricsSpec())); return validationResults.stream() .filter(result -> !result.isValid()) .findFirst() @@ -149,17 +150,23 @@ public class ClientCompactionRunnerInfo } /** - * Validate rollup is set to false in granularitySpec when metricsSpec is specified. + * Validate rollup in granularitySpec is set to true when metricsSpec is specified and false if it's null. + * If rollup set to null, all existing segments are analyzed, and it's set to true iff all segments have rollup + * set to true. */ public static CompactionConfigValidationResult validateRollupForMSQ( AggregatorFactory[] metricsSpec, @Nullable Boolean isRollup ) { - if (metricsSpec != null && isRollup != null && !isRollup) { + if (metricsSpec != null && metricsSpec.length != 0 && isRollup != null && !isRollup) { return CompactionConfigValidationResult.failure( "MSQ: 'granularitySpec.rollup' must be true if 'metricsSpec' is specified" ); + } else if ((metricsSpec == null || metricsSpec.length == 0) && isRollup != null && isRollup) { + return CompactionConfigValidationResult.failure( + "MSQ: 'granularitySpec.rollup' must be false if 'metricsSpec' is null" + ); } return CompactionConfigValidationResult.success(); } @@ -181,4 +188,23 @@ public class ClientCompactionRunnerInfo } return CompactionConfigValidationResult.success(); } + + /** + * Validate each metric is idempotent, i.e. it defines some aggregatorFactory 'A' s.t. 'A = A.combiningFactory()'. + */ + public static CompactionConfigValidationResult validateMetricsSpecForMSQ(AggregatorFactory[] metricsSpec) + { + if (metricsSpec == null) { + return CompactionConfigValidationResult.success(); + } + return Arrays.stream(metricsSpec) + .filter(aggregatorFactory -> !aggregatorFactory.equals(aggregatorFactory.getCombiningFactory())) + .findFirst() + .map(aggregatorFactory -> + CompactionConfigValidationResult.failure( + "MSQ: Non-idempotent aggregator[%s] not supported in 'metricsSpec'.", + aggregatorFactory.getName() + ) + ).orElse(CompactionConfigValidationResult.success()); + } } diff --git a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java b/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java deleted file mode 100644 index b2cb90bc0ce..00000000000 --- a/server/src/main/java/org/apache/druid/segment/indexing/CombinedDataSchema.java +++ /dev/null @@ -1,64 +0,0 @@ -/* - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, - * software distributed under the License is distributed on an - * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY - * KIND, either express or implied. See the License for the - * specific language governing permissions and limitations - * under the License. - */ - -package org.apache.druid.segment.indexing; - -import org.apache.druid.data.input.impl.DimensionsSpec; -import org.apache.druid.data.input.impl.TimestampSpec; -import org.apache.druid.query.aggregation.AggregatorFactory; -import org.apache.druid.segment.indexing.granularity.GranularitySpec; -import org.apache.druid.segment.transform.TransformSpec; - -import javax.annotation.Nullable; - -/** - * Class representing the combined DataSchema of a set of segments, currently used only by Compaction. - */ -public class CombinedDataSchema extends DataSchema -{ - private final boolean hasRolledUpSegments; - - public CombinedDataSchema( - String dataSource, - @Nullable TimestampSpec timestampSpec, - @Nullable DimensionsSpec dimensionsSpec, - AggregatorFactory[] aggregators, - GranularitySpec granularitySpec, - TransformSpec transformSpec, - @Nullable boolean hasRolledUpSegments - ) - { - super( - dataSource, - timestampSpec, - dimensionsSpec, - aggregators, - granularitySpec, - transformSpec, - null, - null - ); - this.hasRolledUpSegments = hasRolledUpSegments; - } - - public boolean hasRolledUpSegments() - { - return hasRolledUpSegments; - } -} diff --git a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java index 7742eaaf138..011a4640da3 100644 --- a/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java +++ b/server/src/test/java/org/apache/druid/client/indexing/ClientCompactionRunnerInfoTest.java @@ -52,7 +52,7 @@ public class ClientCompactionRunnerInfoTest @Test public void testMSQEngineWithHashedPartitionsSpecIsInvalid() { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( new HashedPartitionsSpec(100, null, null), Collections.emptyMap(), null, @@ -72,7 +72,7 @@ public class ClientCompactionRunnerInfoTest @Test public void testMSQEngineWithMaxTotalRowsIsInvalid() { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( new DynamicPartitionsSpec(100, 100L), Collections.emptyMap(), null, @@ -92,7 +92,7 @@ public class ClientCompactionRunnerInfoTest @Test public void testMSQEngineWithDynamicPartitionsSpecIsValid() { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( new DynamicPartitionsSpec(100, null), Collections.emptyMap(), null, @@ -105,7 +105,7 @@ public class ClientCompactionRunnerInfoTest @Test public void testMSQEngineWithDimensionRangePartitionsSpecIsValid() { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( new DimensionRangePartitionsSpec(100, null, ImmutableList.of("partitionDim"), false), Collections.emptyMap(), null, @@ -118,7 +118,7 @@ public class ClientCompactionRunnerInfoTest @Test public void testMSQEngineWithQueryGranularityAllIsValid() { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( new DynamicPartitionsSpec(3, null), Collections.emptyMap(), new UserCompactionTaskGranularityConfig(Granularities.ALL, Granularities.ALL, false), @@ -131,7 +131,7 @@ public class ClientCompactionRunnerInfoTest @Test public void testMSQEngineWithRollupFalseWithMetricsSpecIsInvalid() { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( new DynamicPartitionsSpec(3, null), Collections.emptyMap(), new UserCompactionTaskGranularityConfig(null, null, false), @@ -148,10 +148,53 @@ public class ClientCompactionRunnerInfoTest ); } + @Test + public void testMSQEngineWithRollupTrueWithoutMetricsSpecIsInvalid() + { + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(null, null, true), + null + ); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "MSQ: 'granularitySpec.rollup' must be false if 'metricsSpec' is null", + validationResult.getReason() + ); + } + + @Test + public void testMSQEngineWithUnsupportedMetricsSpecIsInvalid() + { + // Aggregators having combiningFactory different from the aggregatorFactory are unsupported. + final String inputColName = "added"; + final String outputColName = "sum_added"; + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( + new DynamicPartitionsSpec(3, null), + Collections.emptyMap(), + new UserCompactionTaskGranularityConfig(null, null, null), + new AggregatorFactory[]{new LongSumAggregatorFactory(outputColName, inputColName)} + ); + CompactionConfigValidationResult validationResult = ClientCompactionRunnerInfo.validateCompactionConfig( + compactionConfig, + CompactionEngine.NATIVE + ); + Assert.assertFalse(validationResult.isValid()); + Assert.assertEquals( + "MSQ: Non-idempotent aggregator[sum_added] not supported in 'metricsSpec'.", + validationResult.getReason() + ); + } + @Test public void testMSQEngineWithRollupNullWithMetricsSpecIsValid() { - DataSourceCompactionConfig compactionConfig = createCompactionConfig( + DataSourceCompactionConfig compactionConfig = createMSQCompactionConfig( new DynamicPartitionsSpec(3, null), Collections.emptyMap(), new UserCompactionTaskGranularityConfig(null, null, null), @@ -161,7 +204,7 @@ public class ClientCompactionRunnerInfoTest .isValid()); } - private static DataSourceCompactionConfig createCompactionConfig( + private static DataSourceCompactionConfig createMSQCompactionConfig( PartitionsSpec partitionsSpec, Map context, @Nullable UserCompactionTaskGranularityConfig granularitySpec, diff --git a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java index 87ddac50f01..78294fca0c4 100644 --- a/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java +++ b/server/src/test/java/org/apache/druid/segment/indexing/DataSchemaTest.java @@ -684,24 +684,4 @@ public class DataSchemaTest extends InitializedNullHandlingTest Assert.assertSame(oldSchema.getParserMap(), newSchema.getParserMap()); } - - @Test - public void testCombinedDataSchemaSetsHasRolledUpSegments() - { - CombinedDataSchema schema = new CombinedDataSchema( - IdUtilsTest.VALID_ID_CHARS, - new TimestampSpec("time", "auto", null), - DimensionsSpec.builder() - .setDimensions( - DimensionsSpec.getDefaultSchemas(ImmutableList.of("dimA", "dimB", "metric1")) - ) - .setDimensionExclusions(ImmutableList.of("dimC")) - .build(), - null, - new ArbitraryGranularitySpec(Granularities.DAY, ImmutableList.of(Intervals.of("2014/2015"))), - null, - true - ); - Assert.assertTrue(schema.hasRolledUpSegments()); - } } diff --git a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java index c44b9b2f358..24cbfbd9462 100644 --- a/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java +++ b/server/src/test/java/org/apache/druid/server/http/CoordinatorCompactionConfigsResourceTest.java @@ -202,7 +202,7 @@ public class CoordinatorCompactionConfigsResourceTest .withInputSegmentSizeBytes(1000L) .withSkipOffsetFromLatest(Period.hours(3)) .withGranularitySpec( - new UserCompactionTaskGranularityConfig(Granularities.DAY, null, true) + new UserCompactionTaskGranularityConfig(Granularities.DAY, null, false) ) .withEngine(CompactionEngine.MSQ) .build(); From 2b81c18fd74d7db15c5ed6defad059fbd8e6f71c Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 6 Aug 2024 21:59:38 +0530 Subject: [PATCH 4/6] Refactor SemanticCreator (#16700) Refactors the SemanticCreator annotation. Moves the interface to the semantic package. Create a SemanticUtils to hold logic for storing semantic maps. Add FrameMaker interface. --- .../BaseColumnarLongsBenchmark.java | 4 +- .../common/task/CompactionTaskTest.java | 33 +++-- .../semantic}/SemanticCreator.java | 8 +- .../druid/common/semantic/SemanticUtils.java | 90 ++++++++++++ .../apache/druid/error/DruidException.java | 11 ++ .../rowsandcols/ArrayListRowsAndColumns.java | 4 +- .../LazilyDecoratedRowsAndColumns.java | 4 +- .../query/rowsandcols/RowsAndColumns.java | 31 ---- .../QueryableIndexRowsAndColumns.java | 5 +- .../semantic/DefaultFrameMaker.java | 81 +++++++++++ .../rowsandcols/semantic/FrameMaker.java | 40 +++++ .../org/apache/druid/segment/IndexIO.java | 56 ++++--- .../druid/segment/QueryableIndexSegment.java | 19 ++- .../druid/segment/SimpleQueryableIndex.java | 11 +- .../druid/segment/column/BaseColumn.java | 8 + .../druid/segment/column/LongsColumn.java | 9 ++ .../BlockLayoutColumnarLongsSupplier.java | 10 +- .../druid/segment/data/ColumnarInts.java | 6 + .../druid/segment/data/ColumnarLongs.java | 13 +- .../segment/data/ColumnarLongsSerializer.java | 9 ++ .../CompressedNestedDataComplexColumn.java | 32 +++- .../semantic/SemanticCreatorUsageTest.java | 7 +- .../common/semantic/SemanticUtilsTest.java | 137 ++++++++++++++++++ .../rowsandcols/semantic/FrameMakerTest.java | 76 ++++++++++ .../IndexIONullColumnsCompatibilityTest.java | 38 +++-- .../IndexMergerLongestSharedDimOrderTest.java | 10 +- .../CompressedColumnarIntsSupplierTest.java | 6 + .../CompressedLongsAutoEncodingSerdeTest.java | 4 +- .../data/CompressedLongsSerdeTest.java | 4 +- 29 files changed, 640 insertions(+), 126 deletions(-) rename processing/src/main/java/org/apache/druid/{query/rowsandcols => common/semantic}/SemanticCreator.java (84%) create mode 100644 processing/src/main/java/org/apache/druid/common/semantic/SemanticUtils.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFrameMaker.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FrameMaker.java rename processing/src/test/java/org/apache/druid/{query/rowsandcols => common}/semantic/SemanticCreatorUsageTest.java (96%) create mode 100644 processing/src/test/java/org/apache/druid/common/semantic/SemanticUtilsTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FrameMakerTest.java diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/BaseColumnarLongsBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/BaseColumnarLongsBenchmark.java index f912f5e70b2..1a6fc81e4eb 100644 --- a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/BaseColumnarLongsBenchmark.java +++ b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/BaseColumnarLongsBenchmark.java @@ -304,9 +304,7 @@ public class BaseColumnarLongsBenchmark } serializer.open(); - for (long val : vals) { - serializer.add(val); - } + serializer.addAll(vals, 0, vals.length); serializer.writeTo(output, null); return (int) serializer.getSerializedSize(); } diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java index f9849b1483d..3c144929546 100644 --- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java +++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/CompactionTaskTest.java @@ -737,7 +737,7 @@ public class CompactionTaskTest ); provider.checkSegments(LockGranularity.TIME_CHUNK, ImmutableList.of()); } - + @Test public void testCreateIngestionSchema() throws IOException { @@ -1855,14 +1855,6 @@ public class CompactionTaskTest } } - final Metadata metadata = new Metadata( - null, - aggregatorFactories.toArray(new AggregatorFactory[0]), - null, - null, - null - ); - queryableIndexMap.put( entry.getValue(), new SimpleQueryableIndex( @@ -1871,9 +1863,21 @@ public class CompactionTaskTest null, columnMap, null, - metadata, false ) + { + @Override + public Metadata getMetadata() + { + return new Metadata( + null, + aggregatorFactories.toArray(new AggregatorFactory[0]), + null, + null, + null + ); + } + } ); } } @@ -1896,10 +1900,15 @@ public class CompactionTaskTest index.getBitmapFactoryForDimensions(), index.getColumns(), index.getFileMapper(), - null, false ) - ); + { + @Override + public Metadata getMetadata() + { + return null; + } + }); } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/SemanticCreator.java b/processing/src/main/java/org/apache/druid/common/semantic/SemanticCreator.java similarity index 84% rename from processing/src/main/java/org/apache/druid/query/rowsandcols/SemanticCreator.java rename to processing/src/main/java/org/apache/druid/common/semantic/SemanticCreator.java index bb1af0e4d9f..0142b3e8ed0 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/SemanticCreator.java +++ b/processing/src/main/java/org/apache/druid/common/semantic/SemanticCreator.java @@ -17,7 +17,9 @@ * under the License. */ -package org.apache.druid.query.rowsandcols; +package org.apache.druid.common.semantic; + +import org.apache.druid.query.rowsandcols.RowsAndColumns; import java.lang.annotation.ElementType; import java.lang.annotation.Retention; @@ -26,8 +28,8 @@ import java.lang.annotation.Target; /** * Annotation used to indicate that the method is used as a creator for a semantic interface. - * - * Used in conjuction with {@link RowsAndColumns#makeAsMap(Class)} to build maps for simplified implementation of + *

+ * Used in conjuction with {@link SemanticUtils#makeAsMap(Class)} to build maps for simplified implementation of * the {@link RowsAndColumns#as(Class)} method. */ @Retention(RetentionPolicy.RUNTIME) diff --git a/processing/src/main/java/org/apache/druid/common/semantic/SemanticUtils.java b/processing/src/main/java/org/apache/druid/common/semantic/SemanticUtils.java new file mode 100644 index 00000000000..4424b5fcccc --- /dev/null +++ b/processing/src/main/java/org/apache/druid/common/semantic/SemanticUtils.java @@ -0,0 +1,90 @@ +/* + * 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.common.semantic; + +import org.apache.druid.error.DruidException; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.HashMap; +import java.util.LinkedHashMap; +import java.util.Map; +import java.util.function.Function; + +public class SemanticUtils +{ + private static final Map, Map, Function>> OVERRIDES = new LinkedHashMap<>(); + + /** + * Allows the registration of overrides, which allows overriding of already existing mappings. + * This allows extensions to register mappings. + */ + @SuppressWarnings("unused") + public static void registerAsOverride(Class clazz, Class asInterface, Function fn) + { + final Map, Function> classOverrides = OVERRIDES.computeIfAbsent( + clazz, + theClazz -> new LinkedHashMap<>() + ); + + final Function oldVal = classOverrides.get(asInterface); + if (oldVal != null) { + throw DruidException.defensive( + "Attempt to side-override the same interface [%s] multiple times for the same class [%s].", + asInterface, + clazz + ); + } else { + classOverrides.put(asInterface, fn); + } + } + + public static Map, Function> makeAsMap(Class clazz) + { + final Map, Function> retVal = new HashMap<>(); + + for (Method method : clazz.getMethods()) { + if (method.isAnnotationPresent(SemanticCreator.class)) { + if (method.getParameterCount() != 0) { + throw DruidException.defensive("Method [%s] annotated with SemanticCreator was not 0-argument.", method); + } + + retVal.put(method.getReturnType(), arg -> { + try { + return method.invoke(arg); + } + catch (InvocationTargetException | IllegalAccessException e) { + throw DruidException.defensive().build(e, "Problem invoking method [%s]", method); + } + }); + } + } + + final Map, Function> classOverrides = OVERRIDES.get(clazz); + if (classOverrides != null) { + for (Map.Entry, Function> overrideEntry : classOverrides.entrySet()) { + //noinspection unchecked + retVal.put(overrideEntry.getKey(), (Function) overrideEntry.getValue()); + } + } + + return retVal; + } +} diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index a04f3f6512c..f4cc3065c7f 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -176,6 +176,17 @@ public class DruidException extends RuntimeException return defensive().build(format, args); } + /** + * Build a "defensive" exception, this is an exception that should never actually be triggered, but we are + * throwing it inside a defensive check. + * + * @return A builder for a defensive exception. + */ + public static DruidException defensive(Throwable cause, String format, Object... args) + { + return defensive().build(cause, format, args); + } + /** * Build a "defensive" exception, this is an exception that should never actually be triggered. Throw to * allow messages to be seen by developers diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java index 05b9dee5458..3b5541ca5bb 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/ArrayListRowsAndColumns.java @@ -23,6 +23,8 @@ import it.unimi.dsi.fastutil.Arrays; import it.unimi.dsi.fastutil.ints.IntArrayList; import it.unimi.dsi.fastutil.ints.IntComparator; import it.unimi.dsi.fastutil.ints.IntList; +import org.apache.druid.common.semantic.SemanticCreator; +import org.apache.druid.common.semantic.SemanticUtils; import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.guava.Sequences; import org.apache.druid.query.operator.ColumnWithDirection; @@ -73,7 +75,7 @@ import java.util.function.Function; public class ArrayListRowsAndColumns implements AppendableRowsAndColumns { @SuppressWarnings("rawtypes") - private static final Map, Function> AS_MAP = RowsAndColumns + private static final Map, Function> AS_MAP = SemanticUtils .makeAsMap(ArrayListRowsAndColumns.class); private final ArrayList rows; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java index 0dae40467f3..ce199a7803c 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/LazilyDecoratedRowsAndColumns.java @@ -20,6 +20,8 @@ package org.apache.druid.query.rowsandcols; import com.google.common.collect.ImmutableList; +import org.apache.druid.common.semantic.SemanticCreator; +import org.apache.druid.common.semantic.SemanticUtils; import org.apache.druid.frame.Frame; import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; import org.apache.druid.frame.key.KeyColumn; @@ -66,7 +68,7 @@ import java.util.function.Function; public class LazilyDecoratedRowsAndColumns implements RowsAndColumns { - private static final Map, Function> AS_MAP = RowsAndColumns + private static final Map, Function> AS_MAP = SemanticUtils .makeAsMap(LazilyDecoratedRowsAndColumns.class); private RowsAndColumns base; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java index d139265d147..7b6a1f6215d 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/RowsAndColumns.java @@ -19,19 +19,13 @@ package org.apache.druid.query.rowsandcols; -import org.apache.druid.error.DruidException; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.query.rowsandcols.semantic.FramedOnHeapAggregatable; import javax.annotation.Nonnull; import javax.annotation.Nullable; -import java.lang.reflect.InvocationTargetException; -import java.lang.reflect.Method; import java.util.Collection; -import java.util.HashMap; -import java.util.Map; -import java.util.function.Function; /** * An interface representing a chunk of RowsAndColumns. Essentially a RowsAndColumns is just a batch of rows @@ -75,31 +69,6 @@ public interface RowsAndColumns return retVal; } - static Map, Function> makeAsMap(Class clazz) - { - Map, Function> retVal = new HashMap<>(); - - for (Method method : clazz.getMethods()) { - if (method.isAnnotationPresent(SemanticCreator.class)) { - if (method.getParameterCount() != 0) { - throw DruidException.defensive("Method [%s] annotated with SemanticCreator was not 0-argument.", method); - } - - retVal.put(method.getReturnType(), arg -> { - try { - return method.invoke(arg); - } - catch (InvocationTargetException | IllegalAccessException e) { - throw DruidException.defensive().build(e, "Problem invoking method [%s]", method); - } - }); - } - } - - return retVal; - } - - /** * The set of column names available from the RowsAndColumns * diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java index 209d4430b1d..73fc72a1ee4 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/QueryableIndexRowsAndColumns.java @@ -19,10 +19,11 @@ package org.apache.druid.query.rowsandcols.concrete; +import org.apache.druid.common.semantic.SemanticCreator; +import org.apache.druid.common.semantic.SemanticUtils; import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.rowsandcols.RowsAndColumns; -import org.apache.druid.query.rowsandcols.SemanticCreator; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.segment.CloseableShapeshifter; import org.apache.druid.segment.QueryableIndex; @@ -41,7 +42,7 @@ import java.util.function.Function; public class QueryableIndexRowsAndColumns implements RowsAndColumns, AutoCloseable, CloseableShapeshifter { - private static final Map, Function> AS_MAP = RowsAndColumns + private static final Map, Function> AS_MAP = SemanticUtils .makeAsMap(QueryableIndexRowsAndColumns.class); private final QueryableIndex index; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFrameMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFrameMaker.java new file mode 100644 index 00000000000..204b5bd8548 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/DefaultFrameMaker.java @@ -0,0 +1,81 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.allocation.ArenaMemoryAllocatorFactory; +import org.apache.druid.frame.write.FrameWriter; +import org.apache.druid.frame.write.FrameWriters; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.ColumnSelectorFactory; +import org.apache.druid.segment.column.RowSignature; + +import java.util.Collections; +import java.util.concurrent.atomic.AtomicInteger; + +public class DefaultFrameMaker implements FrameMaker +{ + private final RowsAndColumns rac; + + public DefaultFrameMaker(RowsAndColumns rac) + { + this.rac = rac; + } + + @Override + public RowSignature computeSignature() + { + final RowSignature.Builder signatureBuilder = RowSignature.builder(); + for (String column : rac.getColumnNames()) { + final Column racColumn = rac.findColumn(column); + if (racColumn == null) { + continue; + } + signatureBuilder.add(column, racColumn.toAccessor().getType()); + } + + return signatureBuilder.build(); + } + + @Override + public Frame toColumnBasedFrame() + { + final AtomicInteger rowId = new AtomicInteger(0); + final int numRows = rac.numRows(); + final ColumnSelectorFactoryMaker csfm = ColumnSelectorFactoryMaker.fromRAC(rac); + final ColumnSelectorFactory selectorFactory = csfm.make(rowId); + + final ArenaMemoryAllocatorFactory memFactory = new ArenaMemoryAllocatorFactory(200 << 20); // 200 MB + + final FrameWriter frameWriter = FrameWriters.makeColumnBasedFrameWriterFactory( + memFactory, + computeSignature(), + Collections.emptyList() + ).newFrameWriter(selectorFactory); + + rowId.set(0); + for (; rowId.get() < numRows; rowId.incrementAndGet()) { + frameWriter.addSelection(); + } + + return Frame.wrap(frameWriter.toByteArray()); + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FrameMaker.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FrameMaker.java new file mode 100644 index 00000000000..095bfe1ed87 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/semantic/FrameMaker.java @@ -0,0 +1,40 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.frame.Frame; +import org.apache.druid.query.rowsandcols.RowsAndColumns; +import org.apache.druid.segment.column.RowSignature; + +public interface FrameMaker +{ + static FrameMaker fromRAC(RowsAndColumns rac) + { + FrameMaker retVal = rac.as(FrameMaker.class); + if (retVal == null) { + retVal = new DefaultFrameMaker(rac); + } + return retVal; + } + + RowSignature computeSignature(); + + Frame toColumnBasedFrame(); +} diff --git a/processing/src/main/java/org/apache/druid/segment/IndexIO.java b/processing/src/main/java/org/apache/druid/segment/IndexIO.java index dd0ac9ab117..966de405206 100644 --- a/processing/src/main/java/org/apache/druid/segment/IndexIO.java +++ b/processing/src/main/java/org/apache/druid/segment/IndexIO.java @@ -510,9 +510,15 @@ public class IndexIO new ConciseBitmapFactory(), columns, index.getFileMapper(), - null, lazy - ); + ) + { + @Override + public Metadata getMetadata() + { + return null; + } + }; } private Supplier getColumnHolderSupplier(ColumnBuilder builder, boolean lazy) @@ -604,25 +610,6 @@ public class IndexIO allDims = null; } - Metadata metadata = null; - ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); - if (metadataBB != null) { - try { - metadata = mapper.readValue( - SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), - Metadata.class - ); - } - catch (JsonParseException | JsonMappingException ex) { - // Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which - // is no longer supported then it is OK to not use the metadata instead of failing segment loading - log.warn(ex, "Failed to load metadata for segment [%s]", inDir); - } - catch (IOException ex) { - throw new IOException("Failed to read metadata", ex); - } - } - Map> columns = new LinkedHashMap<>(); // Register the time column @@ -663,9 +650,32 @@ public class IndexIO segmentBitmapSerdeFactory.getBitmapFactory(), columns, smooshedFiles, - metadata, lazy - ); + ) + { + @Override + public Metadata getMetadata() + { + try { + ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); + if (metadataBB != null) { + return mapper.readValue( + SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), + Metadata.class + ); + } + } + catch (JsonParseException | JsonMappingException ex) { + // Any jackson deserialization errors are ignored e.g. if metadata contains some aggregator which + // is no longer supported then it is OK to not use the metadata instead of failing segment loading + log.warn(ex, "Failed to load metadata for segment [%s]", inDir); + } + catch (IOException ex) { + log.warn(ex, "Failed to read metadata for segment [%s]", inDir); + } + return null; + } + }; log.debug("Mapped v9 index[%s] in %,d millis", inDir, System.currentTimeMillis() - startTime); diff --git a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java index 9d75748b416..b8d4d2d16cf 100644 --- a/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java +++ b/processing/src/main/java/org/apache/druid/segment/QueryableIndexSegment.java @@ -19,17 +19,24 @@ package org.apache.druid.segment; +import org.apache.druid.common.semantic.SemanticCreator; +import org.apache.druid.common.semantic.SemanticUtils; import org.apache.druid.query.rowsandcols.concrete.QueryableIndexRowsAndColumns; import org.apache.druid.timeline.SegmentId; import org.joda.time.Interval; import javax.annotation.Nonnull; import javax.annotation.Nullable; +import java.util.Map; +import java.util.function.Function; /** */ public class QueryableIndexSegment implements Segment { + private static final Map, Function> AS_MAP = SemanticUtils + .makeAsMap(QueryableIndexSegment.class); + private final QueryableIndex index; private final QueryableIndexStorageAdapter storageAdapter; private final SegmentId segmentId; @@ -77,10 +84,18 @@ public class QueryableIndexSegment implements Segment @Override public T as(@Nonnull Class clazz) { - if (CloseableShapeshifter.class.equals(clazz)) { - return (T) new QueryableIndexRowsAndColumns(index); + final Function fn = AS_MAP.get(clazz); + if (fn != null) { + return (T) fn.apply(this); } return Segment.super.as(clazz); } + + @SemanticCreator + @SuppressWarnings("unused") + public CloseableShapeshifter toCloseableShapeshifter() + { + return new QueryableIndexRowsAndColumns(index); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java index 924c7911f8a..013a634fdc4 100644 --- a/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java +++ b/processing/src/main/java/org/apache/druid/segment/SimpleQueryableIndex.java @@ -38,7 +38,7 @@ import java.util.Map; /** * */ -public class SimpleQueryableIndex implements QueryableIndex +public abstract class SimpleQueryableIndex implements QueryableIndex { private final Interval dataInterval; private final List columnNames; @@ -46,8 +46,6 @@ public class SimpleQueryableIndex implements QueryableIndex private final BitmapFactory bitmapFactory; private final Map> columns; private final SmooshedFileMapper fileMapper; - @Nullable - private final Metadata metadata; private final Supplier> dimensionHandlers; public SimpleQueryableIndex( @@ -56,7 +54,6 @@ public class SimpleQueryableIndex implements QueryableIndex BitmapFactory bitmapFactory, Map> columns, SmooshedFileMapper fileMapper, - @Nullable Metadata metadata, boolean lazy ) { @@ -73,7 +70,6 @@ public class SimpleQueryableIndex implements QueryableIndex this.bitmapFactory = bitmapFactory; this.columns = columns; this.fileMapper = fileMapper; - this.metadata = metadata; if (lazy) { this.dimensionHandlers = Suppliers.memoize(() -> initDimensionHandlers(availableDimensions)); @@ -141,10 +137,7 @@ public class SimpleQueryableIndex implements QueryableIndex } @Override - public Metadata getMetadata() - { - return metadata; - } + public abstract Metadata getMetadata(); @Override public Map getDimensionHandlers() diff --git a/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java b/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java index f22693365e1..4829ed14599 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/BaseColumn.java @@ -26,6 +26,7 @@ import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorObjectSelector; import org.apache.druid.segment.vector.VectorValueSelector; +import javax.annotation.Nullable; import java.io.Closeable; public interface BaseColumn extends Closeable @@ -41,4 +42,11 @@ public interface BaseColumn extends Closeable { throw new UOE("Cannot make VectorObjectSelector for column with class[%s]", getClass().getName()); } + + @SuppressWarnings("unused") + @Nullable + default T as(Class clazz) + { + return null; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/column/LongsColumn.java b/processing/src/main/java/org/apache/druid/segment/column/LongsColumn.java index 6f17dfb7c01..1f88ab044d1 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/LongsColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/LongsColumn.java @@ -28,6 +28,8 @@ import org.apache.druid.segment.data.ReadableOffset; import org.apache.druid.segment.vector.ReadableVectorOffset; import org.apache.druid.segment.vector.VectorValueSelector; +import javax.annotation.Nullable; + /** */ public class LongsColumn implements NumericColumn @@ -75,6 +77,13 @@ public class LongsColumn implements NumericColumn return column.get(rowNum); } + @Override + @Nullable + public T as(Class clazz) + { + return column.as(clazz); + } + @Override public void close() { diff --git a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java index 6fe04fbd31f..36dbf5f5309 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java +++ b/processing/src/main/java/org/apache/druid/segment/data/BlockLayoutColumnarLongsSupplier.java @@ -47,7 +47,7 @@ public class BlockLayoutColumnarLongsSupplier implements Supplier CompressionStrategy strategy ) { - baseLongBuffers = GenericIndexed.read(fromBuffer, DecompressingByteBufferObjectStrategy.of(order, strategy)); + this.baseLongBuffers = GenericIndexed.read(fromBuffer, DecompressingByteBufferObjectStrategy.of(order, strategy)); this.totalSize = totalSize; this.sizePer = sizePer; this.baseReader = reader; @@ -156,6 +156,12 @@ public class BlockLayoutColumnarLongsSupplier implements Supplier @Override public void get(final long[] out, final int start, final int length) + { + get(out, 0, start, length); + } + + @Override + public void get(long[] out, int offset, int start, int length) { // division + remainder is optimized by the compiler so keep those together int bufferNum = start / sizePer; @@ -169,7 +175,7 @@ public class BlockLayoutColumnarLongsSupplier implements Supplier } final int limit = Math.min(length - p, sizePer - bufferIndex); - reader.read(out, p, bufferIndex, limit); + reader.read(out, offset + p, bufferIndex, limit); p += limit; bufferNum++; bufferIndex = 0; diff --git a/processing/src/main/java/org/apache/druid/segment/data/ColumnarInts.java b/processing/src/main/java/org/apache/druid/segment/data/ColumnarInts.java index dc2adbbb671..e4633d03298 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/ColumnarInts.java +++ b/processing/src/main/java/org/apache/druid/segment/data/ColumnarInts.java @@ -28,4 +28,10 @@ import java.io.Closeable; */ public interface ColumnarInts extends IndexedInts, Closeable { + default void get(int[] out, int offset, int start, int length) + { + for (int i = 0; i < length; i++) { + out[offset + i] = get(i + start); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongs.java b/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongs.java index 256c9934a21..6d8162ef267 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongs.java +++ b/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongs.java @@ -46,9 +46,14 @@ public interface ColumnarLongs extends Closeable long get(int index); default void get(long[] out, int start, int length) + { + get(out, 0, start, length); + } + + default void get(long[] out, int offset, int start, int length) { for (int i = 0; i < length; i++) { - out[i] = get(i + start); + out[offset + i] = get(i + start); } } @@ -62,6 +67,12 @@ public interface ColumnarLongs extends Closeable @Override void close(); + @Nullable + default T as(Class clazz) + { + return null; + } + default ColumnValueSelector makeColumnValueSelector(ReadableOffset offset, ImmutableBitmap nullValueBitmap) { if (nullValueBitmap.isEmpty()) { diff --git a/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongsSerializer.java b/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongsSerializer.java index 05cf26439e3..2166874a9f3 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongsSerializer.java +++ b/processing/src/main/java/org/apache/druid/segment/data/ColumnarLongsSerializer.java @@ -29,6 +29,15 @@ import java.io.IOException; public interface ColumnarLongsSerializer extends Serializer { void open() throws IOException; + int size(); + void add(long value) throws IOException; + + default void addAll(long[] values, int start, int end) throws IOException + { + for (int i = start; i < end; ++i) { + add(values[i]); + } + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java index 7faf837db1c..5f1d765bb56 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/CompressedNestedDataComplexColumn.java @@ -452,7 +452,10 @@ public abstract class CompressedNestedDataComplexColumn= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { - throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); } - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField, arrayFieldIndex).getColumn(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder( + arrayField, + arrayFieldIndex + ).getColumn(); ColumnValueSelector arraySelector = col.makeColumnValueSelector(readableOffset); return new ColumnValueSelector() { @@ -634,9 +642,14 @@ public abstract class CompressedNestedDataComplexColumn= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { - throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); } - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField, arrayFieldIndex).getColumn(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder( + arrayField, + arrayFieldIndex + ).getColumn(); VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); return new VectorObjectSelector() @@ -702,9 +715,14 @@ public abstract class CompressedNestedDataComplexColumn= 0) { final int elementNumber = ((NestedPathArrayElement) lastPath).getIndex(); if (elementNumber < 0) { - throw new IAE("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.INVALID_INPUT) + .build("Cannot make array element selector for path [%s], negative array index not supported for this selector", path); } - DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder(arrayField, arrayFieldIndex).getColumn(); + DictionaryEncodedColumn col = (DictionaryEncodedColumn) getColumnHolder( + arrayField, + arrayFieldIndex + ).getColumn(); VectorObjectSelector arraySelector = col.makeVectorObjectSelector(readableOffset); return new VectorValueSelector() diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SemanticCreatorUsageTest.java b/processing/src/test/java/org/apache/druid/common/semantic/SemanticCreatorUsageTest.java similarity index 96% rename from processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SemanticCreatorUsageTest.java rename to processing/src/test/java/org/apache/druid/common/semantic/SemanticCreatorUsageTest.java index b5de751651e..0dd61fb4b3e 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/SemanticCreatorUsageTest.java +++ b/processing/src/test/java/org/apache/druid/common/semantic/SemanticCreatorUsageTest.java @@ -17,10 +17,9 @@ * under the License. */ -package org.apache.druid.query.rowsandcols.semantic; +package org.apache.druid.common.semantic; import org.apache.druid.java.util.common.StringUtils; -import org.apache.druid.query.rowsandcols.SemanticCreator; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -80,7 +79,7 @@ public class SemanticCreatorUsageTest /** * {@link SemanticCreator} must return with an interface. - * + *

* An exact implementation may indicate that some interface methods might be missing. */ @Test @@ -95,7 +94,7 @@ public class SemanticCreatorUsageTest /** * {@link SemanticCreator} method names must follow the naming pattern toReturnType(). - * + *

* For example: a method returning with a type of Ball should be named as "toBall" */ @Test diff --git a/processing/src/test/java/org/apache/druid/common/semantic/SemanticUtilsTest.java b/processing/src/test/java/org/apache/druid/common/semantic/SemanticUtilsTest.java new file mode 100644 index 00000000000..d26670e83fc --- /dev/null +++ b/processing/src/test/java/org/apache/druid/common/semantic/SemanticUtilsTest.java @@ -0,0 +1,137 @@ +/* + * 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.common.semantic; + +import org.apache.druid.error.DruidException; +import org.apache.druid.segment.CloseableShapeshifter; +import org.junit.Assert; +import org.junit.Test; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Map; +import java.util.function.Function; + +public class SemanticUtilsTest +{ + @Test + public void testInvalidParameters() + { + Assert.assertThrows( + DruidException.class, + () -> SemanticUtils.makeAsMap(InvalidShapeshifter.class) + ); + } + + @Test + public void testValidParameters() + { + TestShapeshifter testShapeshifter = new TestShapeshifter(); + Assert.assertTrue(testShapeshifter.as(A.class) instanceof A); + } + + @Test + public void testOverrideForNewMapping() + { + SemanticUtils.registerAsOverride( + TestShapeshifter.class, + OverrideClass.class, + (testShapeshifter) -> new OverrideClass() + ); + TestShapeshifter testShapeshifter = new TestShapeshifter(); + Assert.assertTrue(testShapeshifter.as(A.class) instanceof A); + Assert.assertTrue(testShapeshifter.as(OverrideClass.class) instanceof OverrideClass); + } + + @Test + public void testOverrideForExistingMapping() + { + SemanticUtils.registerAsOverride( + TestShapeshifter.class, + A.class, + (testShapeshifter) -> new OverrideClass() + ); + TestShapeshifter testShapeshifter = new TestShapeshifter(); + Assert.assertTrue(testShapeshifter.as(A.class) instanceof OverrideClass); + } + + static class TestShapeshifter implements CloseableShapeshifter + { + private final Map, Function> asMap; + + public TestShapeshifter() + { + this.asMap = SemanticUtils.makeAsMap(TestShapeshifter.class); + } + + @SuppressWarnings("unchecked") + @Override + @Nullable + public T as(@Nonnull Class clazz) + { + //noinspection ReturnOfNull + return (T) asMap.getOrDefault(clazz, arg -> null).apply(this); + } + + @Override + public void close() + { + } + + @SemanticCreator + public AInterface toAInterface() + { + return new A(); + } + } + + static class InvalidShapeshifter implements CloseableShapeshifter + { + @Nullable + @Override + public T as(@Nonnull Class clazz) + { + return null; + } + + @Override + public void close() + { + } + + @SemanticCreator + public AInterface toAInterface(String invalidParameter) + { + return new A(); + } + } + + interface AInterface + { + } + + static class A implements AInterface + { + } + + static class OverrideClass extends A + { + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FrameMakerTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FrameMakerTest.java new file mode 100644 index 00000000000..e0a448884b2 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/FrameMakerTest.java @@ -0,0 +1,76 @@ +/* + * 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.rowsandcols.semantic; + +import org.apache.druid.frame.Frame; +import org.apache.druid.query.rowsandcols.ArrayListRowsAndColumnsTest; +import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; +import org.apache.druid.query.rowsandcols.column.ColumnAccessor; +import org.apache.druid.query.rowsandcols.concrete.ColumnBasedFrameRowsAndColumns; +import org.apache.druid.segment.column.ColumnType; +import org.apache.druid.segment.column.RowSignature; +import org.junit.Assert; +import org.junit.Test; + +public class FrameMakerTest +{ + public static RowSignature ROW_SIGNATURE = RowSignature.builder() + .add("dim1", ColumnType.STRING) + .add("dim2", ColumnType.STRING) + .add("dim3", ColumnType.STRING) + .add("m1", ColumnType.LONG) + .add("m2", ColumnType.LONG) + .build(); + + @Test + public void testFrameMaker() + { + final MapOfColumnsRowsAndColumns mapOfColumnsRowsAndColumns = MapOfColumnsRowsAndColumns + .builder() + .add("dim1", ColumnType.STRING, "a", "b", "c") + .add("dim2", ColumnType.STRING, "m", "d", "e") + .add("dim3", ColumnType.STRING, "a") + .add("m1", ColumnType.LONG, 1L, 2L, 3L) + .add("m2", ColumnType.LONG, 52L, 42L) + .build(); + + final FrameMaker frameMaker = FrameMaker.fromRAC(ArrayListRowsAndColumnsTest.MAKER.apply(mapOfColumnsRowsAndColumns)); + + Assert.assertEquals(ROW_SIGNATURE, frameMaker.computeSignature()); + + final Frame frame = frameMaker.toColumnBasedFrame(); + ColumnBasedFrameRowsAndColumns columnBasedFrameRowsAndColumns = new ColumnBasedFrameRowsAndColumns( + frame, + frameMaker.computeSignature() + ); + for (String columnName : mapOfColumnsRowsAndColumns.getColumnNames()) { + ColumnAccessor expectedColumn = mapOfColumnsRowsAndColumns.findColumn(columnName).toAccessor(); + ColumnAccessor actualColumn = columnBasedFrameRowsAndColumns.findColumn(columnName).toAccessor(); + + for (int i = 0; i < expectedColumn.numRows(); i++) { + Assert.assertEquals( + expectedColumn.getObject(i), + actualColumn.getObject(i) + ); + } + } + Assert.assertEquals(3, frame.numRows()); + } +} diff --git a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java index 2e328449075..703de4439e4 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexIONullColumnsCompatibilityTest.java @@ -31,6 +31,7 @@ import com.google.common.io.Files; import com.google.common.primitives.Ints; import org.apache.druid.data.input.MapBasedInputRow; import org.apache.druid.data.input.impl.DimensionsSpec; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.java.util.common.io.smoosh.Smoosh; @@ -184,20 +185,6 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling segmentBitmapSerdeFactory = new BitmapSerde.LegacyBitmapSerdeFactory(); } - Metadata metadata = null; - ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); - if (metadataBB != null) { - try { - metadata = mapper.readValue( - IndexIO.SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), - Metadata.class - ); - } - catch (IOException ex) { - throw new IOException("Failed to read metadata", ex); - } - } - Map> columns = new HashMap<>(); for (String columnName : cols) { @@ -251,9 +238,28 @@ public class IndexIONullColumnsCompatibilityTest extends InitializedNullHandling segmentBitmapSerdeFactory.getBitmapFactory(), columns, smooshedFiles, - metadata, lazy - ); + ) + { + @Override + public Metadata getMetadata() + { + try { + ByteBuffer metadataBB = smooshedFiles.mapFile("metadata.drd"); + if (metadataBB != null) { + return mapper.readValue( + IndexIO.SERIALIZER_UTILS.readBytes(metadataBB, metadataBB.remaining()), + Metadata.class + ); + } else { + return null; + } + } + catch (IOException ex) { + throw DruidException.defensive(ex, "Failed to read metadata"); + } + } + }; return index; } diff --git a/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java b/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java index 2bb84c0eeb9..1fedf260553 100644 --- a/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java +++ b/processing/src/test/java/org/apache/druid/segment/IndexMergerLongestSharedDimOrderTest.java @@ -167,11 +167,15 @@ public class IndexMergerLongestSharedDimOrderTest mockBitmapFactory, ImmutableMap.of(ColumnHolder.TIME_COLUMN_NAME, mockSupplier), mockSmooshedFileMapper, - null, true ) + { + @Override + public Metadata getMetadata() + { + return null; + } + } ); } } - - diff --git a/processing/src/test/java/org/apache/druid/segment/data/CompressedColumnarIntsSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/data/CompressedColumnarIntsSupplierTest.java index 01c9cc26dca..945f86eb8ef 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/CompressedColumnarIntsSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/CompressedColumnarIntsSupplierTest.java @@ -36,6 +36,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.nio.IntBuffer; import java.nio.channels.Channels; +import java.util.Arrays; import java.util.Random; import java.util.concurrent.CountDownLatch; import java.util.concurrent.ThreadLocalRandom; @@ -290,6 +291,11 @@ public class CompressedColumnarIntsSupplierTest extends CompressionStrategyTest indices[i] = i; } + int[] offsetValues = new int[columnarInts.size() + 1]; + columnarInts.get(offsetValues, 1, 0, columnarInts.size()); + Assert.assertEquals(0, offsetValues[0]); + Assert.assertArrayEquals(vals, Arrays.copyOfRange(offsetValues, 1, offsetValues.length)); + // random access, limited to 1000 elements for large lists (every element would take too long) IntArrays.shuffle(indices, ThreadLocalRandom.current()); final int limit = Math.min(columnarInts.size(), 1000); diff --git a/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsAutoEncodingSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsAutoEncodingSerdeTest.java index 0fd5bbf6f89..4876a347fb2 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsAutoEncodingSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsAutoEncodingSerdeTest.java @@ -108,9 +108,7 @@ public class CompressedLongsAutoEncodingSerdeTest ); serializer.open(); - for (long value : values) { - serializer.add(value); - } + serializer.addAll(values, 0, values.length); Assert.assertEquals(values.length, serializer.size()); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); diff --git a/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsSerdeTest.java b/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsSerdeTest.java index ba35a03bff5..00a7b339ddb 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsSerdeTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/CompressedLongsSerdeTest.java @@ -186,9 +186,7 @@ public class CompressedLongsSerdeTest ); serializer.open(); - for (long value : values) { - serializer.add(value); - } + serializer.addAll(values, 0, values.length); Assert.assertEquals(values.length, serializer.size()); final ByteArrayOutputStream baos = new ByteArrayOutputStream(); From 739068469c071784a0e1951932a8ea2901f736b8 Mon Sep 17 00:00:00 2001 From: Adarsh Sanjeev Date: Tue, 6 Aug 2024 22:17:08 +0530 Subject: [PATCH 5/6] General Druid refactors (#16708) Some general refactors across Druid. Switch to DruidExceptions Add javadocs Fix a bug in IntArrayColumns Add a class for LongArrayColumns Remove wireTransferable since it would never be called Refactor DictionaryWriter to return the index written as a return value from write. --- .../apache/druid/error/DruidException.java | 18 +- .../apache/druid/error/NotYetImplemented.java | 67 ++++++ .../apache/druid/frame/read/FrameReader.java | 6 +- .../apache/druid/guice/JsonConfigurator.java | 2 +- .../MapOfColumnsRowsAndColumns.java | 7 + .../column/BinarySearchableAccessor.java | 4 + .../column/ConstantObjectColumn.java | 10 +- .../rowsandcols/column/DoubleArrayColumn.java | 20 +- .../rowsandcols/column/IntArrayColumn.java | 20 +- .../rowsandcols/column/LongArrayColumn.java | 204 ++++++++++++++++++ .../ColumnBasedFrameRowsAndColumns.java | 5 - .../concrete/ColumnHolderRACColumn.java | 2 +- .../druid/segment/AutoTypeColumnMerger.java | 6 +- .../StringUtf8DictionaryEncodedColumn.java | 25 +-- .../druid/segment/data/BitmapSerde.java | 13 ++ .../segment/data/CompressionFactory.java | 11 + .../segment/data/DeltaLongEncodingReader.java | 6 + .../druid/segment/data/DictionaryWriter.java | 31 ++- .../data/EncodedStringDictionaryWriter.java | 4 +- .../segment/data/FixedIndexedWriter.java | 38 ++-- .../segment/data/FrontCodedIndexedWriter.java | 11 +- .../data/FrontCodedIntArrayIndexedWriter.java | 35 ++- .../segment/data/GenericIndexedWriter.java | 5 +- .../segment/data/LongsLongEncodingReader.java | 6 + .../segment/data/LongsLongEncodingWriter.java | 1 - .../data/RoaringBitmapSerdeFactory.java | 2 +- .../segment/data/TableLongEncodingReader.java | 6 + .../segment/nested/DictionaryIdLookup.java | 2 +- .../druid/segment/serde/MetaSerdeHelper.java | 6 +- .../druid/segment/serde/cell/IOIterator.java | 6 + .../org/apache/druid/error/ExceptionTest.java | 48 +++++ .../column/LongArrayColumnTest.java | 82 +++++++ .../AppendableRowsAndColumnsTest.java | 5 +- .../segment/data/BitmapSerdeFactoryTest.java | 9 + .../segment/data/FrontCodedIndexedTest.java | 2 +- 35 files changed, 638 insertions(+), 87 deletions(-) create mode 100644 processing/src/main/java/org/apache/druid/error/NotYetImplemented.java create mode 100644 processing/src/main/java/org/apache/druid/query/rowsandcols/column/LongArrayColumn.java create mode 100644 processing/src/test/java/org/apache/druid/error/ExceptionTest.java create mode 100644 processing/src/test/java/org/apache/druid/query/rowsandcols/column/LongArrayColumnTest.java diff --git a/processing/src/main/java/org/apache/druid/error/DruidException.java b/processing/src/main/java/org/apache/druid/error/DruidException.java index f4cc3065c7f..b1056bbef52 100644 --- a/processing/src/main/java/org/apache/druid/error/DruidException.java +++ b/processing/src/main/java/org/apache/druid/error/DruidException.java @@ -24,6 +24,7 @@ import com.google.common.base.Preconditions; import org.apache.druid.java.util.common.StringUtils; import javax.annotation.concurrent.NotThreadSafe; +import java.util.Arrays; import java.util.LinkedHashMap; import java.util.Map; @@ -130,6 +131,8 @@ import java.util.Map; @NotThreadSafe public class DruidException extends RuntimeException { + public static final String CLASS_NAME_STR = DruidException.class.getName(); + /** * Starts building a "general" DruidException targeting the specified persona. * @@ -478,7 +481,7 @@ public class DruidException extends RuntimeException public DruidException build(Throwable cause, String formatMe, Object... vals) { - return new DruidException( + final DruidException retVal = new DruidException( cause, errorCode, targetPersona, @@ -486,6 +489,19 @@ public class DruidException extends RuntimeException StringUtils.nonStrictFormat(formatMe, vals), deserialized ); + + StackTraceElement[] stackTrace = retVal.getStackTrace(); + int firstNonDruidExceptionIndex = 0; + while ( + firstNonDruidExceptionIndex < stackTrace.length + && stackTrace[firstNonDruidExceptionIndex].getClassName().startsWith(CLASS_NAME_STR)) { + ++firstNonDruidExceptionIndex; + } + if (firstNonDruidExceptionIndex > 0) { + retVal.setStackTrace(Arrays.copyOfRange(stackTrace, firstNonDruidExceptionIndex, stackTrace.length)); + } + + return retVal; } } diff --git a/processing/src/main/java/org/apache/druid/error/NotYetImplemented.java b/processing/src/main/java/org/apache/druid/error/NotYetImplemented.java new file mode 100644 index 00000000000..b283034fab3 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/error/NotYetImplemented.java @@ -0,0 +1,67 @@ +/* + * 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.error; + +/** + * A failure class that is used to indicate that something is just not implemented yet. This is useful when a + * developer builds something and they intentionally do not implement a specific branch of code or type of object. + *

+ * The lack of implementation is not necessarily a statement that it SHOULDN'T be implemented, it's just an indication + * that it has not YET been implemented. When one of these exceptions is seen, it is usually an indication that it is + * now time to actually implement the path that was previously elided. + *

+ * Oftentimes, the code path wasn't implemented because the developer thought that it wasn't actually possible to + * see it executed. So, collecting and providing information about why the particular path got executed is often + * extremely helpful in understanding why it happened and accelerating the implementation of what the correct behavior + * should be. + */ +public class NotYetImplemented extends DruidException.Failure +{ + public static DruidException ex(Throwable t, String msg, Object... args) + { + return DruidException.fromFailure(new NotYetImplemented(t, msg, args)); + } + + private final Throwable t; + private final String msg; + private final Object[] args; + + public NotYetImplemented(Throwable t, String msg, Object[] args) + { + super("notYetImplemented"); + this.t = t; + this.msg = msg; + this.args = args; + } + + + @Override + protected DruidException makeException(DruidException.DruidExceptionBuilder bob) + { + bob = bob.forPersona(DruidException.Persona.DEVELOPER) + .ofCategory(DruidException.Category.DEFENSIVE); + + if (t == null) { + return bob.build(msg, args); + } else { + return bob.build(t, msg, args); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java index 8ddf99325d3..46a848fb6b1 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java +++ b/processing/src/main/java/org/apache/druid/frame/read/FrameReader.java @@ -20,6 +20,7 @@ package org.apache.druid.frame.read; import com.google.common.base.Preconditions; +import org.apache.druid.error.DruidException; import org.apache.druid.frame.Frame; import org.apache.druid.frame.field.FieldReader; import org.apache.druid.frame.field.FieldReaders; @@ -31,7 +32,6 @@ import org.apache.druid.frame.read.columnar.FrameColumnReaders; import org.apache.druid.frame.segment.row.FrameCursorFactory; import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.java.util.common.IAE; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.segment.CursorFactory; import org.apache.druid.segment.column.ColumnCapabilities; import org.apache.druid.segment.column.ColumnType; @@ -44,7 +44,7 @@ import java.util.Set; /** * Embeds the logic to read frames with a given {@link RowSignature}. - * + *

* Stateless and immutable. */ public class FrameReader @@ -146,7 +146,7 @@ public class FrameReader case ROW_BASED: return new FrameCursorFactory(frame, this, fieldReaders); default: - throw new ISE("Unrecognized frame type [%s]", frame.type()); + throw DruidException.defensive("Unrecognized frame type [%s]", frame.type()); } } diff --git a/processing/src/main/java/org/apache/druid/guice/JsonConfigurator.java b/processing/src/main/java/org/apache/druid/guice/JsonConfigurator.java index 1e4f18dc1cd..8a53dbffabf 100644 --- a/processing/src/main/java/org/apache/druid/guice/JsonConfigurator.java +++ b/processing/src/main/java/org/apache/druid/guice/JsonConfigurator.java @@ -236,7 +236,7 @@ public class JsonConfigurator // to configure ParametrizedUriEmitterConfig object. So skipping xxx=yyy key-value pair when configuring Emitter // doesn't make any difference. That is why we just log this situation, instead of throwing an exception. log.info( - "Skipping %s property: one of it's prefixes is also used as a property key. Prefix: %s", + "Skipping property [%s]: one of it's prefixes [%s] is also used as a property key.", originalProperty, propertyPrefix ); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java index 121e4863bcd..42972f9340d 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/MapOfColumnsRowsAndColumns.java @@ -25,6 +25,7 @@ import org.apache.druid.query.groupby.ResultRow; import org.apache.druid.query.rowsandcols.column.Column; import org.apache.druid.query.rowsandcols.column.DoubleArrayColumn; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.rowsandcols.column.LongArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.query.rowsandcols.semantic.AppendableRowsAndColumns; import org.apache.druid.segment.column.ColumnType; @@ -170,6 +171,12 @@ public class MapOfColumnsRowsAndColumns implements RowsAndColumns return add(name, new IntArrayColumn(vals)); } + @SuppressWarnings("unused") + public Builder add(String name, long[] vals) + { + return add(name, new LongArrayColumn(vals)); + } + public Builder add(String name, double[] vals) { return add(name, new DoubleArrayColumn(vals)); diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/BinarySearchableAccessor.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/BinarySearchableAccessor.java index 4eddcc77f1c..f7c339b2080 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/BinarySearchableAccessor.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/BinarySearchableAccessor.java @@ -22,6 +22,10 @@ package org.apache.druid.query.rowsandcols.column; import org.apache.druid.java.util.common.ISE; import org.apache.druid.query.rowsandcols.util.FindResult; +/** + * The implementations of this interface will not validate that things are sorted for the binary search, it assumes that + * they must be. As such, behavior are undefined if the column is not actually sorted. + */ public interface BinarySearchableAccessor extends ColumnAccessor { static BinarySearchableAccessor fromColumn(Column col) diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java index 28a7c3dd10d..01af9b07536 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/ConstantObjectColumn.java @@ -19,7 +19,7 @@ package org.apache.druid.query.rowsandcols.column; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.error.DruidException; import org.apache.druid.query.rowsandcols.util.FindResult; import org.apache.druid.segment.column.ColumnType; @@ -55,7 +55,13 @@ public class ConstantObjectColumn implements Column if (VectorCopier.class.equals(clazz)) { return (T) (VectorCopier) (into, intoStart) -> { if (Integer.MAX_VALUE - numRows < intoStart) { - throw new ISE("too many rows!!! intoStart[%,d], numRows[%,d] combine to exceed max_int", intoStart, numRows); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.CAPACITY_EXCEEDED) + .build( + "too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int", + intoStart, + numRows + ); } Arrays.fill(into, intoStart, intoStart + numRows, obj); }; diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java index 9c3b799d30e..18cb8ad9c5a 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/DoubleArrayColumn.java @@ -19,8 +19,8 @@ package org.apache.druid.query.rowsandcols.column; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Numbers; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.NotYetImplemented; import org.apache.druid.query.rowsandcols.util.FindResult; import org.apache.druid.segment.column.ColumnType; @@ -54,11 +54,13 @@ public class DoubleArrayColumn implements Column if (VectorCopier.class.equals(clazz)) { return (T) (VectorCopier) (into, intoStart) -> { if (Integer.MAX_VALUE - vals.length < intoStart) { - throw new ISE( - "too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int", - intoStart, - vals.length - ); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.CAPACITY_EXCEEDED) + .build( + "too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int", + intoStart, + vals.length + ); } for (int i = 0; i < vals.length; ++i) { into[intoStart + i] = vals[i]; @@ -183,13 +185,13 @@ public class DoubleArrayColumn implements Column @Override public FindResult findString(int startIndex, int endIndex, String val) { - return findDouble(startIndex, endIndex, Numbers.tryParseDouble(val, 0)); + throw NotYetImplemented.ex(null, "findString is not currently supported for DoubleArrayColumns"); } @Override public FindResult findComplex(int startIndex, int endIndex, Object val) { - return findDouble(startIndex, endIndex, Numbers.tryParseDouble(val, 0)); + throw NotYetImplemented.ex(null, "findComplex is not currently supported for DoubleArrayColumns"); } } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java index 673cebf0e2e..4a9d7c2c5b9 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/IntArrayColumn.java @@ -19,8 +19,8 @@ package org.apache.druid.query.rowsandcols.column; -import org.apache.druid.java.util.common.ISE; -import org.apache.druid.java.util.common.Numbers; +import org.apache.druid.error.DruidException; +import org.apache.druid.error.NotYetImplemented; import org.apache.druid.query.rowsandcols.util.FindResult; import org.apache.druid.segment.column.ColumnType; @@ -54,11 +54,13 @@ public class IntArrayColumn implements Column if (VectorCopier.class.equals(clazz)) { return (T) (VectorCopier) (into, intoStart) -> { if (Integer.MAX_VALUE - vals.length < intoStart) { - throw new ISE( - "too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int", - intoStart, - vals.length - ); + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.CAPACITY_EXCEEDED) + .build( + "too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int", + intoStart, + vals.length + ); } for (int i = 0; i < vals.length; ++i) { into[intoStart + i] = vals[i]; @@ -189,13 +191,13 @@ public class IntArrayColumn implements Column @Override public FindResult findString(int startIndex, int endIndex, String val) { - return findInt(startIndex, endIndex, (int) Numbers.tryParseLong(val, 0)); + throw NotYetImplemented.ex(null, "findString is not currently supported for IntArrayColumns"); } @Override public FindResult findComplex(int startIndex, int endIndex, Object val) { - return findDouble(startIndex, endIndex, (int) Numbers.tryParseLong(val, 0)); + throw NotYetImplemented.ex(null, "findComplex is not currently supported for IntArrayColumns"); } } } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/column/LongArrayColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/LongArrayColumn.java new file mode 100644 index 00000000000..bddf235eeb8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/column/LongArrayColumn.java @@ -0,0 +1,204 @@ +/* + * 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.rowsandcols.column; + +import org.apache.druid.error.DruidException; +import org.apache.druid.error.NotYetImplemented; +import org.apache.druid.query.rowsandcols.util.FindResult; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nonnull; +import javax.annotation.Nullable; +import java.util.Arrays; + +public class LongArrayColumn implements Column +{ + private final long[] vals; + + public LongArrayColumn( + long[] vals + ) + { + this.vals = vals; + } + + @Nonnull + @Override + public ColumnAccessor toAccessor() + { + return new MyColumnAccessor(); + } + + @Nullable + @SuppressWarnings("unchecked") + @Override + public T as(Class clazz) + { + if (VectorCopier.class.equals(clazz)) { + return (T) (VectorCopier) (into, intoStart) -> { + if (Integer.MAX_VALUE - vals.length < intoStart) { + throw DruidException.forPersona(DruidException.Persona.USER) + .ofCategory(DruidException.Category.CAPACITY_EXCEEDED) + .build( + "too many rows!!! intoStart[%,d], vals.length[%,d] combine to exceed max_int", + intoStart, + vals.length + ); + } + for (int i = 0; i < vals.length; ++i) { + into[intoStart + i] = vals[i]; + } + }; + } + if (ColumnValueSwapper.class.equals(clazz)) { + return (T) (ColumnValueSwapper) (lhs, rhs) -> { + long tmp = vals[lhs]; + vals[lhs] = vals[rhs]; + vals[rhs] = tmp; + }; + } + return null; + } + + private class MyColumnAccessor implements BinarySearchableAccessor + { + @Override + public ColumnType getType() + { + return ColumnType.LONG; + } + + @Override + public int numRows() + { + return vals.length; + } + + @Override + public boolean isNull(int rowNum) + { + return false; + } + + @Override + public Object getObject(int rowNum) + { + return vals[rowNum]; + } + + @Override + public double getDouble(int rowNum) + { + return vals[rowNum]; + } + + @Override + public float getFloat(int rowNum) + { + return vals[rowNum]; + } + + @Override + public long getLong(int rowNum) + { + return vals[rowNum]; + } + + @Override + public int getInt(int rowNum) + { + return (int) vals[rowNum]; + } + + @Override + public int compareRows(int lhsRowNum, int rhsRowNum) + { + return Long.compare(vals[lhsRowNum], vals[rhsRowNum]); + } + + + @Override + public FindResult findNull(int startIndex, int endIndex) + { + return FindResult.notFound(endIndex); + } + + @Override + public FindResult findDouble(int startIndex, int endIndex, double val) + { + return findLong(startIndex, endIndex, (long) val); + } + + @Override + public FindResult findFloat(int startIndex, int endIndex, float val) + { + return findLong(startIndex, endIndex, (long) val); + } + + @Override + public FindResult findLong(int startIndex, int endIndex, long val) + { + if (vals[startIndex] == val) { + int end = startIndex + 1; + + while (end < endIndex && vals[end] == val) { + ++end; + } + return FindResult.found(startIndex, end); + } + + int i = Arrays.binarySearch(vals, startIndex, endIndex, val); + if (i > 0) { + int foundStart = i; + int foundEnd = i + 1; + + while (foundStart - 1 >= startIndex && vals[foundStart - 1] == val) { + --foundStart; + } + + while (foundEnd < endIndex && vals[foundEnd] == val) { + ++foundEnd; + } + + return FindResult.found(foundStart, foundEnd); + } else { + return FindResult.notFound(-(i + 1)); + } + } + + @SuppressWarnings("unused") + public FindResult findInt(int startIndex, int endIndex, int val) + { + return findLong(startIndex, endIndex, val); + } + + @Override + public FindResult findString(int startIndex, int endIndex, String val) + { + throw NotYetImplemented.ex(null, "findString is not currently supported for LongArrayColumns"); + } + + @Override + public FindResult findComplex(int startIndex, int endIndex, Object val) + { + throw NotYetImplemented.ex(null, "findComplex is not currently supported for LongArrayColumns"); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java index ada3da164ea..71c2541b387 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnBasedFrameRowsAndColumns.java @@ -28,7 +28,6 @@ import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.Intervals; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.Column; -import org.apache.druid.query.rowsandcols.semantic.WireTransferable; import org.apache.druid.segment.CloseableShapeshifter; import org.apache.druid.segment.StorageAdapter; import org.apache.druid.segment.column.ColumnType; @@ -80,7 +79,6 @@ public class ColumnBasedFrameRowsAndColumns implements RowsAndColumns, AutoClose } } return colCache.get(name); - } @SuppressWarnings("unchecked") @@ -91,9 +89,6 @@ public class ColumnBasedFrameRowsAndColumns implements RowsAndColumns, AutoClose if (StorageAdapter.class.equals(clazz)) { return (T) new FrameStorageAdapter(frame, FrameReader.create(signature), Intervals.ETERNITY); } - if (WireTransferable.class.equals(clazz)) { - return (T) this; - } return null; } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java index ed4f8ead52e..d68f8872bf4 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/ColumnHolderRACColumn.java @@ -91,7 +91,7 @@ public class ColumnHolderRACColumn implements Column, Closeable public boolean isNull(int rowNum) { offset.set(rowNum); - return valueSelector.isNull(); + return valueSelector.getObject() == null; } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java index 5d1198f5460..801eaf112a5 100644 --- a/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java +++ b/processing/src/main/java/org/apache/druid/segment/AutoTypeColumnMerger.java @@ -22,7 +22,7 @@ package org.apache.druid.segment; import com.google.common.base.Preconditions; import com.google.common.collect.Iterators; import com.google.common.collect.PeekingIterator; -import org.apache.druid.java.util.common.ISE; +import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.java.util.common.logger.Logger; import org.apache.druid.segment.column.ColumnDescriptor; @@ -212,7 +212,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 ); break; default: - throw new ISE( + throw DruidException.defensive( "How did we get here? Column [%s] with type [%s] does not have specialized serializer", name, logicalType @@ -349,7 +349,7 @@ public class AutoTypeColumnMerger implements DimensionMergerV9 @Override public ColumnDescriptor makeColumnDescriptor() { - ColumnDescriptor.Builder descriptorBuilder = new ColumnDescriptor.Builder(); + ColumnDescriptor.Builder descriptorBuilder = ColumnDescriptor.builder(); final NestedCommonFormatColumnPartSerde partSerde = NestedCommonFormatColumnPartSerde.serializerBuilder() diff --git a/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java b/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java index fe8ade4a9ed..c3ebde1854c 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java +++ b/processing/src/main/java/org/apache/druid/segment/column/StringUtf8DictionaryEncodedColumn.java @@ -528,11 +528,11 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum /** * Base type for a {@link SingleValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING} * built around a {@link ColumnarInts}. Dictionary not included - BYO dictionary lookup methods. - * + *

* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}. */ public abstract static class StringSingleValueDimensionVectorSelector - implements SingleValueDimensionVectorSelector, IdLookup + implements SingleValueDimensionVectorSelector, IdLookup { private final ColumnarInts column; private final ReadableVectorOffset offset; @@ -540,8 +540,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum private int id = ReadableVectorInspector.NULL_ID; public StringSingleValueDimensionVectorSelector( - ColumnarInts column, - ReadableVectorOffset offset + ColumnarInts column, + ReadableVectorOffset offset ) { this.column = column; @@ -601,11 +601,11 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum /** * Base type for a {@link MultiValueDimensionVectorSelector} for a dictionary encoded {@link ColumnType#STRING} * built around a {@link ColumnarMultiInts}. Dictionary not included - BYO dictionary lookup methods. - * + *

* Assumes that all implementations return true for {@link #supportsLookupNameUtf8()}. */ public abstract static class StringMultiValueDimensionVectorSelector - implements MultiValueDimensionVectorSelector, IdLookup + implements MultiValueDimensionVectorSelector, IdLookup { private final ColumnarMultiInts multiValueColumn; private final ReadableVectorOffset offset; @@ -614,8 +614,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum private int id = ReadableVectorInspector.NULL_ID; public StringMultiValueDimensionVectorSelector( - ColumnarMultiInts multiValueColumn, - ReadableVectorOffset offset + ColumnarMultiInts multiValueColumn, + ReadableVectorOffset offset ) { this.multiValueColumn = multiValueColumn; @@ -670,6 +670,7 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum { return this; } + @Override public int getCurrentVectorSize() { @@ -697,8 +698,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum private int id = ReadableVectorInspector.NULL_ID; public StringVectorObjectSelector( - ColumnarInts column, - ReadableVectorOffset offset + ColumnarInts column, + ReadableVectorOffset offset ) { this.column = column; @@ -757,8 +758,8 @@ public class StringUtf8DictionaryEncodedColumn implements DictionaryEncodedColum private int id = ReadableVectorInspector.NULL_ID; public MultiValueStringVectorObjectSelector( - ColumnarMultiInts multiValueColumn, - ReadableVectorOffset offset + ColumnarMultiInts multiValueColumn, + ReadableVectorOffset offset ) { this.multiValueColumn = multiValueColumn; diff --git a/processing/src/main/java/org/apache/druid/segment/data/BitmapSerde.java b/processing/src/main/java/org/apache/druid/segment/data/BitmapSerde.java index aa80186e10d..b25862b1749 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/BitmapSerde.java +++ b/processing/src/main/java/org/apache/druid/segment/data/BitmapSerde.java @@ -20,7 +20,10 @@ package org.apache.druid.segment.data; import com.fasterxml.jackson.annotation.JsonTypeName; +import org.apache.druid.collections.bitmap.BitmapFactory; +import org.apache.druid.collections.bitmap.ConciseBitmapFactory; import org.apache.druid.collections.bitmap.RoaringBitmapFactory; +import org.apache.druid.error.DruidException; public class BitmapSerde { @@ -48,4 +51,14 @@ public class BitmapSerde { return new LegacyBitmapSerdeFactory(); } + + public static BitmapSerdeFactory forBitmapFactory(BitmapFactory factory) + { + if (factory instanceof RoaringBitmapFactory) { + return new DefaultBitmapSerdeFactory(); + } else if (factory instanceof ConciseBitmapFactory) { + return new ConciseBitmapSerdeFactory(); + } + throw DruidException.defensive("Unknown type of bitmapFactory [%s]", factory.getClass()); + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/CompressionFactory.java b/processing/src/main/java/org/apache/druid/segment/data/CompressionFactory.java index dde6a440d9e..91ec70b7f17 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/CompressionFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/data/CompressionFactory.java @@ -233,6 +233,14 @@ public class CompressionFactory void write(long value) throws IOException; + @SuppressWarnings("unused") + default void write(long[] values, int offset, int length) throws IOException + { + for (int i = offset; i < length; ++i) { + write(values[i]); + } + } + /** * Flush the unwritten content to the current output. */ @@ -294,6 +302,9 @@ public class CompressionFactory * various duplicates. */ LongEncodingReader duplicate(); + + @SuppressWarnings("unused") + LongEncodingStrategy getStrategy(); } public static Supplier getLongSupplier( diff --git a/processing/src/main/java/org/apache/druid/segment/data/DeltaLongEncodingReader.java b/processing/src/main/java/org/apache/druid/segment/data/DeltaLongEncodingReader.java index 435aa2ddfd1..b7feb3b1dd3 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/DeltaLongEncodingReader.java +++ b/processing/src/main/java/org/apache/druid/segment/data/DeltaLongEncodingReader.java @@ -82,4 +82,10 @@ public class DeltaLongEncodingReader implements CompressionFactory.LongEncodingR { return new DeltaLongEncodingReader(buffer.duplicate(), base, bitsPerValue); } + + @Override + public CompressionFactory.LongEncodingStrategy getStrategy() + { + return CompressionFactory.LongEncodingStrategy.AUTO; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java b/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java index 170f6975f28..5901e2e1320 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/DictionaryWriter.java @@ -28,12 +28,41 @@ public interface DictionaryWriter extends Serializer { boolean isSorted(); + /** + * Prepares the writer for writing + * + * @throws IOException if there is a problem with IO + */ void open() throws IOException; - void write(@Nullable T objectToWrite) throws IOException; + /** + * Writes an object to the dictionary. + *

+ * Returns the index of the value that was just written. This is defined as the `int` value that can be passed + * into {@link #get} such that it will return the same value back. + * + * @param objectToWrite object to be written to the dictionary + * @return index of the value that was just written + * @throws IOException if there is a problem with IO + */ + int write(@Nullable T objectToWrite) throws IOException; + /** + * Returns an object that has already been written via the {@link #write} method. + * + * @param dictId index of the object to return + * @return the object identified by the given index + * @throws IOException if there is a problem with IO + */ @Nullable T get(int dictId) throws IOException; + /** + * Returns the number of items that have been written so far in this dictionary. Any number lower than this + * cardinality can be passed into {@link #get} and a value will be returned. If a value greater than or equal to + * the cardinality is passed into {@link #get} all sorts of things could happen, but likely none of them are good. + * + * @return the number of items that have been written so far + */ int getCardinality(); } diff --git a/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java b/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java index 371a73bebd7..799ed3766f2 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/EncodedStringDictionaryWriter.java @@ -58,9 +58,9 @@ public class EncodedStringDictionaryWriter implements DictionaryWriter } @Override - public void write(@Nullable String objectToWrite) throws IOException + public int write(@Nullable String objectToWrite) throws IOException { - delegate.write(StringUtils.toUtf8Nullable(NullHandling.emptyToNullIfNeeded(objectToWrite))); + return delegate.write(StringUtils.toUtf8Nullable(NullHandling.emptyToNullIfNeeded(objectToWrite))); } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java index b1b473b3419..42ca16b78f4 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FixedIndexedWriter.java @@ -20,8 +20,8 @@ package org.apache.druid.segment.data; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.io.Channels; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.segment.column.TypeStrategy; import org.apache.druid.segment.writeout.SegmentWriteOutMedium; @@ -46,14 +46,16 @@ public class FixedIndexedWriter implements DictionaryWriter private final Comparator comparator; private final ByteBuffer scratch; private final ByteBuffer readBuffer; - private int numWritten; + private final boolean isSorted; + private final int width; + + private int cardinality = 0; + @Nullable private WriteOutBytes valuesOut = null; private boolean hasNulls = false; - private boolean isSorted; @Nullable private T prevObject = null; - private final int width; public FixedIndexedWriter( SegmentWriteOutMedium segmentWriteOutMedium, @@ -87,7 +89,7 @@ public class FixedIndexedWriter implements DictionaryWriter @Override public int getCardinality() { - return hasNulls ? numWritten + 1 : numWritten; + return cardinality; } @Override @@ -97,28 +99,31 @@ public class FixedIndexedWriter implements DictionaryWriter } @Override - public void write(@Nullable T objectToWrite) throws IOException + public int write(@Nullable T objectToWrite) throws IOException { if (prevObject != null && isSorted && comparator.compare(prevObject, objectToWrite) >= 0) { - throw new ISE( + throw DruidException.defensive( "Values must be sorted and unique. Element [%s] with value [%s] is before or equivalent to [%s]", - numWritten, + cardinality, objectToWrite, prevObject ); } if (objectToWrite == null) { + if (cardinality != 0) { + throw DruidException.defensive("Null must come first, got it at cardinality[%,d]!=0", cardinality); + } hasNulls = true; - return; + return cardinality++; } scratch.clear(); typeStrategy.write(scratch, objectToWrite, width); scratch.flip(); Channels.writeFully(valuesOut, scratch); - numWritten++; prevObject = objectToWrite; + return cardinality++; } @Override @@ -141,7 +146,7 @@ public class FixedIndexedWriter implements DictionaryWriter scratch.flip(); Channels.writeFully(channel, scratch); scratch.clear(); - scratch.putInt(numWritten); + scratch.putInt(hasNulls ? cardinality - 1 : cardinality); // we don't actually write the null entry, so subtract 1 scratch.flip(); Channels.writeFully(channel, scratch); valuesOut.writeTo(channel); @@ -166,7 +171,7 @@ public class FixedIndexedWriter implements DictionaryWriter public Iterator getIterator() { final ByteBuffer iteratorBuffer = ByteBuffer.allocate(width * PAGE_SIZE).order(readBuffer.order()); - final int totalCount = hasNulls ? 1 + numWritten : numWritten; + final int totalCount = cardinality; final int startPos = hasNulls ? 1 : 0; return new Iterator() @@ -197,13 +202,8 @@ public class FixedIndexedWriter implements DictionaryWriter { iteratorBuffer.clear(); try { - if (numWritten - (pos - startPos) < PAGE_SIZE) { - int size = (numWritten - (pos - startPos)) * width; - iteratorBuffer.limit(size); - valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer); - } else { - valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer); - } + iteratorBuffer.limit(Math.min(PAGE_SIZE, (cardinality - pos) * width)); + valuesOut.readFully((long) (pos - startPos) * width, iteratorBuffer); iteratorBuffer.clear(); } catch (IOException e) { diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java index c24d2e55d71..707e3894793 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIndexedWriter.java @@ -21,6 +21,7 @@ package org.apache.druid.segment.data; import com.google.common.primitives.Ints; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.io.Channels; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -102,7 +103,7 @@ public class FrontCodedIndexedWriter implements DictionaryWriter } @Override - public void write(@Nullable byte[] value) throws IOException + public int write(@Nullable byte[] value) throws IOException { if (prevObject != null && compareNullableUtf8UsingJavaStringOrdering(prevObject, value) >= 0) { throw new ISE( @@ -114,8 +115,11 @@ public class FrontCodedIndexedWriter implements DictionaryWriter } if (value == null) { + if (numWritten != 0) { + throw DruidException.defensive("Null must come first, got it at cardinality[%,d]!=0", numWritten); + } hasNulls = true; - return; + return 0; } // if the bucket buffer is full, write the bucket @@ -143,8 +147,9 @@ public class FrontCodedIndexedWriter implements DictionaryWriter bucketBuffer[numWritten % bucketSize] = value; - ++numWritten; + int retVal = numWritten++; prevObject = value; + return retVal + (hasNulls ? 1 : 0); } diff --git a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java index 8116882191b..50e350f3d64 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/FrontCodedIntArrayIndexedWriter.java @@ -21,6 +21,7 @@ package org.apache.druid.segment.data; import com.google.common.primitives.Ints; import org.apache.druid.common.config.NullHandling; +import org.apache.druid.error.DruidException; import org.apache.druid.io.Channels; import org.apache.druid.java.util.common.IAE; import org.apache.druid.java.util.common.ISE; @@ -81,6 +82,10 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter private boolean isClosed = false; private boolean hasNulls = false; + private int readCachedBucket = -1; + @Nullable + private ByteBuffer readBufferCache = null; + public FrontCodedIntArrayIndexedWriter( SegmentWriteOutMedium segmentWriteOutMedium, ByteOrder byteOrder, @@ -107,7 +112,7 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter } @Override - public void write(@Nullable int[] value) throws IOException + public int write(@Nullable int[] value) throws IOException { if (prevObject != null && ARRAY_COMPARATOR.compare(prevObject, value) >= 0) { @@ -120,8 +125,11 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter } if (value == null) { + if (numWritten != 0) { + throw DruidException.defensive("Null must come first, got it at numWritten[%,d]!=0", numWritten); + } hasNulls = true; - return; + return 0; } // if the bucket buffer is full, write the bucket @@ -147,8 +155,9 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter bucketBuffer[numWritten % bucketSize] = value; - ++numWritten; + int retVal = numWritten++; prevObject = value; + return retVal + (hasNulls ? 1 : 0); } @@ -206,6 +215,11 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter return bucketBuffer[relativeIndex]; } else { final int bucket = adjustedIndex >> div; + if (readCachedBucket == bucket) { + readBufferCache.position(0); + return getFromBucket(readBufferCache, relativeIndex); + } + long startOffset; if (bucket == 0) { startOffset = 0; @@ -217,10 +231,17 @@ public class FrontCodedIntArrayIndexedWriter implements DictionaryWriter if (currentBucketSize == 0) { return null; } - final ByteBuffer bucketBuffer = ByteBuffer.allocate(currentBucketSize).order(byteOrder); - valuesOut.readFully(startOffset, bucketBuffer); - bucketBuffer.clear(); - return getFromBucket(bucketBuffer, relativeIndex); + if (readBufferCache == null || readBufferCache.capacity() < currentBucketSize) { + readBufferCache = ByteBuffer.allocate(currentBucketSize).order(byteOrder); + } + readBufferCache.clear(); + readBufferCache.limit(currentBucketSize); + valuesOut.readFully(startOffset, readBufferCache); + + readCachedBucket = bucket; + + readBufferCache.position(0); + return getFromBucket(readBufferCache, relativeIndex); } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java index 8b38125322b..a87a61843fa 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/GenericIndexedWriter.java @@ -242,7 +242,7 @@ public class GenericIndexedWriter implements DictionaryWriter } @Override - public void write(@Nullable T objectToWrite) throws IOException + public int write(@Nullable T objectToWrite) throws IOException { if (objectsSorted && prevObject != null && strategy.compare(prevObject, objectToWrite) >= 0) { objectsSorted = false; @@ -263,7 +263,7 @@ public class GenericIndexedWriter implements DictionaryWriter // Increment number of values written. Important to do this after the check above, since numWritten is // accessed during "initializeHeaderOutLong" to determine the length of the header. - ++numWritten; + int retVal = numWritten++; if (!requireMultipleFiles) { headerOut.writeInt(checkedCastNonnegativeLongToInt(valuesOut.size())); @@ -280,6 +280,7 @@ public class GenericIndexedWriter implements DictionaryWriter if (objectsSorted) { prevObject = objectToWrite; } + return retVal; } @Nullable diff --git a/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingReader.java b/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingReader.java index 2ed0459121a..7df866f22c7 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingReader.java +++ b/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingReader.java @@ -71,4 +71,10 @@ public class LongsLongEncodingReader implements CompressionFactory.LongEncodingR { return new LongsLongEncodingReader(buffer.getByteBuffer(), buffer.getTypeByteOrder()); } + + @Override + public CompressionFactory.LongEncodingStrategy getStrategy() + { + return CompressionFactory.LongEncodingStrategy.LONGS; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingWriter.java b/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingWriter.java index 2aeb194d9a8..728a50aa2fc 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingWriter.java +++ b/processing/src/main/java/org/apache/druid/segment/data/LongsLongEncodingWriter.java @@ -22,7 +22,6 @@ package org.apache.druid.segment.data; import org.apache.druid.segment.writeout.WriteOutBytes; import javax.annotation.Nullable; - import java.io.IOException; import java.io.OutputStream; import java.nio.ByteBuffer; diff --git a/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java b/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java index ea6bb9bd994..f7cf9bdbb00 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/data/RoaringBitmapSerdeFactory.java @@ -84,7 +84,7 @@ public class RoaringBitmapSerdeFactory implements BitmapSerdeFactory @Override public byte[] toBytes(@Nullable ImmutableBitmap val) { - if (val == null || val.size() == 0) { + if (val == null || val.isEmpty()) { return new byte[]{}; } return val.toBytes(); diff --git a/processing/src/main/java/org/apache/druid/segment/data/TableLongEncodingReader.java b/processing/src/main/java/org/apache/druid/segment/data/TableLongEncodingReader.java index 6a5e17b1080..7e9b1fdc927 100644 --- a/processing/src/main/java/org/apache/druid/segment/data/TableLongEncodingReader.java +++ b/processing/src/main/java/org/apache/druid/segment/data/TableLongEncodingReader.java @@ -88,4 +88,10 @@ public class TableLongEncodingReader implements CompressionFactory.LongEncodingR { return new TableLongEncodingReader(buffer.duplicate(), table, bitsPerValue); } + + @Override + public CompressionFactory.LongEncodingStrategy getStrategy() + { + return CompressionFactory.LongEncodingStrategy.AUTO; + } } diff --git a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java index 6827497f7a6..d5e8a4ab6f4 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/DictionaryIdLookup.java @@ -51,7 +51,7 @@ import java.util.EnumSet; /** * Value to dictionary id lookup, backed with memory mapped dictionaries populated lazily by the supplied - * @link DictionaryWriter}. + * {@link DictionaryWriter}. */ public final class DictionaryIdLookup implements Closeable { diff --git a/processing/src/main/java/org/apache/druid/segment/serde/MetaSerdeHelper.java b/processing/src/main/java/org/apache/druid/segment/serde/MetaSerdeHelper.java index 113821cee15..08e97e3e501 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/MetaSerdeHelper.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/MetaSerdeHelper.java @@ -112,7 +112,11 @@ public final class MetaSerdeHelper public int size(T x) { - return fieldWriters.stream().mapToInt(w -> w.size(x)).sum(); + int retVal = 0; + for (FieldWriter fieldWriter : fieldWriters) { + retVal += fieldWriter.size(x); + } + return retVal; } public interface FieldWriter diff --git a/processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java b/processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java index 887f1fb65ac..3931601dd4f 100644 --- a/processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java +++ b/processing/src/main/java/org/apache/druid/segment/serde/cell/IOIterator.java @@ -22,6 +22,12 @@ package org.apache.druid.segment.serde.cell; import java.io.Closeable; import java.io.IOException; +/** + * An Iterator-like interface that is intentionally not extending Iterator. This is because it is Closeable + * and we never want to lose track of the fact that the object needs to be closed. + * + * @param + */ public interface IOIterator extends Closeable { boolean hasNext() throws IOException; diff --git a/processing/src/test/java/org/apache/druid/error/ExceptionTest.java b/processing/src/test/java/org/apache/druid/error/ExceptionTest.java new file mode 100644 index 00000000000..bf587e4cbd9 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/error/ExceptionTest.java @@ -0,0 +1,48 @@ +/* + * 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.error; + +import org.junit.Assert; +import org.junit.Test; + +import java.util.Map; + +public class ExceptionTest +{ + @Test + public void testNoCause() + { + DruidException exception = DruidException.defensive().build("defensive"); + StackTraceElement[] stackTrace = exception.getStackTrace(); + for (StackTraceElement stackTraceElement : stackTrace) { + Assert.assertFalse(stackTraceElement.getClassName().startsWith(DruidException.CLASS_NAME_STR)); + } + } + + @Test + public void testNoStacktrace() + { + ErrorResponse errorResponse = new ErrorResponse(Forbidden.exception()); + final Map asMap = errorResponse.getAsMap(); + DruidException exception = ErrorResponse.fromMap(asMap).getUnderlyingException(); + Assert.assertTrue(exception.getCause() instanceof DruidException); + Assert.assertEquals(0, exception.getCause().getStackTrace().length); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/column/LongArrayColumnTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/column/LongArrayColumnTest.java new file mode 100644 index 00000000000..38a53a17a33 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/column/LongArrayColumnTest.java @@ -0,0 +1,82 @@ +/* + * 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.rowsandcols.column; + +import org.apache.druid.query.rowsandcols.util.FindResult; +import org.junit.Assert; +import org.junit.Test; + +public class LongArrayColumnTest +{ + @Test + public void testLongArrayColumnWithLongValues() + { + Column column = new LongArrayColumn(new long[]{0, 1, 2, 3, 4, 5, 6, 7, 8, 9}); + ColumnAccessor accessor = column.toAccessor(); + + for (int i = 0; i < 10; i++) { + Assert.assertFalse(accessor.isNull(i)); + Assert.assertEquals(i, accessor.getLong(i)); + Assert.assertEquals((long) i, accessor.getObject(i)); + Assert.assertEquals(i, accessor.getDouble(i), 0); + Assert.assertEquals(i, accessor.getInt(i)); + } + } + + @Test + public void testFindLong() + { + Column column = new LongArrayColumn(new long[] {1, 1, 1, 3, 5, 5, 6, 7, 8, 9}); + BinarySearchableAccessor accessor = (BinarySearchableAccessor) column.toAccessor(); + + FindResult findResult = accessor.findLong(0, accessor.numRows(), 1); + Assert.assertTrue(findResult.wasFound()); + Assert.assertEquals(0, findResult.getStartRow()); + Assert.assertEquals(3, findResult.getEndRow()); + + findResult = accessor.findLong(0, accessor.numRows(), 6); + Assert.assertTrue(findResult.wasFound()); + Assert.assertEquals(6, findResult.getStartRow()); + Assert.assertEquals(7, findResult.getEndRow()); + + Assert.assertFalse(accessor.findLong(0, accessor.numRows(), 2).wasFound()); + Assert.assertFalse(accessor.findLong(0, 3, 9).wasFound()); + } + + @Test + public void testOtherTypeFinds() + { + Column column = new LongArrayColumn(new long[] {0, 1, 2, 3, 4, 5, Long.MAX_VALUE}); + BinarySearchableAccessor accessor = (BinarySearchableAccessor) column.toAccessor(); + + FindResult findResult = accessor.findNull(0, accessor.numRows()); + Assert.assertFalse(findResult.wasFound()); // Always false for long array columns + + findResult = accessor.findDouble(0, accessor.numRows(), 3.0); + Assert.assertTrue(findResult.wasFound()); + Assert.assertEquals(3, findResult.getStartRow()); + Assert.assertEquals(4, findResult.getEndRow()); + + findResult = accessor.findFloat(0, accessor.numRows(), 1.0f); + Assert.assertTrue(findResult.wasFound()); + Assert.assertEquals(1, findResult.getStartRow()); + Assert.assertEquals(2, findResult.getEndRow()); + } +} diff --git a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java index e5d6eb1faa7..a305e98ff9e 100644 --- a/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java +++ b/processing/src/test/java/org/apache/druid/query/rowsandcols/semantic/AppendableRowsAndColumnsTest.java @@ -24,6 +24,7 @@ import org.apache.druid.query.operator.window.RowsAndColumnsHelper; import org.apache.druid.query.rowsandcols.MapOfColumnsRowsAndColumns; import org.apache.druid.query.rowsandcols.RowsAndColumns; import org.apache.druid.query.rowsandcols.column.IntArrayColumn; +import org.apache.druid.query.rowsandcols.column.LongArrayColumn; import org.apache.druid.query.rowsandcols.column.ObjectArrayColumn; import org.apache.druid.segment.column.ColumnType; import org.junit.Assert; @@ -48,7 +49,7 @@ public class AppendableRowsAndColumnsTest extends SemanticTestBase RowsAndColumns rac = make(MapOfColumnsRowsAndColumns.fromMap( ImmutableMap.of( "colA", new IntArrayColumn(new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}), - "colB", new IntArrayColumn(new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) + "colB", new LongArrayColumn(new long[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) ) )); @@ -58,7 +59,7 @@ public class AppendableRowsAndColumnsTest extends SemanticTestBase new RowsAndColumnsHelper() .expectColumn("colA", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}) - .expectColumn("colB", new int[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) + .expectColumn("colB", new long[]{4, -4, 3, -3, 4, 82, -90, 4, 0, 0}) .expectColumn("newCol", new int[]{1, 2, 3, 4, 5, 6, 7, 8, 9, 10}) .allColumnsRegistered() .validate(appender); diff --git a/processing/src/test/java/org/apache/druid/segment/data/BitmapSerdeFactoryTest.java b/processing/src/test/java/org/apache/druid/segment/data/BitmapSerdeFactoryTest.java index f799d35059b..34b93f10165 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/BitmapSerdeFactoryTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/BitmapSerdeFactoryTest.java @@ -20,6 +20,8 @@ package org.apache.druid.segment.data; import com.fasterxml.jackson.databind.ObjectMapper; +import org.apache.druid.collections.bitmap.ConciseBitmapFactory; +import org.apache.druid.collections.bitmap.RoaringBitmapFactory; import org.apache.druid.jackson.DefaultObjectMapper; import org.junit.Assert; import org.junit.Test; @@ -46,4 +48,11 @@ public class BitmapSerdeFactoryTest Assert.assertTrue(mapper.readValue("{\"type\":\"concise\"}", BitmapSerdeFactory.class) instanceof ConciseBitmapSerdeFactory); Assert.assertTrue(mapper.readValue("{\"type\":\"BitmapSerde$SomeRandomClass\"}", BitmapSerdeFactory.class) instanceof RoaringBitmapSerdeFactory); } + + @Test + public void testForBitmapFactory() + { + Assert.assertTrue(BitmapSerde.forBitmapFactory(new RoaringBitmapFactory()) instanceof BitmapSerde.DefaultBitmapSerdeFactory); + Assert.assertTrue(BitmapSerde.forBitmapFactory(new ConciseBitmapFactory()) instanceof ConciseBitmapSerdeFactory); + } } diff --git a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java index 8b055188e63..c1312731b91 100644 --- a/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java +++ b/processing/src/test/java/org/apache/druid/segment/data/FrontCodedIndexedTest.java @@ -436,7 +436,7 @@ public class FrontCodedIndexedTest extends InitializedNullHandlingTest while (sortedStrings.hasNext()) { final String next = sortedStrings.next(); final byte[] nextBytes = StringUtils.toUtf8Nullable(next); - writer.write(nextBytes); + Assert.assertEquals(index, writer.write(nextBytes)); if (nextBytes == null) { Assert.assertNull(writer.get(index)); } else { From ebea34a814b397ffae79ab8a630e9fa2bff9cc62 Mon Sep 17 00:00:00 2001 From: Edgar Melendrez Date: Tue, 6 Aug 2024 11:32:26 -0700 Subject: [PATCH 6/6] [Docs] Batch06: starting string functions (#16838) * batch06, starting string functions * addind space after Syntax * quick change * correcting spelling * Update docs/querying/sql-functions.md * Update sql-functions.md * applying suggestions * Update docs/querying/sql-functions.md * Update docs/querying/sql-functions.md --------- Co-authored-by: Benedict Jin Co-authored-by: Charles Smith --- docs/querying/sql-functions.md | 249 +++++++++++++++++++++++++++------ 1 file changed, 209 insertions(+), 40 deletions(-) diff --git a/docs/querying/sql-functions.md b/docs/querying/sql-functions.md index 2ec056290b6..4325537a627 100644 --- a/docs/querying/sql-functions.md +++ b/docs/querying/sql-functions.md @@ -655,20 +655,23 @@ Returns the following: ## CHAR_LENGTH -`CHAR_LENGTH(expr)` - -**Function type:** [Scalar, string](sql-scalar.md#string-functions) - Alias for [`LENGTH`](#length). +* **Syntax:** `CHAR_LENGTH(expr)` +* **Function type:** Scalar, string + +[Learn more](sql-scalar.md#string-functions) + ## CHARACTER_LENGTH -`CHARACTER_LENGTH(expr)` - -**Function type:** [Scalar, string](sql-scalar.md#string-functions) - Alias for [`LENGTH`](#length). +* **Syntax:** `CHARACTER_LENGTH(expr)` +* **Function type:** Scalar, string + +[Learn more](sql-scalar.md#string-functions) + + ## COALESCE `COALESCE(expr, expr, ...)` @@ -679,19 +682,64 @@ Returns the first non-null value. ## CONCAT -`CONCAT(expr, expr...)` - -**Function type:** [Scalar, string](sql-scalar.md#string-functions) - Concatenates a list of expressions. +* **Syntax:** `CONCAT(expr[, expr,...])` +* **Function type:** Scalar, string + +

Example + +The following example concatenates the `OriginCityName` column from `flight-carriers`, the string ` to `, and the `DestCityName` column from `flight-carriers`. + +```sql +SELECT + "OriginCityName" AS "origin_city", + "DestCityName" AS "destination_city", + CONCAT("OriginCityName", ' to ', "DestCityName") AS "concatenate_flight_details" +FROM "flight-carriers" +LIMIT 1 +``` + +Returns the following: + +| `origin_city` | `destination_city` | `concatenate_flight_details` | +| -- | -- | -- | +| `San Juan, PR` | `Washington, DC` | `San Juan, PR to Washington, DC` | + +
+ +[Learn more](sql-scalar.md#string-functions) + ## CONTAINS_STRING -`CONTAINS_STRING(, )` +Returns `true` if `str` is a substring of `expr`, case-sensitive. Otherwise returns `false`. -**Function type:** [Scalar, string](sql-scalar.md#string-functions) +* **Syntax:** `CONTAINS_STRING(expr, str)` +* **Function type:** Scalar, string -Finds whether a string is in a given expression, case-sensitive. +
Example + +The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the substring `San`. + +```sql +SELECT + "OriginCityName" AS "origin_city", + CONTAINS_STRING("OriginCityName", 'San') AS "contains_string" +FROM "flight-carriers" +LIMIT 2 +``` + +Returns the following: + +| `origin_city` | `contains_string` | +| -- | -- | +| `San Juan, PR` | `true` | +| `Boston, MA` | `false` | + +
+ + +[Learn more](sql-scalar.md#string-functions) ## COS @@ -791,13 +839,31 @@ Decodes a Base64-encoded string into a complex data type, where `dataType` is th ## DECODE_BASE64_UTF8 -`DECODE_BASE64_UTF8(expr)` - -**Function type:** [Scalar, string](sql-scalar.md#string-functions) - - Decodes a Base64-encoded string into a UTF-8 encoded string. +* **Syntax:** `DECODE_BASE64_UTF8(expr)` +* **Function type:** Scalar, string + +
Example + +The following example converts the base64 encoded string `SGVsbG8gV29ybGQhCg==` into an UTF-8 encoded string. + +```sql +SELECT + 'SGVsbG8gV29ybGQhCg==' AS "base64_encoding", + DECODE_BASE64_UTF8('SGVsbG8gV29ybGQhCg==') AS "convert_to_UTF8_encoding" +``` + +Returns the following: + +| `base64_encoding` | `convert_to_UTF8_encoding` | +| -- | -- | +| `SGVsbG8gV29ybGQhCg==` | `Hello World!` | + +
+ +[Learn more](sql-scalar.md#string-functions) + ## DEGREES Converts an angle from radians to degrees. @@ -1191,11 +1257,33 @@ Returns the following: ## ICONTAINS_STRING -`ICONTAINS_STRING(, str)` +Returns `true` if `str` is a substring of `expr`, case-insensitive. Otherwise returns `false`. -**Function type:** [Scalar, string](sql-scalar.md#string-functions) +* **Syntax:** `ICONTAINS_STRING(expr, str)` +* **Function type:** Scalar, string -Finds whether a string is in a given expression, case-insensitive. +
Example + +The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the case-insensitive substring `san`. + +```sql +SELECT + "OriginCityName" AS "origin_city", + ICONTAINS_STRING("OriginCityName", 'san') AS "contains_case_insensitive_string" +FROM "flight-carriers" +LIMIT 2 +``` + +Returns the following: + +| `origin_city` | `contains_case_insensitive_string` | +| -- | -- | +| `San Juan, PR` | `true` | +| `Boston, MA` | `false` | + +
+ +[Learn more](sql-scalar.md#string-functions) ## IPV4_MATCH @@ -1327,19 +1415,59 @@ Returns the minimum value from the provided arguments. ## LEFT -`LEFT(expr, [length])` +Returns the `N` leftmost characters of an expression, where `N` is an integer value. -**Function type:** [Scalar, string](sql-scalar.md#string-functions) +* **Syntax:** `LEFT(expr, N)` +* **Function type:** Scalar, string -Returns the leftmost number of characters from an expression. +
Example + +The following example returns the `3` leftmost characters of the expression `ABCDEFG`. + +```sql +SELECT + 'ABCDEFG' AS "expression", + LEFT('ABCDEFG', 3) AS "leftmost_characters" +``` + +Returns the following: + +| `expression` | `leftmost_characters` | +| -- | -- | +| `ABCDEFG` | `ABC` | + +
+ +[Learn more](sql-scalar.md#string-functions) ## LENGTH -`LENGTH(expr)` +Returns the length of the expression in UTF-16 code units. -**Function type:** [Scalar, string](sql-scalar.md#string-functions) +* **Syntax:** `LENGTH(expr)` +* **Function type:** Scalar, string -Returns the length of the expression in UTF-16 encoding. +
Example + +The following example returns the character length of the `OriginCityName` column from the `flight-carriers` datasource. + +```sql +SELECT + "OriginCityName" AS "origin_city_name", + LENGTH("OriginCityName") AS "city_name_length" +FROM "flight-carriers" +LIMIT 1 +``` + +Returns the following: + +| `origin_city_name` | `city_name_length` | +| -- | -- | +| `San Juan, PR` | `12` | + +
+ +[Learn more](sql-scalar.md#string-functions) ## LN @@ -1755,11 +1883,30 @@ Reverses the given expression. ## RIGHT -`RIGHT(expr, [length])` +Returns the `N` rightmost characters of an expression, where `N` is an integer value. -**Function type:** [Scalar, string](sql-scalar.md#string-functions) +* **Syntax:** `RIGHT(expr, N)` +* **Function type:** Scalar, string -Returns the rightmost number of characters from an expression. +
Example + +The following example returns the `3` rightmost characters of the expression `ABCDEFG`. + +```sql +SELECT + 'ABCDEFG' AS "expression", + RIGHT('ABCDEFG', 3) AS "rightmost_characters" +``` + +Returns the following: + +| `expression` | `rightmost_characters` | +| -- | -- | +| `ABCDEFG` | `EFG` | + +
+ +[Learn more](sql-scalar.md#string-functions) ## ROUND @@ -1943,12 +2090,13 @@ Splits `str1` into an multi-value string on the delimiter specified by `str2`, w ## STRLEN -`STRLEN(expr)` - -**Function type:** [Scalar, string](sql-scalar.md#string-functions) - Alias for [`LENGTH`](#length). +* **Syntax:** `STRLEN(expr)` +* **Function type:** Scalar, string + +[Learn more](sql-scalar.md#string-functions) + ## STRPOS `STRPOS(, )` @@ -2022,12 +2170,33 @@ Returns the quantile for the specified fraction from a T-Digest sketch construct ## TEXTCAT -`TEXTCAT(, )` - -**Function type:** [Scalar, string](sql-scalar.md#string-functions) - Concatenates two string expressions. +* **Syntax:** `TEXTCAT(expr, expr)` +* **Function type:** Scalar, string + +
Example + +The following example concatenates the `OriginState` column from the `flight-carriers` datasource to `, USA`. + +```sql +SELECT + "OriginState" AS "origin_state", + TEXTCAT("OriginState", ', USA') AS "concatenate_state_with_USA" +FROM "flight-carriers" +LIMIT 1 +``` + +Returns the following: + +| `origin_state` | `concatenate_state_with_USA` | +| -- | -- | +| `PR` | `PR, USA` | + +
+ +[Learn more](sql-scalar.md#string-functions) + ## THETA_SKETCH_ESTIMATE `THETA_SKETCH_ESTIMATE(expr)`