diff --git a/docs/multi-stage-query/reference.md b/docs/multi-stage-query/reference.md index 6236b6545258..010bbff2a270 100644 --- a/docs/multi-stage-query/reference.md +++ b/docs/multi-stage-query/reference.md @@ -246,6 +246,7 @@ The following table lists the context parameters for the MSQ task engine: | `durableShuffleStorage` | SELECT, INSERT, REPLACE

Whether to use durable storage for shuffle mesh. To use this feature, configure the durable storage at the server level using `druid.msq.intermediate.storage.enable=true`). If these properties are not configured, any query with the context variable `durableShuffleStorage=true` fails with a configuration error.

| `false` | | `faultTolerance` | SELECT, INSERT, REPLACE

Whether to turn on fault tolerance mode or not. Failed workers are retried based on [Limits](#limits). Cannot be used when `durableShuffleStorage` is explicitly set to false. | `false` | | `selectDestination` | SELECT

Controls where the final result of the select query is written.
Use `taskReport`(the default) to write select results to the task report. This is not scalable since task reports size explodes for large results
Use `durableStorage` to write results to durable storage location. For large results sets, its recommended to use `durableStorage` . To configure durable storage see [`this`](#durable-storage) section. | `taskReport` | +| `waitTillSegmentsLoad` | INSERT, REPLACE

If set, the ingest query waits for the generated segment to be loaded before exiting, else the ingest query exits without waiting. The task and live reports contain the information about the status of loading segments if this flag is set. This will ensure that any future queries made after the ingestion exits will include results from the ingestion. The drawback is that the controller task will stall till the segments are loaded. | `false` | ## Joins diff --git a/docs/querying/sql.md b/docs/querying/sql.md index 378bf302872b..13259bdf4044 100644 --- a/docs/querying/sql.md +++ b/docs/querying/sql.md @@ -57,7 +57,7 @@ Druid SQL supports SELECT queries with the following structure: [ WITH tableName [ ( column1, column2, ... ) ] AS ( query ) ] SELECT [ ALL | DISTINCT ] { * | exprs } FROM { | () | [ INNER | LEFT ] JOIN ON condition } -[, UNNEST(source_expression) as table_alias_name(column_alias_name) ] +[ CROSS JOIN UNNEST(source_expression) as table_alias_name(column_alias_name) ] [ WHERE expr ] [ GROUP BY [ exprs | GROUPING SETS ( (exprs), ... ) | ROLLUP (exprs) | CUBE (exprs) ] ] [ HAVING expr ] @@ -97,7 +97,7 @@ The UNNEST clause unnests array values. It's the SQL equivalent to the [unnest d The following is the general syntax for UNNEST, specifically a query that returns the column that gets unnested: ```sql -SELECT column_alias_name FROM datasource, UNNEST(source_expression1) AS table_alias_name1(column_alias_name1), UNNEST(source_expression2) AS table_alias_name2(column_alias_name2), ... +SELECT column_alias_name FROM datasource CROSS JOIN UNNEST(source_expression1) AS table_alias_name1(column_alias_name1) CROSS JOIN UNNEST(source_expression2) AS table_alias_name2(column_alias_name2) ... ``` * The `datasource` for UNNEST can be any Druid datasource, such as the following: @@ -112,7 +112,7 @@ Keep the following things in mind when writing your query: - You must include the context parameter `"enableUnnest": true`. - You can unnest multiple source expressions in a single query. -- Notice the comma between the datasource and the UNNEST function. This is needed in most cases of the UNNEST function. Specifically, it is not needed when you're unnesting an inline array since the array itself is the datasource. +- Notice the CROSS JOIN between the datasource and the UNNEST function. This is needed in most cases of the UNNEST function. Specifically, it is not needed when you're unnesting an inline array since the array itself is the datasource. - If you view the native explanation of a SQL UNNEST, you'll notice that Druid uses `j0.unnest` as a virtual column to perform the unnest. An underscore is added for each unnest, so you may notice virtual columns named `_j0.unnest` or `__j0.unnest`. - UNNEST preserves the ordering of the source array that is being unnested. diff --git a/docs/tutorials/tutorial-unnest-arrays.md b/docs/tutorials/tutorial-unnest-arrays.md index 1f8c530f8d01..49fdfe98af25 100644 --- a/docs/tutorials/tutorial-unnest-arrays.md +++ b/docs/tutorials/tutorial-unnest-arrays.md @@ -163,7 +163,7 @@ In the results, notice that the column named `dim3` has nested values like `["a" The following is the general syntax for UNNEST: ```sql -SELECT column_alias_name FROM datasource, UNNEST(source_expression) AS table_alias_name(column_alias_name) +SELECT column_alias_name FROM datasource CROSS JOIN UNNEST(source_expression) AS table_alias_name(column_alias_name) ``` In addition, you must supply the following context parameter: @@ -179,7 +179,7 @@ For more information about the syntax, see [UNNEST](../querying/sql.md#unnest). The following query returns a column called `d3` from the table `nested_data`. `d3` contains the unnested values from the source column `dim3`: ```sql -SELECT d3 FROM "nested_data", UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) +SELECT d3 FROM "nested_data" CROSS JOIN UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) ``` Notice the MV_TO_ARRAY helper function, which converts the multi-value records in `dim3` to arrays. It is required since `dim3` is a multi-value string dimension. @@ -191,7 +191,7 @@ If the column you are unnesting is not a string dimension, then you do not need You can unnest into a virtual column (multiple columns treated as one). The following query returns the two source columns and a third virtual column containing the unnested data: ```sql -SELECT dim4,dim5,d45 FROM nested_data, UNNEST(ARRAY[dim4,dim5]) AS example_table(d45) +SELECT dim4,dim5,d45 FROM nested_data CROSS JOIN UNNEST(ARRAY[dim4,dim5]) AS example_table(d45) ``` The virtual column `d45` is the product of the two source columns. Notice how the total number of rows has grown. The table `nested_data` had only seven rows originally. @@ -199,7 +199,7 @@ The virtual column `d45` is the product of the two source columns. Notice how th Another way to unnest a virtual column is to concatenate them with ARRAY_CONCAT: ```sql -SELECT dim4,dim5,d45 FROM nested_data, UNNEST(ARRAY_CONCAT(dim4,dim5)) AS example_table(d45) +SELECT dim4,dim5,d45 FROM nested_data CROSS JOIN UNNEST(ARRAY_CONCAT(dim4,dim5)) AS example_table(d45) ``` Decide which method to use based on what your goals are. @@ -221,7 +221,7 @@ The example query returns the following from the `nested_data` datasource: - an unnested virtual column composed of `dim4` and `dim5` aliased to `d45` ```sql -SELECT dim3,dim4,dim5,d3,d45 FROM "nested_data", UNNEST(MV_TO_ARRAY("dim3")) AS foo1(d3), UNNEST(ARRAY[dim4,dim5]) AS foo2(d45) +SELECT dim3,dim4,dim5,d3,d45 FROM "nested_data" CROSS JOIN UNNEST(MV_TO_ARRAY("dim3")) AS foo1(d3) CROSS JOIN UNNEST(ARRAY[dim4,dim5]) AS foo2(d45) ``` @@ -230,7 +230,7 @@ SELECT dim3,dim4,dim5,d3,d45 FROM "nested_data", UNNEST(MV_TO_ARRAY("dim3")) AS The following query uses only three columns from the `nested_data` table as the datasource. From that subset, it unnests the column `dim3` into `d3` and returns `d3`. ```sql -SELECT d3 FROM (SELECT dim1, dim2, dim3 FROM "nested_data"), UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) +SELECT d3 FROM (SELECT dim1, dim2, dim3 FROM "nested_data") CROSS JOIN UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) ``` ### Unnest with a filter @@ -242,7 +242,7 @@ You can specify which rows to unnest by including a filter in your query. The fo * Returns the records for the unnested `d3` that have a `dim2` record that matches the filter ```sql -SELECT d3 FROM (SELECT * FROM nested_data WHERE dim2 IN ('abc')), UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) +SELECT d3 FROM (SELECT * FROM nested_data WHERE dim2 IN ('abc')) CROSS JOIN UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) ``` You can also filter the results of an UNNEST clause. The following example unnests the inline array `[1,2,3]` but only returns the rows that match the filter: @@ -257,7 +257,7 @@ This means that you can run a query like the following where Druid only return r - The value of `m1` is less than 2. ```sql -SELECT * FROM nested_data, UNNEST(MV_TO_ARRAY("dim3")) AS foo(d3) WHERE d3 IN ('b', 'd') and m1 < 2 +SELECT * FROM nested_data CROSS JOIN UNNEST(MV_TO_ARRAY("dim3")) AS foo(d3) WHERE d3 IN ('b', 'd') and m1 < 2 ``` The query only returns a single row since only one row meets the conditions. You can see the results change if you modify the filter. @@ -267,7 +267,7 @@ The query only returns a single row since only one row meets the conditions. You The following query unnests `dim3` and then performs a GROUP BY on the output `d3`. ```sql -SELECT d3 FROM nested_data, UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) GROUP BY d3 +SELECT d3 FROM nested_data CROSS JOIN UNNEST(MV_TO_ARRAY(dim3)) AS example_table(d3) GROUP BY d3 ``` You can further transform your results by including clauses like `ORDER BY d3 DESC` or LIMIT. diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java index c423b959eccf..c7b10f245c1d 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/ControllerImpl.java @@ -463,14 +463,18 @@ public TaskStatus runTask(final Closer closer) } } + boolean shouldWaitForSegmentLoad = MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context()); try { releaseTaskLocks(); cleanUpDurableStorageIfNeeded(); if (queryKernel != null && queryKernel.isSuccess()) { - if (segmentLoadWaiter != null) { - // If successful and there are segments created, segmentLoadWaiter should wait for them to become available. + if (shouldWaitForSegmentLoad && segmentLoadWaiter != null) { + // If successful, there are segments created and segment load is enabled, segmentLoadWaiter should wait + // for them to become available. + log.info("Controller will now wait for segments to be loaded. The query has already finished executing," + + " and results will be included once the segments are loaded, even if this query is cancelled now."); segmentLoadWaiter.waitForSegmentsToLoad(); } } @@ -1363,31 +1367,35 @@ private void publishAllSegments(final Set segments) throws IOExcept } } else { Set versionsToAwait = segmentsWithTombstones.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); + if (MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context())) { + segmentLoadWaiter = new SegmentLoadStatusFetcher( + context.injector().getInstance(BrokerClient.class), + context.jsonMapper(), + task.getId(), + task.getDataSource(), + versionsToAwait, + segmentsWithTombstones.size(), + true + ); + } + performSegmentPublish( + context.taskActionClient(), + SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones) + ); + } + } else if (!segments.isEmpty()) { + Set versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); + if (MultiStageQueryContext.shouldWaitForSegmentLoad(task.getQuerySpec().getQuery().context())) { segmentLoadWaiter = new SegmentLoadStatusFetcher( context.injector().getInstance(BrokerClient.class), context.jsonMapper(), task.getId(), task.getDataSource(), versionsToAwait, - segmentsWithTombstones.size(), + segments.size(), true ); - performSegmentPublish( - context.taskActionClient(), - SegmentTransactionalInsertAction.overwriteAction(null, segmentsWithTombstones) - ); } - } else if (!segments.isEmpty()) { - Set versionsToAwait = segments.stream().map(DataSegment::getVersion).collect(Collectors.toSet()); - segmentLoadWaiter = new SegmentLoadStatusFetcher( - context.injector().getInstance(BrokerClient.class), - context.jsonMapper(), - task.getId(), - task.getDataSource(), - versionsToAwait, - segments.size(), - true - ); // Append mode. performSegmentPublish( context.taskActionClient(), diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java index 478c632a7491..17f46bad23a2 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/exec/SegmentLoadStatusFetcher.java @@ -41,13 +41,10 @@ import javax.annotation.Nullable; import javax.ws.rs.core.MediaType; -import java.util.HashMap; -import java.util.Iterator; -import java.util.Map; import java.util.Set; -import java.util.TreeSet; import java.util.concurrent.TimeUnit; import java.util.concurrent.atomic.AtomicReference; +import java.util.stream.Collectors; /** * Class that periodically checks with the broker if all the segments generated are loaded by querying the sys table @@ -84,14 +81,14 @@ public class SegmentLoadStatusFetcher implements AutoCloseable + "COUNT(*) FILTER (WHERE is_available = 0 AND is_published = 1 AND replication_factor != 0) AS pendingSegments,\n" + "COUNT(*) FILTER (WHERE replication_factor = -1) AS unknownSegments\n" + "FROM sys.segments\n" - + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version = '%s'"; + + "WHERE datasource = '%s' AND is_overshadowed = 0 AND version in (%s)"; private final BrokerClient brokerClient; private final ObjectMapper objectMapper; // Map of version vs latest load status. - private final Map versionToLoadStatusMap; + private final AtomicReference versionLoadStatusReference; private final String datasource; - private final Set versionsToAwait; + private final String versionsInClauseString; private final int totalSegmentsGenerated; private final boolean doWait; // since live reports fetch the value in another thread, we need to use AtomicReference @@ -112,8 +109,11 @@ public SegmentLoadStatusFetcher( this.brokerClient = brokerClient; this.objectMapper = objectMapper; this.datasource = datasource; - this.versionsToAwait = new TreeSet<>(versionsToAwait); - this.versionToLoadStatusMap = new HashMap<>(); + this.versionsInClauseString = String.join( + ",", + versionsToAwait.stream().map(s -> StringUtils.format("'%s'", s)).collect(Collectors.toSet()) + ); + this.versionLoadStatusReference = new AtomicReference<>(new VersionLoadStatus(0, 0, 0, 0, totalSegmentsGenerated)); this.totalSegmentsGenerated = totalSegmentsGenerated; this.status = new AtomicReference<>(new SegmentLoadWaiterStatus( State.INIT, @@ -145,8 +145,9 @@ public void waitForSegmentsToLoad() final AtomicReference hasAnySegmentBeenLoaded = new AtomicReference<>(false); try { FutureUtils.getUnchecked(executorService.submit(() -> { + long lastLogMillis = -TimeUnit.MINUTES.toMillis(1); try { - while (!versionsToAwait.isEmpty()) { + while (!(hasAnySegmentBeenLoaded.get() && versionLoadStatusReference.get().isLoadingComplete())) { // Check the timeout and exit if exceeded. long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); if (runningMillis > TIMEOUT_DURATION_MILLIS) { @@ -159,29 +160,21 @@ public void waitForSegmentsToLoad() return; } - Iterator iterator = versionsToAwait.iterator(); - log.info( - "Fetching segment load status for datasource[%s] from broker for segment versions[%s]", - datasource, - versionsToAwait - ); - - // Query the broker for all pending versions - while (iterator.hasNext()) { - String version = iterator.next(); - - // Fetch the load status for this version from the broker - VersionLoadStatus loadStatus = fetchLoadStatusForVersion(version); - versionToLoadStatusMap.put(version, loadStatus); - hasAnySegmentBeenLoaded.set(hasAnySegmentBeenLoaded.get() || loadStatus.getUsedSegments() > 0); - - // If loading is done for this stage, remove it from future loops. - if (hasAnySegmentBeenLoaded.get() && loadStatus.isLoadingComplete()) { - iterator.remove(); - } + if (runningMillis - lastLogMillis >= TimeUnit.MINUTES.toMillis(1)) { + lastLogMillis = runningMillis; + log.info( + "Fetching segment load status for datasource[%s] from broker for segment versions[%s]", + datasource, + versionsInClauseString + ); } - if (!versionsToAwait.isEmpty()) { + // Fetch the load status from the broker + VersionLoadStatus loadStatus = fetchLoadStatusFromBroker(); + versionLoadStatusReference.set(loadStatus); + hasAnySegmentBeenLoaded.set(hasAnySegmentBeenLoaded.get() || loadStatus.getUsedSegments() > 0); + + if (!(hasAnySegmentBeenLoaded.get() && versionLoadStatusReference.get().isLoadingComplete())) { // Update the status. updateStatus(State.WAITING, startTime); // Sleep for a bit before checking again. @@ -216,50 +209,45 @@ private void waitIfNeeded(long waitTimeMillis) throws Exception } /** - * Updates the {@link #status} with the latest details based on {@link #versionToLoadStatusMap} + * Updates the {@link #status} with the latest details based on {@link #versionLoadStatusReference} */ private void updateStatus(State state, DateTime startTime) { - int pendingSegmentCount = 0, usedSegmentsCount = 0, precachedSegmentCount = 0, onDemandSegmentCount = 0, unknownSegmentCount = 0; - for (Map.Entry entry : versionToLoadStatusMap.entrySet()) { - usedSegmentsCount += entry.getValue().getUsedSegments(); - precachedSegmentCount += entry.getValue().getPrecachedSegments(); - onDemandSegmentCount += entry.getValue().getOnDemandSegments(); - unknownSegmentCount += entry.getValue().getUnknownSegments(); - pendingSegmentCount += entry.getValue().getPendingSegments(); - } - long runningMillis = new Interval(startTime, DateTimes.nowUtc()).toDurationMillis(); + VersionLoadStatus versionLoadStatus = versionLoadStatusReference.get(); status.set( new SegmentLoadWaiterStatus( state, startTime, runningMillis, totalSegmentsGenerated, - usedSegmentsCount, - precachedSegmentCount, - onDemandSegmentCount, - pendingSegmentCount, - unknownSegmentCount + versionLoadStatus.getUsedSegments(), + versionLoadStatus.getPrecachedSegments(), + versionLoadStatus.getOnDemandSegments(), + versionLoadStatus.getPendingSegments(), + versionLoadStatus.getUnknownSegments() ) ); } /** - * Uses {@link #brokerClient} to fetch latest load status for a given version. Converts the response into a + * Uses {@link #brokerClient} to fetch latest load status for a given set of versions. Converts the response into a * {@link VersionLoadStatus} and returns it. */ - private VersionLoadStatus fetchLoadStatusForVersion(String version) throws Exception + private VersionLoadStatus fetchLoadStatusFromBroker() throws Exception { Request request = brokerClient.makeRequest(HttpMethod.POST, "/druid/v2/sql/"); - SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, version), + SqlQuery sqlQuery = new SqlQuery(StringUtils.format(LOAD_QUERY, datasource, versionsInClauseString), ResultFormat.OBJECTLINES, false, false, false, null, null ); request.setContent(MediaType.APPLICATION_JSON, objectMapper.writeValueAsBytes(sqlQuery)); String response = brokerClient.sendQuery(request); - if (response.trim().isEmpty()) { + if (response == null) { + // Unable to query broker + return new VersionLoadStatus(0, 0, 0, 0, totalSegmentsGenerated); + } else if (response.trim().isEmpty()) { // If no segments are returned for a version, all segments have been dropped by a drop rule. return new VersionLoadStatus(0, 0, 0, 0, 0); } else { diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java index c7b20ac46e08..2efc94740ac7 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/DimensionSchemaUtils.java @@ -75,6 +75,10 @@ public static DimensionSchema createDimensionSchema( switch (type.getElementType().getType()) { case STRING: return new StringDimensionSchema(column, DimensionSchema.MultiValueHandling.ARRAY, null); + case LONG: + case FLOAT: + case DOUBLE: + return new AutoTypeColumnSchema(column); default: throw new ISE("Cannot create dimension for type [%s]", type.toString()); } diff --git a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java index 265f5eae0fe1..98dcd471d0fe 100644 --- a/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java +++ b/extensions-core/multi-stage-query/src/main/java/org/apache/druid/msq/util/MultiStageQueryContext.java @@ -97,6 +97,8 @@ public class MultiStageQueryContext public static final String CTX_FAULT_TOLERANCE = "faultTolerance"; public static final boolean DEFAULT_FAULT_TOLERANCE = false; + public static final String CTX_SEGMENT_LOAD_WAIT = "waitTillSegmentsLoad"; + public static final boolean DEFAULT_SEGMENT_LOAD_WAIT = false; public static final String CTX_MAX_INPUT_BYTES_PER_WORKER = "maxInputBytesPerWorker"; public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode"; @@ -148,6 +150,14 @@ public static boolean isFaultToleranceEnabled(final QueryContext queryContext) ); } + public static boolean shouldWaitForSegmentLoad(final QueryContext queryContext) + { + return queryContext.getBoolean( + CTX_SEGMENT_LOAD_WAIT, + DEFAULT_SEGMENT_LOAD_WAIT + ); + } + public static boolean isReindex(final QueryContext queryContext) { return queryContext.getBoolean( diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java index b3bf3c408a29..42bb1506a307 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQFaultsTest.java @@ -31,7 +31,6 @@ import org.apache.druid.msq.indexing.error.TooManyColumnsFault; import org.apache.druid.msq.indexing.error.TooManyInputFilesFault; import org.apache.druid.msq.indexing.error.TooManyPartitionsFault; -import org.apache.druid.msq.indexing.error.UnknownFault; import org.apache.druid.msq.test.MSQTestBase; import org.apache.druid.msq.test.MSQTestFileUtils; import org.apache.druid.segment.column.ColumnType; @@ -220,31 +219,6 @@ public void testInsertWithTooManySegments() throws IOException } - @Test - public void testInsertWithUnsupportedColumnType() - { - RowSignature dummyRowSignature = RowSignature.builder().add("__time", ColumnType.LONG).build(); - - testIngestQuery() - .setSql(StringUtils.format( - " insert into foo1 SELECT\n" - + " floor(TIME_PARSE(\"timestamp\") to day) AS __time,\n" - + " col1\n" - + "FROM TABLE(\n" - + " EXTERN(\n" - + " '{ \"files\": [\"ignored\"],\"type\":\"local\"}',\n" - + " '{\"type\": \"json\"}',\n" - + " '[{\"name\": \"timestamp\", \"type\": \"string\"},{\"name\": \"col1\", \"type\": \"long_array\"} ]'\n" - + " )\n" - + ") PARTITIONED by day" - )) - .setExpectedDataSource("foo1") - .setExpectedRowSignature(dummyRowSignature) - .setExpectedMSQFault(UnknownFault.forMessage( - "org.apache.druid.java.util.common.ISE: Cannot create dimension for type [ARRAY]")) - .verifyResults(); - } - @Test public void testInsertWithManyColumns() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java index 81ba53f755ef..e54027c2449b 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQInsertTest.java @@ -63,6 +63,7 @@ import java.util.ArrayList; import java.util.Arrays; import java.util.Collection; +import java.util.Collections; import java.util.HashMap; import java.util.List; import java.util.Map; @@ -1463,6 +1464,193 @@ public void testInsertArraysAutoType() throws IOException .verifyResults(); } + @Test + public void testInsertArrays() throws IOException + { + List expectedRows = Arrays.asList( + new Object[]{ + 1672531200000L, + null, + null, + new Object[]{1L, 2L, 3L}, + new Object[]{}, + new Object[]{1.1d, 2.2d, 3.3d}, + null + }, + new Object[]{ + 1672531200000L, + null, + Arrays.asList("a", "b"), + null, + new Object[]{2L, 3L}, + null, + new Object[]{null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("a", "b"), + null, + null, + new Object[]{null, 2L, 9L}, + null, + new Object[]{999.0d, 5.5d, null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("a", "b"), + Arrays.asList("a", "b"), + new Object[]{1L, 2L, 3L}, + new Object[]{1L, null, 3L}, + new Object[]{1.1d, 2.2d, 3.3d}, + new Object[]{1.1d, 2.2d, null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("a", "b", "c"), + Arrays.asList(null, "b"), + new Object[]{2L, 3L}, + null, + new Object[]{3.3d, 4.4d, 5.5d}, + new Object[]{999.0d, null, 5.5d} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("b", "c"), + Arrays.asList("d", null, "b"), + new Object[]{1L, 2L, 3L, 4L}, + new Object[]{1L, 2L, 3L}, + new Object[]{1.1d, 3.3d}, + new Object[]{null, 2.2d, null} + }, + new Object[]{ + 1672531200000L, + Arrays.asList("d", "e"), + Arrays.asList("b", "b"), + new Object[]{1L, 4L}, + new Object[]{1L}, + new Object[]{2.2d, 3.3d, 4.0d}, + null + }, + new Object[]{ + 1672617600000L, + null, + null, + new Object[]{1L, 2L, 3L}, + null, + new Object[]{1.1d, 2.2d, 3.3d}, + new Object[]{} + }, + new Object[]{ + 1672617600000L, + null, + Arrays.asList("a", "b"), + null, + new Object[]{2L, 3L}, + null, + new Object[]{null, 1.1d} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("a", "b"), + null, + null, + new Object[]{null, 2L, 9L}, + null, + new Object[]{999.0d, 5.5d, null} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("a", "b"), + Collections.emptyList(), + new Object[]{1L, 2L, 3L}, + new Object[]{1L, null, 3L}, + new Object[]{1.1d, 2.2d, 3.3d}, + new Object[]{1.1d, 2.2d, null} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("a", "b", "c"), + Arrays.asList(null, "b"), + new Object[]{2L, 3L}, + null, + new Object[]{3.3d, 4.4d, 5.5d}, + new Object[]{999.0d, null, 5.5d} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("b", "c"), + Arrays.asList("d", null, "b"), + new Object[]{1L, 2L, 3L, 4L}, + new Object[]{1L, 2L, 3L}, + new Object[]{1.1d, 3.3d}, + new Object[]{null, 2.2d, null} + }, + new Object[]{ + 1672617600000L, + Arrays.asList("d", "e"), + Arrays.asList("b", "b"), + new Object[]{1L, 4L}, + new Object[]{null}, + new Object[]{2.2d, 3.3d, 4.0}, + null + } + ); + + RowSignature rowSignatureWithoutTimeAndStringColumns = + RowSignature.builder() + .add("arrayLong", ColumnType.LONG_ARRAY) + .add("arrayLongNulls", ColumnType.LONG_ARRAY) + .add("arrayDouble", ColumnType.DOUBLE_ARRAY) + .add("arrayDoubleNulls", ColumnType.DOUBLE_ARRAY) + .build(); + + + RowSignature fileSignature = RowSignature.builder() + .add("timestamp", ColumnType.STRING) + .add("arrayString", ColumnType.STRING_ARRAY) + .add("arrayStringNulls", ColumnType.STRING_ARRAY) + .addAll(rowSignatureWithoutTimeAndStringColumns) + .build(); + + // MSQ writes strings instead of string arrays + RowSignature rowSignature = RowSignature.builder() + .add("__time", ColumnType.LONG) + .add("arrayString", ColumnType.STRING) + .add("arrayStringNulls", ColumnType.STRING) + .addAll(rowSignatureWithoutTimeAndStringColumns) + .build(); + + final Map adjustedContext = new HashMap<>(context); + final File tmpFile = temporaryFolder.newFile(); + final InputStream resourceStream = NestedDataTestUtils.class.getClassLoader().getResourceAsStream(NestedDataTestUtils.ARRAY_TYPES_DATA_FILE); + final InputStream decompressing = CompressionUtils.decompress(resourceStream, NestedDataTestUtils.ARRAY_TYPES_DATA_FILE); + Files.copy(decompressing, tmpFile.toPath(), StandardCopyOption.REPLACE_EXISTING); + decompressing.close(); + + final String toReadFileNameAsJson = queryFramework().queryJsonMapper().writeValueAsString(tmpFile); + + testIngestQuery().setSql(" INSERT INTO foo1 SELECT\n" + + " TIME_PARSE(\"timestamp\") as __time,\n" + + " arrayString,\n" + + " arrayStringNulls,\n" + + " arrayLong,\n" + + " arrayLongNulls,\n" + + " arrayDouble,\n" + + " arrayDoubleNulls\n" + + "FROM TABLE(\n" + + " EXTERN(\n" + + " '{ \"files\": [" + toReadFileNameAsJson + "],\"type\":\"local\"}',\n" + + " '{\"type\": \"json\"}',\n" + + " '" + queryFramework().queryJsonMapper().writeValueAsString(fileSignature) + "'\n" + + " )\n" + + ") PARTITIONED BY day") + .setQueryContext(adjustedContext) + .setExpectedResultRows(expectedRows) + .setExpectedDataSource("foo1") + .setExpectedRowSignature(rowSignature) + .verifyResults(); + } + @Nonnull private List expectedFooRows() { diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysSelectQueryMSQTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java similarity index 97% rename from extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysSelectQueryMSQTest.java rename to extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java index 16aef6bdcaaa..82301f4ddfed 100644 --- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysSelectQueryMSQTest.java +++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/test/CalciteArraysQueryMSQTest.java @@ -36,7 +36,7 @@ /** * Runs {@link CalciteArraysQueryTest} but with MSQ engine */ -public class CalciteArraysSelectQueryMSQTest extends CalciteArraysQueryTest +public class CalciteArraysQueryMSQTest extends CalciteArraysQueryTest { private TestGroupByBuffers groupByBuffers; diff --git a/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java new file mode 100644 index 000000000000..48f2a5117ee8 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/DoubleArrayFieldReader.java @@ -0,0 +1,62 @@ +/* + * 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.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +/** + * Reader for fields written by {@link NumericArrayFieldWriter#getDoubleArrayFieldWriter} + */ +public class DoubleArrayFieldReader extends NumericArrayFieldReader +{ + @Override + public ColumnValueSelector makeColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer + ) + { + return new NumericArrayFieldSelector(memory, fieldPointer) + { + final SettableFieldPointer fieldPointer = new SettableFieldPointer(); + final ColumnValueSelector columnValueSelector = + DoubleFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); + + @Nullable + @Override + public Double getIndividualValueAtMemory(long position) + { + fieldPointer.setPosition(position); + if (columnValueSelector.isNull()) { + return null; + } + return columnValueSelector.getDouble(); + } + + @Override + public int getIndividualFieldSize() + { + return Byte.BYTES + Double.BYTES; + } + }; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java index 531e14bbda29..7f7a3f8639eb 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldReader.java @@ -21,74 +21,60 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.DoubleColumnSelector; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.column.ValueTypes; - -import javax.annotation.Nullable; /** - * Reads values written by {@link DoubleFieldWriter}. - * - * Values are sortable as bytes without decoding. + * Reads the values produced by {@link DoubleFieldWriter} * - * Format: - * - * - 1 byte: {@link DoubleFieldWriter#NULL_BYTE} or {@link DoubleFieldWriter#NOT_NULL_BYTE} - * - 8 bytes: encoded double, using {@link DoubleFieldWriter#transform} + * @see DoubleFieldWriter + * @see NumericFieldWriter for the details of the byte-format that it expects for reading */ -public class DoubleFieldReader implements FieldReader +public class DoubleFieldReader extends NumericFieldReader { - DoubleFieldReader() + + public static DoubleFieldReader forPrimitive() { + return new DoubleFieldReader(false); } - @Override - public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + public static DoubleFieldReader forArray() { - return new Selector(memory, fieldPointer); + return new DoubleFieldReader(true); } - @Override - public DimensionSelector makeDimensionSelector( - Memory memory, - ReadableFieldPointer fieldPointer, - @Nullable ExtractionFn extractionFn - ) + private DoubleFieldReader(final boolean forArray) { - return ValueTypes.makeNumericWrappingDimensionSelector( - ValueType.DOUBLE, - makeColumnValueSelector(memory, fieldPointer), - extractionFn - ); + super(forArray); } @Override - public boolean isNull(Memory memory, long position) + public ValueType getValueType() { - return memory.getByte(position) == DoubleFieldWriter.NULL_BYTE; + return ValueType.DOUBLE; } @Override - public boolean isComparable() + public ColumnValueSelector getColumnValueSelector( + final Memory memory, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) { - return true; + return new DoubleFieldReader.DoubleFieldSelector(memory, fieldPointer, nullIndicatorByte); } - /** - * Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}. - */ - private static class Selector implements DoubleColumnSelector + private static class DoubleFieldSelector extends NumericFieldReader.Selector implements DoubleColumnSelector { - private final Memory dataRegion; - private final ReadableFieldPointer fieldPointer; - private Selector(final Memory dataRegion, final ReadableFieldPointer fieldPointer) + final Memory dataRegion; + final ReadableFieldPointer fieldPointer; + + public DoubleFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte) { + super(dataRegion, fieldPointer, nullIndicatorByte); this.dataRegion = dataRegion; this.fieldPointer = fieldPointer; } @@ -98,19 +84,19 @@ public double getDouble() { assert NullHandling.replaceWithDefault() || !isNull(); final long bits = dataRegion.getLong(fieldPointer.position() + Byte.BYTES); - return DoubleFieldWriter.detransform(bits); + return TransformUtils.detransformToDouble(bits); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return dataRegion.getByte(fieldPointer.position()) == DoubleFieldWriter.NULL_BYTE; + } @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + public boolean isNull() { - // Do nothing. + return super._isNull(); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java index 87ab109af1c3..08cd2a2bf4c8 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/field/DoubleFieldWriter.java @@ -25,65 +25,49 @@ /** * Wraps a {@link BaseDoubleColumnValueSelector} and writes field values. * - * See {@link DoubleFieldReader} for format details. + * @see NumericFieldWriter for the details of the byte-format that it writes as */ -public class DoubleFieldWriter implements FieldWriter +public class DoubleFieldWriter extends NumericFieldWriter { - public static final int SIZE = Double.BYTES + Byte.BYTES; + private final BaseDoubleColumnValueSelector selector; - // Different from the values in NullHandling, since we want to be able to sort as bytes, and we want - // nulls to come before non-nulls. - public static final byte NULL_BYTE = 0x00; - public static final byte NOT_NULL_BYTE = 0x01; + public static DoubleFieldWriter forPrimitive(final BaseDoubleColumnValueSelector selector) + { + return new DoubleFieldWriter(selector, false); + } - private final BaseDoubleColumnValueSelector selector; + public static DoubleFieldWriter forArray(final BaseDoubleColumnValueSelector selector) + { + return new DoubleFieldWriter(selector, true); + } - public DoubleFieldWriter(final BaseDoubleColumnValueSelector selector) + private DoubleFieldWriter(final BaseDoubleColumnValueSelector selector, final boolean forArray) { + super(selector, forArray); this.selector = selector; } @Override - public long writeTo(final WritableMemory memory, final long position, final long maxSize) + public int getNumericSizeBytes() { - if (maxSize < SIZE) { - return -1; - } - - if (selector.isNull()) { - memory.putByte(position, NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(0)); - } else { - memory.putByte(position, NOT_NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(selector.getDouble())); - } - - return SIZE; + return Double.BYTES; } @Override - public void close() + public void writeSelectorToMemory(WritableMemory memory, long position) { - // Nothing to close. + writeToMemory(memory, position, selector.getDouble()); } - /** - * Transforms a double into a form where it can be compared as unsigned bytes without decoding. - */ - public static long transform(final double n) + @Override + public void writeNullToMemory(WritableMemory memory, long position) { - final long bits = Double.doubleToLongBits(n); - final long mask = ((bits & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; - return Long.reverseBytes(bits ^ mask); + writeToMemory(memory, position, 0); } - /** - * Inverse of {@link #transform}. - */ - public static double detransform(final long bits) + private void writeToMemory(WritableMemory memory, long position, double value) { - final long reversedBits = Long.reverseBytes(bits); - final long mask = (((reversedBits ^ Long.MIN_VALUE) & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; - return Double.longBitsToDouble(reversedBits ^ mask); + memory.putLong(position, TransformUtils.transformFromDouble(value)); } + } diff --git a/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java b/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java index 23b3907ff18c..7221bfe9da26 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FieldReaders.java @@ -24,7 +24,6 @@ import org.apache.druid.frame.key.RowKeyReader; import org.apache.druid.frame.write.UnsupportedColumnTypeException; import org.apache.druid.segment.column.ColumnType; -import org.apache.druid.segment.column.ValueType; /** * Helper used to read field values from row-based frames or {@link RowKey}. @@ -46,23 +45,37 @@ public static FieldReader create(final String columnName, final ColumnType colum { switch (Preconditions.checkNotNull(columnType, "columnType").getType()) { case LONG: - return new LongFieldReader(); + return LongFieldReader.forPrimitive(); case FLOAT: - return new FloatFieldReader(); + return FloatFieldReader.forPrimitive(); case DOUBLE: - return new DoubleFieldReader(); + return DoubleFieldReader.forPrimitive(); case STRING: - return new StringFieldReader(false); + return new StringFieldReader(); case COMPLEX: return ComplexFieldReader.createFromType(columnType); case ARRAY: - if (columnType.getElementType().getType() == ValueType.STRING) { - return new StringFieldReader(true); + switch (Preconditions.checkNotNull(columnType.getElementType().getType(), "array elementType")) { + case STRING: + return new StringArrayFieldReader(); + + case LONG: + return new LongArrayFieldReader(); + + case FLOAT: + return new FloatArrayFieldReader(); + + case DOUBLE: + return new DoubleArrayFieldReader(); + + default: + throw new UnsupportedColumnTypeException(columnName, columnType); + } // Fall through to error for other array types diff --git a/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java b/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java index 622f9a6751ea..cc7f68d099d2 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FieldWriters.java @@ -66,21 +66,30 @@ public static FieldWriter create( switch (columnType.getType()) { case LONG: return makeLongWriter(columnSelectorFactory, columnName); + case FLOAT: return makeFloatWriter(columnSelectorFactory, columnName); + case DOUBLE: return makeDoubleWriter(columnSelectorFactory, columnName); + case STRING: return makeStringWriter(columnSelectorFactory, columnName); + + case COMPLEX: + return makeComplexWriter(columnSelectorFactory, columnName, columnType.getComplexTypeName()); + case ARRAY: switch (columnType.getElementType().getType()) { case STRING: return makeStringArrayWriter(columnSelectorFactory, columnName); - default: - throw new UnsupportedColumnTypeException(columnName, columnType); + case LONG: + return makeLongArrayWriter(columnSelectorFactory, columnName); + case FLOAT: + return makeFloatArrayWriter(columnSelectorFactory, columnName); + case DOUBLE: + return makeDoubleArrayWriter(columnSelectorFactory, columnName); } - case COMPLEX: - return makeComplexWriter(columnSelectorFactory, columnName, columnType.getComplexTypeName()); default: throw new UnsupportedColumnTypeException(columnName, columnType); } @@ -92,7 +101,7 @@ private static FieldWriter makeLongWriter( ) { final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); - return new LongFieldWriter(selector); + return LongFieldWriter.forPrimitive(selector); } private static FieldWriter makeFloatWriter( @@ -101,7 +110,7 @@ private static FieldWriter makeFloatWriter( ) { final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); - return new FloatFieldWriter(selector); + return FloatFieldWriter.forPrimitive(selector); } private static FieldWriter makeDoubleWriter( @@ -110,7 +119,7 @@ private static FieldWriter makeDoubleWriter( ) { final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); - return new DoubleFieldWriter(selector); + return DoubleFieldWriter.forPrimitive(selector); } private static FieldWriter makeStringWriter( @@ -127,11 +136,37 @@ private static FieldWriter makeStringArrayWriter( final String columnName ) { - //noinspection rawtypes - final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); return new StringArrayFieldWriter(selector); } + private static FieldWriter makeLongArrayWriter( + final ColumnSelectorFactory selectorFactory, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return NumericArrayFieldWriter.getLongArrayFieldWriter(selector); + } + + private static FieldWriter makeFloatArrayWriter( + final ColumnSelectorFactory selectorFactory, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return NumericArrayFieldWriter.getFloatArrayFieldWriter(selector); + } + + private static FieldWriter makeDoubleArrayWriter( + final ColumnSelectorFactory selectorFactory, + final String columnName + ) + { + final ColumnValueSelector selector = selectorFactory.makeColumnValueSelector(columnName); + return NumericArrayFieldWriter.getDoubleArrayFieldWriter(selector); + } + private static FieldWriter makeComplexWriter( final ColumnSelectorFactory selectorFactory, final String columnName, diff --git a/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java new file mode 100644 index 000000000000..fcbe407bdb26 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/FloatArrayFieldReader.java @@ -0,0 +1,62 @@ +/* + * 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.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +/** + * Reader for fields written by {@link NumericArrayFieldWriter#getFloatArrayFieldWriter} + */ +public class FloatArrayFieldReader extends NumericArrayFieldReader +{ + @Override + public ColumnValueSelector makeColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer + ) + { + return new NumericArrayFieldSelector(memory, fieldPointer) + { + final SettableFieldPointer fieldPointer = new SettableFieldPointer(); + final ColumnValueSelector columnValueSelector = + FloatFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); + + @Nullable + @Override + public Float getIndividualValueAtMemory(long position) + { + fieldPointer.setPosition(position); + if (columnValueSelector.isNull()) { + return null; + } + return columnValueSelector.getFloat(); + } + + @Override + public int getIndividualFieldSize() + { + return Byte.BYTES + Float.BYTES; + } + }; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java index 17baaa95df47..6617d563d679 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldReader.java @@ -21,71 +21,60 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.FloatColumnSelector; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.column.ValueTypes; - -import javax.annotation.Nullable; /** * Reads values written by {@link FloatFieldWriter}. * - * Values are sortable as bytes without decoding. - * - * Format: - * - * - 1 byte: {@link FloatFieldWriter#NULL_BYTE} or {@link FloatFieldWriter#NOT_NULL_BYTE} - * - 4 bytes: encoded float, using {@link FloatFieldWriter#transform} + * @see FloatFieldWriter + * @see NumericFieldWriter for the details of the byte-format that it expects for reading */ -public class FloatFieldReader implements FieldReader +public class FloatFieldReader extends NumericFieldReader { - FloatFieldReader() + + public static FloatFieldReader forPrimitive() { + return new FloatFieldReader(false); } - @Override - public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + public static FloatFieldReader forArray() { - return new Selector(memory, fieldPointer); + return new FloatFieldReader(true); } - @Override - public DimensionSelector makeDimensionSelector( - Memory memory, - ReadableFieldPointer fieldPointer, - @Nullable ExtractionFn extractionFn - ) + private FloatFieldReader(final boolean forArray) { - return ValueTypes.makeNumericWrappingDimensionSelector( - ValueType.FLOAT, - makeColumnValueSelector(memory, fieldPointer), - extractionFn - ); + super(forArray); } @Override - public boolean isNull(Memory memory, long position) + public ValueType getValueType() { - return memory.getByte(position) == FloatFieldWriter.NULL_BYTE; + return ValueType.FLOAT; } @Override - public boolean isComparable() + public ColumnValueSelector getColumnValueSelector( + final Memory memory, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) { - return true; + return new FloatFieldSelector(memory, fieldPointer, nullIndicatorByte); } - private static class Selector implements FloatColumnSelector + private static class FloatFieldSelector extends NumericFieldReader.Selector implements FloatColumnSelector { - private final Memory dataRegion; - private final ReadableFieldPointer fieldPointer; - private Selector(final Memory dataRegion, final ReadableFieldPointer fieldPointer) + final Memory dataRegion; + final ReadableFieldPointer fieldPointer; + + public FloatFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte) { + super(dataRegion, fieldPointer, nullIndicatorByte); this.dataRegion = dataRegion; this.fieldPointer = fieldPointer; } @@ -95,19 +84,19 @@ public float getFloat() { assert NullHandling.replaceWithDefault() || !isNull(); final int bits = dataRegion.getInt(fieldPointer.position() + Byte.BYTES); - return FloatFieldWriter.detransform(bits); + return TransformUtils.detransformToFloat(bits); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return dataRegion.getByte(fieldPointer.position()) == FloatFieldWriter.NULL_BYTE; + } @Override - public void inspectRuntimeShape(RuntimeShapeInspector inspector) + public boolean isNull() { - // Do nothing. + return super._isNull(); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java index e81f0b8e099c..cf826932f8b0 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/field/FloatFieldWriter.java @@ -25,65 +25,54 @@ /** * Wraps a {@link BaseFloatColumnValueSelector} and writes field values. * - * See {@link FloatFieldReader} for format details. + * @see NumericFieldWriter for the details of the byte-format that it writes as */ -public class FloatFieldWriter implements FieldWriter +public class FloatFieldWriter extends NumericFieldWriter { - public static final int SIZE = Float.BYTES + Byte.BYTES; + private final BaseFloatColumnValueSelector selector; - // Different from the values in NullHandling, since we want to be able to sort as bytes, and we want - // nulls to come before non-nulls. - public static final byte NULL_BYTE = 0x00; - public static final byte NOT_NULL_BYTE = 0x01; + public static FloatFieldWriter forPrimitive(final BaseFloatColumnValueSelector selector) + { + return new FloatFieldWriter(selector, false); + } - private final BaseFloatColumnValueSelector selector; + public static FloatFieldWriter forArray(final BaseFloatColumnValueSelector selector) + { + return new FloatFieldWriter(selector, true); + } - public FloatFieldWriter(final BaseFloatColumnValueSelector selector) + private FloatFieldWriter(final BaseFloatColumnValueSelector selector, final boolean forArray) { + super(selector, forArray); this.selector = selector; } @Override - public long writeTo(final WritableMemory memory, final long position, final long maxSize) + public void close() { - if (maxSize < SIZE) { - return -1; - } - - if (selector.isNull()) { - memory.putByte(position, NULL_BYTE); - memory.putInt(position + Byte.BYTES, transform(0)); - } else { - memory.putByte(position, NOT_NULL_BYTE); - memory.putInt(position + Byte.BYTES, transform(selector.getFloat())); - } + // Nothing to close. + } - return SIZE; + @Override + public int getNumericSizeBytes() + { + return Float.BYTES; } @Override - public void close() + public void writeSelectorToMemory(WritableMemory memory, long position) { - // Nothing to close. + writeToMemory(memory, position, selector.getFloat()); } - /** - * Transforms a float into a form where it can be compared as unsigned bytes without decoding. - */ - public static int transform(final float n) + @Override + public void writeNullToMemory(WritableMemory memory, long position) { - final int bits = Float.floatToIntBits(n); - final int mask = ((bits & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; - return Integer.reverseBytes(bits ^ mask); + writeToMemory(memory, position, 0); } - /** - * Inverse of {@link #transform}. - */ - public static float detransform(final int bits) + private void writeToMemory(WritableMemory memory, long position, float value) { - final int reversedBits = Integer.reverseBytes(bits); - final int mask = (((reversedBits ^ Integer.MIN_VALUE) & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; - return Float.intBitsToFloat(reversedBits ^ mask); + memory.putInt(position, TransformUtils.transformFromFloat(value)); } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java new file mode 100644 index 000000000000..b52b39d13c47 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/LongArrayFieldReader.java @@ -0,0 +1,62 @@ +/* + * 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.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; + +/** + * Reader for fields written by {@link NumericArrayFieldWriter#getLongArrayFieldWriter} + */ +public class LongArrayFieldReader extends NumericArrayFieldReader +{ + @Override + public ColumnValueSelector makeColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer + ) + { + return new NumericArrayFieldSelector(memory, fieldPointer) + { + final SettableFieldPointer fieldPointer = new SettableFieldPointer(); + final ColumnValueSelector columnValueSelector = + LongFieldReader.forArray().makeColumnValueSelector(memory, fieldPointer); + + @Nullable + @Override + public Long getIndividualValueAtMemory(long position) + { + fieldPointer.setPosition(position); + if (columnValueSelector.isNull()) { + return null; + } + return columnValueSelector.getLong(); + } + + @Override + public int getIndividualFieldSize() + { + return Byte.BYTES + Long.BYTES; + } + }; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java index dd8b6b40ad27..8f3bbbf04517 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/LongFieldReader.java @@ -21,75 +21,61 @@ import org.apache.datasketches.memory.Memory; import org.apache.druid.common.config.NullHandling; -import org.apache.druid.query.extraction.ExtractionFn; import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; import org.apache.druid.segment.ColumnValueSelector; -import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.LongColumnSelector; import org.apache.druid.segment.column.ValueType; -import org.apache.druid.segment.column.ValueTypes; - -import javax.annotation.Nullable; /** * Reads values written by {@link LongFieldWriter}. * - * Values are sortable as bytes without decoding. - * - * Format: - * - * - 1 byte: {@link LongFieldWriter#NULL_BYTE} or {@link LongFieldWriter#NOT_NULL_BYTE} - * - 8 bytes: encoded long: big-endian order, with sign flipped + * @see LongFieldWriter + * @see NumericFieldWriter for the details of the byte-format that it expects for reading */ -public class LongFieldReader implements FieldReader +public class LongFieldReader extends NumericFieldReader { - LongFieldReader() + + public static LongFieldReader forPrimitive() { + return new LongFieldReader(false); } - @Override - public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + public static LongFieldReader forArray() { - return new Selector(memory, fieldPointer); + return new LongFieldReader(true); } - @Override - public DimensionSelector makeDimensionSelector( - Memory memory, - ReadableFieldPointer fieldPointer, - @Nullable ExtractionFn extractionFn - ) + private LongFieldReader(final boolean forArray) { - return ValueTypes.makeNumericWrappingDimensionSelector( - ValueType.LONG, - makeColumnValueSelector(memory, fieldPointer), - extractionFn - ); + super(forArray); } @Override - public boolean isNull(Memory memory, long position) + public ValueType getValueType() { - return memory.getByte(position) == LongFieldWriter.NULL_BYTE; + return ValueType.LONG; } @Override - public boolean isComparable() + public ColumnValueSelector getColumnValueSelector( + final Memory memory, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) { - return true; + return new LongFieldSelector(memory, fieldPointer, nullIndicatorByte); } - /** - * Selector that reads a value from a location pointed to by {@link ReadableFieldPointer}. - */ - private static class Selector implements LongColumnSelector + private static class LongFieldSelector extends NumericFieldReader.Selector implements LongColumnSelector { - private final Memory memory; - private final ReadableFieldPointer fieldPointer; - private Selector(final Memory memory, final ReadableFieldPointer fieldPointer) + final Memory dataRegion; + final ReadableFieldPointer fieldPointer; + + public LongFieldSelector(Memory dataRegion, ReadableFieldPointer fieldPointer, byte nullIndicatorByte) { - this.memory = memory; + super(dataRegion, fieldPointer, nullIndicatorByte); + this.dataRegion = dataRegion; this.fieldPointer = fieldPointer; } @@ -97,20 +83,20 @@ private Selector(final Memory memory, final ReadableFieldPointer fieldPointer) public long getLong() { assert NullHandling.replaceWithDefault() || !isNull(); - final long bits = memory.getLong(fieldPointer.position() + Byte.BYTES); - return LongFieldWriter.detransform(bits); + final long bits = dataRegion.getLong(fieldPointer.position() + Byte.BYTES); + return TransformUtils.detransformToLong(bits); } @Override - public boolean isNull() + public void inspectRuntimeShape(RuntimeShapeInspector inspector) { - return memory.getByte(fieldPointer.position()) == LongFieldWriter.NULL_BYTE; + } @Override - public void inspectRuntimeShape(final RuntimeShapeInspector inspector) + public boolean isNull() { - // Do nothing. + return super._isNull(); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java index ddd1884a6d21..669d384ff157 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java +++ b/processing/src/main/java/org/apache/druid/frame/field/LongFieldWriter.java @@ -23,63 +23,55 @@ import org.apache.druid.segment.BaseLongColumnValueSelector; /** - * Wraps a {@link BaseLongColumnValueSelector} and writes individual values into rframe rows. + * Wraps a {@link BaseLongColumnValueSelector} and writes individual values into frame rows. * - * See {@link LongFieldReader} for format details. + * @see NumericFieldWriter for the details of the byte-format that it writes as */ -public class LongFieldWriter implements FieldWriter +public class LongFieldWriter extends NumericFieldWriter { - public static final int SIZE = Long.BYTES + Byte.BYTES; - public static final byte NULL_BYTE = 0x00; - public static final byte NOT_NULL_BYTE = 0x01; - private final BaseLongColumnValueSelector selector; + public static LongFieldWriter forPrimitive(final BaseLongColumnValueSelector selector) + { + return new LongFieldWriter(selector, false); + } + + static LongFieldWriter forArray(final BaseLongColumnValueSelector selector) + { + return new LongFieldWriter(selector, true); + } + + // Different from the values in NullHandling, since we want to be able to sort as bytes, and we want // nulls to come before non-nulls. - public LongFieldWriter(final BaseLongColumnValueSelector selector) + private LongFieldWriter(final BaseLongColumnValueSelector selector, final boolean forArray) { + super(selector, forArray); this.selector = selector; } @Override - public long writeTo(final WritableMemory memory, final long position, final long maxSize) + public int getNumericSizeBytes() { - if (maxSize < SIZE) { - return -1; - } - - if (selector.isNull()) { - memory.putByte(position, NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(0)); - } else { - memory.putByte(position, NOT_NULL_BYTE); - memory.putLong(position + Byte.BYTES, transform(selector.getLong())); - } - - return SIZE; + return Long.BYTES; } @Override - public void close() + public void writeSelectorToMemory(WritableMemory memory, long position) { - // Nothing to do. + writeToMemory(memory, position, selector.getLong()); } - /** - * Transforms a long into a form where it can be compared as unsigned bytes without decoding. - */ - public static long transform(final long n) + @Override + public void writeNullToMemory(WritableMemory memory, long position) { - // Must flip the first (sign) bit so comparison-as-bytes works. - return Long.reverseBytes(n ^ Long.MIN_VALUE); + writeToMemory(memory, position, 0); + } - /** - * Reverse the {@link #transform(long)} function. - */ - public static long detransform(final long bits) + private void writeToMemory(WritableMemory memory, long position, long value) { - return Long.reverseBytes(bits) ^ Long.MIN_VALUE; + memory.putLong(position, TransformUtils.transformFromLong(value)); } + } diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java new file mode 100644 index 000000000000..7eab229e6be6 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldReader.java @@ -0,0 +1,59 @@ +/* + * 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.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.segment.DimensionSelector; + +import javax.annotation.Nullable; + +/** + * Reader class for the fields written by {@link NumericArrayFieldWriter}. See the Javadoc for the writer for more + * information on the format + * + * The numeric array fields are byte comparable + */ +public abstract class NumericArrayFieldReader implements FieldReader +{ + @Override + public DimensionSelector makeDimensionSelector( + Memory memory, + ReadableFieldPointer fieldPointer, + @Nullable ExtractionFn extractionFn + ) + { + throw DruidException.defensive("Cannot call makeDimensionSelector on field of type ARRAY"); + } + + @Override + public boolean isNull(Memory memory, long position) + { + final byte firstByte = memory.getByte(position); + return firstByte == NumericArrayFieldWriter.NULL_ROW; + } + + @Override + public boolean isComparable() + { + return true; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.java new file mode 100644 index 000000000000..1871aef06e08 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldSelector.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.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.error.DruidException; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; +import java.util.ArrayList; +import java.util.List; + +/** + * Base implementation of the column value selector that the concrete numeric field reader implementations inherit from. + * The selector contains the logic to construct an array written by {@link NumericArrayFieldWriter}, and present it as + * a column value selector. + * + * The inheritors of this class are expected to implement + * 1. {@link #getIndividualValueAtMemory} Which extracts the element from the field where it was written to. Returns + * null if the value at that location represents a null element + * 2. {@link #getIndividualFieldSize} Which informs the method about the field size corresponding to each element in + * the numeric array's serialized representation + * + * @param Type of the individual array elements + */ +public abstract class NumericArrayFieldSelector implements ColumnValueSelector +{ + /** + * Memory containing the serialized values of the array + */ + protected final Memory memory; + + /** + * Pointer to location in the memory. The callers are expected to update the pointer's position to the start of the + * array that they wish to get prior to {@link #getObject()} call. + * + * Frames read and written using {@link org.apache.druid.frame.write.FrameWriter} and + * {@link org.apache.druid.frame.read.FrameReader} shouldn't worry about this detail, since they automatically update + * and handle the start location + */ + private final ReadableFieldPointer fieldPointer; + + /** + * Position last read, for caching the last fetched result + */ + private long currentFieldPosition = -1; + + /** + * Value of the row at the location beginning at {@link #currentFieldPosition} + */ + private final List currentRow = new ArrayList<>(); + + /** + * Nullity of the row at the location beginning at {@link #currentFieldPosition} + */ + private boolean currentRowIsNull; + + public NumericArrayFieldSelector(final Memory memory, final ReadableFieldPointer fieldPointer) + { + this.memory = memory; + this.fieldPointer = fieldPointer; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + // Do nothing + } + + @Nullable + @Override + public Object getObject() + { + final List currentArray = computeCurrentArray(); + + if (currentArray == null) { + return null; + } + + return currentArray.toArray(); + } + + @Override + public Class classOfObject() + { + return Object.class; + } + + @Override + public double getDouble() + { + return 0; + } + + @Override + public float getFloat() + { + return 0; + } + + @Override + public long getLong() + { + return 0; + } + + @Override + public boolean isNull() + { + long position = fieldPointer.position(); + final byte firstByte = memory.getByte(position); + return firstByte == NumericArrayFieldWriter.NULL_ROW; + } + + /** + * Returns the value of the individual element written at the given position + */ + @Nullable + public abstract ElementType getIndividualValueAtMemory(long position); + + /** + * Returns the field size that each element in the reader array consumes. It is usually 1 + ElementType.SIZE, to hold + * the element's nullity, and it's representation. + */ + public abstract int getIndividualFieldSize(); + + @Nullable + private List computeCurrentArray() + { + final long fieldPosition = fieldPointer.position(); + + if (fieldPosition != currentFieldPosition) { + updateCurrentArray(fieldPosition); + } + + this.currentFieldPosition = fieldPosition; + + if (currentRowIsNull) { + return null; + } + return currentRow; + + } + + private void updateCurrentArray(final long fieldPosition) + { + currentRow.clear(); + currentRowIsNull = false; + + long position = fieldPosition; + long limit = memory.getCapacity(); + + // Check the first byte, and if it is null, update the current value to null and return + if (isNull()) { + currentRowIsNull = true; + return; + } + + // Adding a check here to prevent the position from potentially overflowing + if (position < limit) { + position++; + } + + // Sanity check, to make sure that we see the rowTerminator at the end + boolean rowTerminatorSeen = false; + + while (position < limit) { + final byte kind = memory.getByte(position); + + // Break as soon as we see the ARRAY_TERMINATOR (0x00) + if (kind == NumericArrayFieldWriter.ARRAY_TERMINATOR) { + rowTerminatorSeen = true; + break; + } + + // If terminator not seen, then read the field at that location, and increment the position by the element's field + // size to read the next element. + currentRow.add(getIndividualValueAtMemory(position)); + position += getIndividualFieldSize(); + } + + if (!rowTerminatorSeen) { + throw DruidException.defensive("Unexpected end of field"); + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldWriter.java new file mode 100644 index 000000000000..e220d6bdc519 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericArrayFieldWriter.java @@ -0,0 +1,249 @@ +/* + * 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.frame.field; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.common.config.NullHandling; +import org.apache.druid.frame.write.FrameWriterUtils; +import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector; +import org.apache.druid.segment.ColumnValueSelector; + +import javax.annotation.Nullable; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Writes the values of the type ARRAY where X is a numeric type to row based frames. + * The format of the array written is as follows: + *

+ * Format: + * - 1 Byte - {@link #NULL_ROW} or {@link #NON_NULL_ROW} denoting whether the array itself is null + * - If the array is null, then the writer stops here + * - If the array is not null, then it proceeds to the following steps + *

+ * For each value in the non-null array: + * - 1 Byte - {@link NumericFieldWriter#ARRAY_ELEMENT_NULL_BYTE} or {@link NumericFieldWriter#ARRAY_ELEMENT_NOT_NULL_BYTE} + * denothing whether the proceeding value is null or not. + * - ElementSize Bytes - The encoded value of the element + *

+ * Once all the values in the non-null arrays are over, writes {@link #ARRAY_TERMINATOR}. This is to aid the byte + * comparison, and also let the reader know that the number of elements in the array are over. + *

+ * The format doesn't add the number of elements in the array at the beginning, so that the serialization of the arrays + * are byte-by-byte comparable. + *

+ * Examples: + * 1. null + * | Bytes | Value | Interpretation | + * |--------|-------|-----------------------------| + * | 1 | 0x00 | Denotes that the array null | + *

+ * 2. [] (empty array) + * | Bytes | Value | Interpretation | + * |--------|----- -|------------------------------------| + * | 1 | 0x01 | Denotes that the array is not null | + * | 2 | 0x00 | End of the array | + *

+ * 3. [5L, null, 6L] + * | Bytes | Value | Interpretation | + * |---------|--------------|-----------------------------------------------------------------------------------| + * | 1 | 0x01 | Denotes that the array is not null | + * | 2 | 0x02 | Denotes that the next element is not null | + * | 3-10 | transform(5) | Representation of 5 | + * | 11 | 0x01 | Denotes that the next element is null | + * | 12-19 | transform(0) | Representation of 0 (default value, the reader will ignore it if SqlCompatible mode is on | + * | 20 | 0x02 | Denotes that the next element is not null | + * | 21-28 | transform(6) | Representation of 6 | + * | 29 | 0x00 | End of array | + */ +public class NumericArrayFieldWriter implements FieldWriter +{ + + /** + * Denotes that the array itself is null + */ + public static final byte NULL_ROW = 0x00; + + /** + * Denotes that the array is non null + */ + public static final byte NON_NULL_ROW = 0x01; + + /** + * Marks the end of the array. Since {@link #NULL_ROW} and {@link #ARRAY_TERMINATOR} will only occur at different + * locations, therefore there is no clash in keeping both's values at 0x00 + */ + public static final byte ARRAY_TERMINATOR = 0x00; + + private final ColumnValueSelector selector; + private final NumericFieldWriterFactory writerFactory; + + /** + * Returns the writer for ARRAY + */ + public static NumericArrayFieldWriter getLongArrayFieldWriter(final ColumnValueSelector selector) + { + return new NumericArrayFieldWriter(selector, LongFieldWriter::forArray); + } + + /** + * Returns the writer for ARRAY + */ + public static NumericArrayFieldWriter getFloatArrayFieldWriter(final ColumnValueSelector selector) + { + return new NumericArrayFieldWriter(selector, FloatFieldWriter::forArray); + } + + /** + * Returns the writer for ARRAY + */ + public static NumericArrayFieldWriter getDoubleArrayFieldWriter(final ColumnValueSelector selector) + { + return new NumericArrayFieldWriter(selector, DoubleFieldWriter::forArray); + } + + public NumericArrayFieldWriter(final ColumnValueSelector selector, NumericFieldWriterFactory writerFactory) + { + this.selector = selector; + this.writerFactory = writerFactory; + } + + @Override + public long writeTo(WritableMemory memory, long position, long maxSize) + { + Object row = selector.getObject(); + if (row == null) { + int requiredSize = Byte.BYTES; + if (requiredSize > maxSize) { + return -1; + } + memory.putByte(position, NULL_ROW); + return requiredSize; + } else { + + List list = FrameWriterUtils.getNumericArrayFromObject(row); + + if (list == null) { + int requiredSize = Byte.BYTES; + if (requiredSize > maxSize) { + return -1; + } + memory.putByte(position, NULL_ROW); + return requiredSize; + } + + // Create a columnValueSelector to write the individual elements re-using the NumericFieldWriter + AtomicInteger index = new AtomicInteger(0); + ColumnValueSelector columnValueSelector = new ColumnValueSelector() + { + @Override + public double getDouble() + { + final Number n = getObject(); + assert NullHandling.replaceWithDefault() || n != null; + return n != null ? n.doubleValue() : 0d; + } + + @Override + public float getFloat() + { + final Number n = getObject(); + assert NullHandling.replaceWithDefault() || n != null; + return n != null ? n.floatValue() : 0f; + } + + @Override + public long getLong() + { + final Number n = getObject(); + assert NullHandling.replaceWithDefault() || n != null; + return n != null ? n.longValue() : 0L; + } + + @Override + public void inspectRuntimeShape(RuntimeShapeInspector inspector) + { + + } + + @Override + public boolean isNull() + { + // Arrays preserve the individual element's nullity when they are written and read. + // Therefore, when working with SQL incompatible mode, [7, null] won't change to [7, 0] when written to and + // read from the underlying serialization (as compared with the primitives). Therefore, + // even when NullHandling.replaceWithDefault() is true we need to write null as is, and not convert it to their + // default value when writing the array. Therefore, the check is `getObject() == null` ignoring the value of + // `NullHandling.replaceWithDefaul()`. + return getObject() == null; + } + + @Nullable + @Override + public Number getObject() + { + return list.get(index.get()); + } + + @Override + public Class classOfObject() + { + return Number.class; + } + }; + + NumericFieldWriter writer = writerFactory.get(columnValueSelector); + + // First byte is reserved for null marker of the array + // Next [(1 + Numeric Size) x Number of elements of array] bytes are reserved for the elements of the array and + // their null markers + // Last byte is reserved for array termination + int requiredSize = Byte.BYTES + (writer.getNumericSizeBytes() + Byte.BYTES) * list.size() + Byte.BYTES; + + if (requiredSize > maxSize) { + return -1; + } + + long offset = 0; + memory.putByte(position + offset, NON_NULL_ROW); + offset += Byte.BYTES; + + for (; index.get() < list.size(); index.incrementAndGet()) { + writer.writeTo( + memory, + position + offset, + maxSize - offset + ); + offset += Byte.BYTES + writer.getNumericSizeBytes(); + } + + memory.putByte(position + offset, ARRAY_TERMINATOR); + + return requiredSize; + + } + } + + @Override + public void close() + { + // Do nothing + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java new file mode 100644 index 000000000000..cf83c405d092 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldReader.java @@ -0,0 +1,131 @@ +/* + * 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.frame.field; + +import org.apache.datasketches.memory.Memory; +import org.apache.druid.query.extraction.ExtractionFn; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.segment.DimensionSelector; +import org.apache.druid.segment.column.ValueType; +import org.apache.druid.segment.column.ValueTypes; + +import javax.annotation.Nullable; + +/** + * Reads the fields created by the {@link NumericFieldWriter}. See the Javadoc for the writer for format details + * + * @see NumericFieldWriter + */ +public abstract class NumericFieldReader implements FieldReader +{ + + /** + * The indicator byte which denotes that the following value is null. + */ + private final byte nullIndicatorByte; + + public NumericFieldReader(boolean forArray) + { + if (!forArray) { + this.nullIndicatorByte = NumericFieldWriter.NULL_BYTE; + } else { + this.nullIndicatorByte = NumericFieldWriter.ARRAY_ELEMENT_NULL_BYTE; + } + } + + @Override + public ColumnValueSelector makeColumnValueSelector(Memory memory, ReadableFieldPointer fieldPointer) + { + return getColumnValueSelector(memory, fieldPointer, nullIndicatorByte); + } + + @Override + public DimensionSelector makeDimensionSelector( + Memory memory, + ReadableFieldPointer fieldPointer, + @Nullable ExtractionFn extractionFn + ) + { + return ValueTypes.makeNumericWrappingDimensionSelector( + getValueType(), + makeColumnValueSelector(memory, fieldPointer), + extractionFn + ); + } + + @Override + public boolean isNull(Memory memory, long position) + { + return memory.getByte(position) == nullIndicatorByte; + } + + + @Override + public boolean isComparable() + { + return true; + } + + /** + * Creates a column value selector for the element written at fieldPointer's position in the memory. + * The nullilty check is handled by the nullIndicatorByte + */ + public abstract ColumnValueSelector getColumnValueSelector( + Memory memory, + ReadableFieldPointer fieldPointer, + byte nullIndicatorByte + ); + + /** + * {@link ValueType} of the inheritor's element + */ + public abstract ValueType getValueType(); + + /** + * Helper class which allows the inheritors to fetch the nullity of the field located at fieldPointer's position in + * the dataRegion. + * + * The implementations of the column value selectors returned by the {@link #getColumnValueSelector} can inherit this + * class and call {@link #_isNull()} in their {@link ColumnValueSelector#isNull()} to offload the responsibility of + * detecting null elements to this Selector, instead of reworking the null handling + */ + public abstract static class Selector + { + private final Memory dataRegion; + private final ReadableFieldPointer fieldPointer; + private final byte nullIndicatorByte; + + public Selector( + final Memory dataRegion, + final ReadableFieldPointer fieldPointer, + final byte nullIndicatorByte + ) + { + this.dataRegion = dataRegion; + this.fieldPointer = fieldPointer; + this.nullIndicatorByte = nullIndicatorByte; + } + + protected boolean _isNull() + { + return dataRegion.getByte(fieldPointer.position()) == nullIndicatorByte; + } + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java new file mode 100644 index 000000000000..7aa604480c91 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriter.java @@ -0,0 +1,139 @@ +/* + * 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.frame.field; + +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.segment.BaseNullableColumnValueSelector; + +/** + * FieldWriter for numeric datatypes. The parent class does the null handling for the underlying data, while + * the individual subclasses write the individual element (long, float or double type). This also allows for a clean + * reuse of the readers and writers between the numeric types and also allowing the array writers ({@link NumericArrayFieldWriter}) + * to use these methods directly without duplication + * + * Format: + * - 1 byte: Whether the following value is null or not. Take a look at the note on the indicator bytes. + * - X bytes: Encoded value of the selector, or the default value if it is null. X denotes the size of the numeric value + * + * Indicator bytes for denoting whether the element is null or not null changes depending on whether the writer is used + * to write the data for individual value (like LONG) or for an element of an array (like ARRAY). This is because + * array support for the numeric types was added later and by then the field writers for individual fields were using + * 0x00 to denote the null byte, which is reserved for denoting the array end when we are writing the elements as part + * of the array instead. (0x00 is used for array end because it helps in preserving the byte comparison property of the + * numeric array field writers). + * + * Therefore, to preserve backward and forward compatibility, the individual element's writers were left unchanged, + * while the array's element's writers used 0x01 and 0x02 to denote null and non-null byte respectively + * + * Values produced by the writer are sortable without decoding + * + * @see NumericArrayFieldWriter for examples of how this class serializes the field for numeric arrays + */ +public abstract class NumericFieldWriter implements FieldWriter +{ + /** + * Indicator byte denoting that the numeric value succeeding it is null. This is used in the primitive + * writers. NULL_BYTE < NOT_NULL_BYTE to preserve the ordering while doing byte comparison + */ + public static final byte NULL_BYTE = 0x00; + + /** + * Indicator byte denoting that the numeric value succeeding it is not null. This is used in the primitive + * writers + */ + public static final byte NOT_NULL_BYTE = 0x01; + + /** + * Indicator byte denoting that the numeric value succeeding it is null. This is used while writing the individual + * elements writers of an array. ARRAY_ELEMENT_NULL_BYTE < ARRAY_ELEMENT_NOT_NULL_BYTE to preserve the ordering + * while doing byte comparison + */ + public static final byte ARRAY_ELEMENT_NULL_BYTE = 0x01; + + /** + * Indicator byte denoting that the numeric value succeeding it is not null. This is used while writing the individual + * elements writers of an array + */ + public static final byte ARRAY_ELEMENT_NOT_NULL_BYTE = 0x02; + + private final BaseNullableColumnValueSelector selector; + private final byte nullIndicatorByte; + private final byte notNullIndicatorByte; + + public NumericFieldWriter( + final BaseNullableColumnValueSelector selector, + final boolean forArray + ) + { + this.selector = selector; + if (!forArray) { + this.nullIndicatorByte = NULL_BYTE; + this.notNullIndicatorByte = NOT_NULL_BYTE; + } else { + this.nullIndicatorByte = ARRAY_ELEMENT_NULL_BYTE; + this.notNullIndicatorByte = ARRAY_ELEMENT_NOT_NULL_BYTE; + } + } + + @Override + public long writeTo(WritableMemory memory, long position, long maxSize) + { + int size = getNumericSizeBytes() + Byte.BYTES; + + if (maxSize < size) { + return -1; + } + + // Using isNull() since this is a primitive type + if (selector.isNull()) { + memory.putByte(position, nullIndicatorByte); + writeNullToMemory(memory, position + Byte.BYTES); + } else { + memory.putByte(position, notNullIndicatorByte); + writeSelectorToMemory(memory, position + Byte.BYTES); + } + + return size; + } + + @Override + public void close() + { + // Nothing to do + } + + /** + * @return The size in bytes of the numeric datatype that the implementation of this writer occupies + */ + public abstract int getNumericSizeBytes(); + + /** + * Writes the value pointed by the selector to memory. The caller should ensure that the selector gives out the + * correct primitive type + */ + public abstract void writeSelectorToMemory(WritableMemory memory, long position); + + /** + * Writes the default value for the type to the memory. For long, it is 0L, for double, it is 0.0d etc. Useful mainly + * when the SQL incompatible mode is turned off, and maintains the fact that the size of the numeric field written + * doesn't vary irrespective of whether the value is null + */ + public abstract void writeNullToMemory(WritableMemory memory, long position); +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java new file mode 100644 index 000000000000..a7ae47c91f4e --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/NumericFieldWriterFactory.java @@ -0,0 +1,33 @@ +/* + * 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.frame.field; + +import org.apache.druid.segment.ColumnValueSelector; + +/** + * Factory for {@link NumericFieldWriter} + */ +public interface NumericFieldWriterFactory +{ + /** + * Constructs an instance of {@link NumericFieldWriter} given the column selector + */ + NumericFieldWriter get(ColumnValueSelector selector); +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.java new file mode 100644 index 000000000000..d26f84f251de --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/SettableFieldPointer.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.frame.field; + +/** + * A simple {@link ReadableFieldPointer} that returns the position that was set on its object. + */ +public class SettableFieldPointer implements ReadableFieldPointer +{ + + long position = 0; + + public void setPosition(long position) + { + this.position = position; + } + + @Override + public long position() + { + return position; + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java new file mode 100644 index 000000000000..67bed4cb051b --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/StringArrayFieldReader.java @@ -0,0 +1,34 @@ +/* + * 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.frame.field; + +/** + * Reads fields written by {@link StringArrayFieldWriter} + * + * @see StringFieldReader for more details on the format that the reader expects + * @see StringFieldReader#StringFieldReader(boolean) for the selector that the reader returns + */ +public class StringArrayFieldReader extends StringFieldReader +{ + StringArrayFieldReader() + { + super(true); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java index 0cce325d2123..6b15804d662a 100644 --- a/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java +++ b/processing/src/main/java/org/apache/druid/frame/field/StringFieldReader.java @@ -70,6 +70,11 @@ public class StringFieldReader implements FieldReader { private final boolean asArray; + public StringFieldReader() + { + this(false); + } + /** * Create a string reader. * @@ -77,7 +82,7 @@ public class StringFieldReader implements FieldReader * selectors (potentially multi-value ones). If true, selectors from {@link #makeColumnValueSelector} * behave like string array selectors. */ - StringFieldReader(final boolean asArray) + protected StringFieldReader(final boolean asArray) { this.asArray = asArray; } diff --git a/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java b/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java new file mode 100644 index 000000000000..f882f0443568 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/field/TransformUtils.java @@ -0,0 +1,86 @@ +/* + * 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.frame.field; + +/** + * Utility methods to map the primitive numeric types into an equi-wide byte representation, such that the + * given byte sequence preserves the ordering of the original type when done byte comparison. + * Checkout {@link org.apache.druid.frame.read.FrameReaderUtils#compareMemoryToByteArrayUnsigned} for how this byte + * comparison is performed. + */ +public class TransformUtils +{ + /** + * Transforms a double into a form where it can be compared as unsigned bytes without decoding. + */ + public static long transformFromDouble(final double n) + { + final long bits = Double.doubleToLongBits(n); + final long mask = ((bits & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; + return Long.reverseBytes(bits ^ mask); + } + + /** + * Inverse of {@link #transformFromDouble}. + */ + public static double detransformToDouble(final long bits) + { + final long reversedBits = Long.reverseBytes(bits); + final long mask = (((reversedBits ^ Long.MIN_VALUE) & Long.MIN_VALUE) >> 11) | Long.MIN_VALUE; + return Double.longBitsToDouble(reversedBits ^ mask); + } + + /** + * Transforms a long into a form where it can be compared as unsigned bytes without decoding. + */ + public static long transformFromLong(final long n) + { + // Must flip the first (sign) bit so comparison-as-bytes works. + return Long.reverseBytes(n ^ Long.MIN_VALUE); + } + + /** + * Reverse the {@link #transformFromLong(long)} function. + */ + public static long detransformToLong(final long bits) + { + return Long.reverseBytes(bits) ^ Long.MIN_VALUE; + } + + /** + * Transforms a float into a form where it can be compared as unsigned bytes without decoding. + */ + public static int transformFromFloat(final float n) + { + final int bits = Float.floatToIntBits(n); + final int mask = ((bits & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; + return Integer.reverseBytes(bits ^ mask); + } + + /** + * Inverse of {@link #transformFromFloat(float)}. + */ + public static float detransformToFloat(final int bits) + { + final int reversedBits = Integer.reverseBytes(bits); + final int mask = (((reversedBits ^ Integer.MIN_VALUE) & Integer.MIN_VALUE) >> 8) | Integer.MIN_VALUE; + return Float.intBitsToFloat(reversedBits ^ mask); + } +} 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 e5cc20746b49..8ddf99325d39 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 @@ -74,6 +74,7 @@ private FrameReader( * If the columnType is null, we store the data as {@link ColumnType#NESTED_DATA}. This can be done if we know that * the data that we receive can be serded generically using the nested data. It is currently used in the brokers to * store the data with unknown types into frames. + * * @param signature signature used to generate the reader */ public static FrameReader create(final RowSignature signature) @@ -96,8 +97,8 @@ public static FrameReader create(final RowSignature signature) signature.getColumnName(columnNumber) ); - columnReaders.add(FrameColumnReaders.create(columnNumber, columnType)); fieldReaders.add(FieldReaders.create(signature.getColumnName(columnNumber), columnType)); + columnReaders.add(FrameColumnReaders.create(signature.getColumnName(columnNumber), columnNumber, columnType)); } return new FrameReader(signature, columnReaders, fieldReaders); @@ -110,7 +111,7 @@ public RowSignature signature() /** * Returns capabilities for a particular column in a particular frame. - * + *

* Preferred over {@link RowSignature#getColumnCapabilities(String)} when reading a particular frame, because this * method has more insight into what's actually going on with that specific frame (nulls, multivalue, etc). The * RowSignature version is based solely on type. @@ -151,7 +152,7 @@ public CursorFactory makeCursorFactory(final Frame frame) /** * Create a {@link FrameComparisonWidget} for the given frame. - * + *

* Only possible for frames of type {@link org.apache.druid.frame.FrameType#ROW_BASED}. The provided * sortColumns must be a prefix of {@link #signature()}. */ diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java index 4caaeaae87b6..98218819ce13 100644 --- a/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/FrameColumnReaders.java @@ -19,10 +19,15 @@ package org.apache.druid.frame.read.columnar; -import org.apache.druid.java.util.common.UOE; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.ValueType; +/** + * Creates {@link FrameColumnReader} corresponding to a given column type and number. + * + * Returns a dummy {@link UnsupportedColumnTypeFrameColumnReader} if the column type is not supported or unknown. + * Calling any method of the dummy reader will throw with relevant error message. + */ public class FrameColumnReaders { private FrameColumnReaders() @@ -30,7 +35,11 @@ private FrameColumnReaders() // No instantiation. } - public static FrameColumnReader create(final int columnNumber, final ColumnType columnType) + public static FrameColumnReader create( + final String columnName, + final int columnNumber, + final ColumnType columnType + ) { switch (columnType.getType()) { case LONG: @@ -51,11 +60,12 @@ public static FrameColumnReader create(final int columnNumber, final ColumnType case ARRAY: if (columnType.getElementType().getType() == ValueType.STRING) { return new StringFrameColumnReader(columnNumber, true); + } else { + return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType); } - // Fall through to error for other array types default: - throw new UOE("Unsupported column type [%s]", columnType); + return new UnsupportedColumnTypeFrameColumnReader(columnName, columnType); } } } diff --git a/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java b/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java new file mode 100644 index 000000000000..06f356128247 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/frame/read/columnar/UnsupportedColumnTypeFrameColumnReader.java @@ -0,0 +1,57 @@ +/* + * 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.frame.read.columnar; + +import org.apache.druid.frame.Frame; +import org.apache.druid.frame.write.UnsupportedColumnTypeException; +import org.apache.druid.query.rowsandcols.column.Column; +import org.apache.druid.segment.column.ColumnType; + +import javax.annotation.Nullable; + +/** + * Dummy reader for unsupported types. Throws {@link UnsupportedColumnTypeException} if we try to call any method of the + * frame reader + */ +public class UnsupportedColumnTypeFrameColumnReader implements FrameColumnReader +{ + + private final String columnName; + @Nullable + private final ColumnType columnType; + + UnsupportedColumnTypeFrameColumnReader(String columnName, @Nullable ColumnType columnType) + { + this.columnName = columnName; + this.columnType = columnType; + } + + @Override + public Column readRACColumn(Frame frame) + { + throw new UnsupportedColumnTypeException(columnName, columnType); + } + + @Override + public ColumnPlus readColumn(Frame frame) + { + throw new UnsupportedColumnTypeException(columnName, columnType); + } +} diff --git a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java index 577443cedd92..ac9574d7f052 100644 --- a/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java +++ b/processing/src/main/java/org/apache/druid/frame/write/FrameWriterUtils.java @@ -33,6 +33,8 @@ import org.apache.druid.segment.DimensionSelector; import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; +import org.apache.druid.segment.data.ComparableIntArray; +import org.apache.druid.segment.data.ComparableList; import org.apache.druid.segment.data.ComparableStringArray; import org.apache.druid.segment.data.IndexedInts; @@ -132,7 +134,7 @@ public static List getUtf8ByteBuffersFromStringSelector( * Retrieves UTF-8 byte buffers from a {@link ColumnValueSelector}, which is expected to be the kind of * selector you get for an {@code ARRAY} column. * - * Null strings are returned as {@link #NULL_STRING_MARKER_ARRAY}. + * Null strings are returned as {@code null}. * * If the entire array returned by {@link BaseObjectColumnValueSelector#getObject()} is null, returns either * null or {@link #NULL_STRING_MARKER_ARRAY} depending on the value of "useNullArrays". @@ -172,6 +174,48 @@ public static List getUtf8ByteBuffersFromStringArraySelector( return retVal; } + /** + * Retrieves a numeric list from a Java object, given that the object is an instance of something that can be returned + * from {@link ColumnValueSelector#getObject()} of valid numeric array selectors representations + * + * While {@link BaseObjectColumnValueSelector} specifies that only instances of {@code Object[]} can be returned from + * the numeric array selectors, this method also handles a few more cases which can be encountered if the selector is + * directly implemented on top of the group by stuff + */ + @Nullable + public static List getNumericArrayFromObject(Object row) + { + if (row == null) { + return null; + } else if (row instanceof Number) { + return Collections.singletonList((Number) row); + } + + final List retVal = new ArrayList<>(); + + if (row instanceof List) { + for (int i = 0; i < ((List) row).size(); i++) { + retVal.add((Number) ((List) row).get(i)); + } + } else if (row instanceof Object[]) { + for (Object value : (Object[]) row) { + retVal.add((Number) value); + } + } else if (row instanceof ComparableList) { + for (Object value : ((ComparableList) row).getDelegate()) { + retVal.add((Number) value); + } + } else if (row instanceof ComparableIntArray) { + for (int value : ((ComparableIntArray) row).getDelegate()) { + retVal.add(value); + } + } else { + throw new ISE("Unexpected type %s found", row.getClass().getName()); + } + + return retVal; + } + /** * Checks the provided signature for any disallowed field names. Returns any that are found. */ diff --git a/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java b/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java index 604ef987394e..d79c517ae3d1 100644 --- a/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java +++ b/processing/src/main/java/org/apache/druid/java/util/common/DefineClassUtils.java @@ -95,7 +95,7 @@ private static MethodHandle defineClassJava9(MethodHandles.Lookup lookup) throws } /** - * "Compile" a MethodHandle that is equilavent to: + * "Compile" a MethodHandle that is equivalent to: * * Class defineClass(Class targetClass, byte[] byteCode, String className) { * return Unsafe.defineClass( @@ -147,7 +147,7 @@ private static MethodHandle defineClassJava8(MethodHandles.Lookup lookup) throws // defineClass(className, byteCode, 0, length, targetClass) defineClass = MethodHandles.insertArguments(defineClass, 2, (int) 0); - // JDK8 does not implement MethodHandles.arrayLength so we have to roll our own + // JDK8 does not implement MethodHandles.arrayLength, so we have to roll our own MethodHandle arrayLength = lookup.findStatic( lookup.lookupClass(), "getArrayLength", @@ -171,6 +171,16 @@ private static MethodHandle defineClassJava8(MethodHandles.Lookup lookup) throws return defineClass; } + /** + * This method is referenced in Java 8 using method handle, therefore it is not actually unused, and shouldn't be + * removed (till Java 8 is supported) + */ + @SuppressWarnings("unused") // method is referenced and used in defineClassJava8 + static int getArrayLength(byte[] bytes) + { + return bytes.length; + } + public static Class defineClass( Class targetClass, byte[] byteCode, diff --git a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java index c5bb271213e8..b4d06edc77cf 100644 --- a/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java +++ b/processing/src/main/java/org/apache/druid/query/IterableRowsCursorHelper.java @@ -20,18 +20,18 @@ package org.apache.druid.query; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.segment.Cursor; import org.apache.druid.segment.RowAdapter; import org.apache.druid.segment.RowBasedCursor; import org.apache.druid.segment.RowWalker; import org.apache.druid.segment.VirtualColumns; import org.apache.druid.segment.column.RowSignature; -import java.util.Iterator; +import java.io.Closeable; /** * Helper methods to create cursor from iterable of rows @@ -43,7 +43,18 @@ public class IterableRowsCursorHelper * Creates a cursor that iterates over all the rows generated by the iterable. Presence of __time column is not a * necessity */ - public static RowBasedCursor getCursorFromIterable(Iterable rows, RowSignature rowSignature) + public static Pair getCursorFromIterable(Iterable rows, RowSignature rowSignature) + { + return getCursorFromSequence(Sequences.simple(rows), rowSignature); + } + + /** + * Creates a cursor that iterates over all the rows generated by the sequence. Presence of __time column is not a + * necessity. + *

+ * Returns a pair of cursor that iterates over the rows and closeable that cleans up the created rowWalker + */ + public static Pair getCursorFromSequence(Sequence rows, RowSignature rowSignature) { RowAdapter rowAdapter = columnName -> { if (rowSignature == null) { @@ -55,8 +66,10 @@ public static RowBasedCursor getCursorFromIterable(Iterable } return row -> row[columnIndex]; }; - RowWalker rowWalker = new RowWalker<>(Sequences.simple(rows), rowAdapter); - return new RowBasedCursor<>( + + RowWalker rowWalker = new RowWalker<>(rows, rowAdapter); + + Cursor baseCursor = new RowBasedCursor<>( rowWalker, rowAdapter, null, @@ -66,41 +79,7 @@ public static RowBasedCursor getCursorFromIterable(Iterable false, rowSignature != null ? rowSignature : RowSignature.empty() ); - } - /** - * Creates a cursor that iterates over all the rows generated by the sequence. Presence of __time column is not a - * necessity - */ - public static RowBasedCursor getCursorFromSequence(Sequence rows, RowSignature rowSignature) - { - return getCursorFromIterable( - new Iterable() - { - Yielder yielder = Yielders.each(rows); - - @Override - public Iterator iterator() - { - return new Iterator() - { - @Override - public boolean hasNext() - { - return !yielder.isDone(); - } - - @Override - public Object[] next() - { - Object[] retVal = yielder.get(); - yielder = yielder.next(null); - return retVal; - } - }; - } - }, - rowSignature - ); + return Pair.of(baseCursor, rowWalker); } } diff --git a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java index 73205d2b75fa..9c746dd41429 100644 --- a/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/groupby/GroupByQueryQueryToolChest.java @@ -43,6 +43,7 @@ import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.MappedSequence; import org.apache.druid.java.util.common.guava.Sequence; @@ -72,6 +73,7 @@ import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.BitSet; @@ -726,12 +728,14 @@ public Optional> resultsAsFrames( ); - Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence( + Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), rowSignature ); + Cursor cursor = cursorAndCloseable.lhs; + Closeable closeble = cursorAndCloseable.rhs; - Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory); + Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).withBaggage(closeble); return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature))); } diff --git a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java index 97237ef77a2c..0eefa288df2f 100644 --- a/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java +++ b/processing/src/main/java/org/apache/druid/query/rowsandcols/concrete/FrameRowsAndColumns.java @@ -70,7 +70,7 @@ public Column findColumn(String name) .getColumnType(columnIndex) .orElseThrow(() -> new ISE("just got the id, why is columnType not there?")); - colCache.put(name, FrameColumnReaders.create(columnIndex, columnType).readRACColumn(frame)); + colCache.put(name, FrameColumnReaders.create(name, columnIndex, columnType).readRACColumn(frame)); } } return colCache.get(name); diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java index b7253d70fe93..4d0885da00d8 100644 --- a/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanQueryQueryToolChest.java @@ -36,12 +36,12 @@ import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.UOE; import org.apache.druid.java.util.common.guava.BaseSequence; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; -import org.apache.druid.java.util.common.guava.Yielder; -import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.FrameSignaturePair; import org.apache.druid.query.GenericQueryMetricsFactory; import org.apache.druid.query.IterableRowsCursorHelper; @@ -57,6 +57,7 @@ import org.apache.druid.segment.column.RowSignature; import org.apache.druid.utils.CloseableUtils; +import java.io.Closeable; import java.util.ArrayList; import java.util.Iterator; import java.util.List; @@ -220,24 +221,7 @@ public Optional> resultsAsFrames( ) { final RowSignature defaultRowSignature = resultArraySignature(query); - Iterator resultSequenceIterator = new Iterator() - { - Yielder yielder = Yielders.each(resultSequence); - - @Override - public boolean hasNext() - { - return !yielder.isDone(); - } - - @Override - public ScanResultValue next() - { - ScanResultValue scanResultValue = yielder.get(); - yielder = yielder.next(null); - return scanResultValue; - } - }; + ScanResultValueIterator resultSequenceIterator = new ScanResultValueIterator(resultSequence); Iterable> retVal = () -> new Iterator>() { @@ -280,7 +264,7 @@ public Sequence next() ); } }; - return Optional.of(Sequences.concat(retVal)); + return Optional.of(Sequences.concat(retVal).withBaggage(resultSequenceIterator)); } private Sequence convertScanResultValuesToFrame( @@ -294,16 +278,22 @@ private Sequence convertScanResultValuesToFrame( Preconditions.checkNotNull(rowSignature, "'rowSignature' must be provided"); List cursors = new ArrayList<>(); + Closer closer = Closer.create(); for (ScanResultValue scanResultValue : batch) { final List rows = (List) scanResultValue.getEvents(); final Function mapper = getResultFormatMapper(query.getResultFormat(), rowSignature.getColumnNames()); final Iterable formattedRows = Lists.newArrayList(Iterables.transform(rows, (Function) mapper)); - cursors.add(IterableRowsCursorHelper.getCursorFromIterable( + Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromIterable( formattedRows, rowSignature - )); + ); + Cursor cursor = cursorAndCloseable.lhs; + Closeable closeable = cursorAndCloseable.rhs; + cursors.add(cursor); + // Cursors created from iterators don't have any resources, therefore this is mostly a defensive check + closer.register(closeable); } RowSignature modifiedRowSignature = useNestedForUnknownTypes @@ -323,7 +313,7 @@ private Sequence convertScanResultValuesToFrame( frameWriterFactory ); - return frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)); + return frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature)).withBaggage(closer); } @Override diff --git a/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueIterator.java b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueIterator.java new file mode 100644 index 000000000000..646c69eaf185 --- /dev/null +++ b/processing/src/main/java/org/apache/druid/query/scan/ScanResultValueIterator.java @@ -0,0 +1,62 @@ +/* + * 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.scan; + +import org.apache.druid.java.util.common.guava.Sequence; +import org.apache.druid.java.util.common.guava.Yielder; +import org.apache.druid.java.util.common.guava.Yielders; +import org.apache.druid.java.util.common.parsers.CloseableIterator; + +import java.io.IOException; + +/** + * Iterates over the scan result sequence and provides an interface to clean up the resources (if any) to close the + * underlying sequence. Similar to {@link Yielder}, once close is called on the iterator, the calls to the rest of the + * iterator's methods are undefined. + */ +public class ScanResultValueIterator implements CloseableIterator +{ + Yielder yielder; + + public ScanResultValueIterator(final Sequence resultSequence) + { + yielder = Yielders.each(resultSequence); + } + + @Override + public void close() throws IOException + { + yielder.close(); + } + + @Override + public boolean hasNext() + { + return !yielder.isDone(); + } + + @Override + public Object next() + { + ScanResultValue scanResultValue = yielder.get(); + yielder = yielder.next(null); + return scanResultValue; + } +} diff --git a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java index c16fe29c14de..cd8e553bf512 100644 --- a/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/timeseries/TimeseriesQueryQueryToolChest.java @@ -38,6 +38,7 @@ import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.DateTimes; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularities; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; @@ -65,6 +66,7 @@ import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; +import java.io.Closeable; import java.util.ArrayList; import java.util.Collections; import java.util.Comparator; @@ -474,10 +476,12 @@ public Optional> resultsAsFrames( ) { final RowSignature rowSignature = resultArraySignature(query); - final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence( + final Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), rowSignature ); + final Cursor cursor = cursorAndCloseable.lhs; + final Closeable closeable = cursorAndCloseable.rhs; RowSignature modifiedRowSignature = useNestedForUnknownTypes ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) @@ -489,7 +493,7 @@ public Optional> resultsAsFrames( new ArrayList<>() ); - Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory); + Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).withBaggage(closeable); // All frames are generated with the same signature therefore we can attach the row signature return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature))); diff --git a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java index 80ffb3e62974..87b50e0e4677 100644 --- a/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java +++ b/processing/src/main/java/org/apache/druid/query/topn/TopNQueryQueryToolChest.java @@ -35,6 +35,7 @@ import org.apache.druid.frame.write.FrameWriterUtils; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.ISE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.granularity.Granularity; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.java.util.common.guava.Sequences; @@ -64,6 +65,7 @@ import org.apache.druid.segment.column.RowSignature; import org.joda.time.DateTime; +import java.io.Closeable; import java.util.ArrayList; import java.util.Collections; import java.util.Iterator; @@ -558,10 +560,12 @@ public Optional> resultsAsFrames( ) { final RowSignature rowSignature = resultArraySignature(query); - final Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence( + final Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromSequence( resultsAsArrays(query, resultSequence), rowSignature ); + Cursor cursor = cursorAndCloseable.lhs; + Closeable closeable = cursorAndCloseable.rhs; RowSignature modifiedRowSignature = useNestedForUnknownTypes ? FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature) @@ -573,7 +577,7 @@ public Optional> resultsAsFrames( new ArrayList<>() ); - Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory); + Sequence frames = FrameCursorUtils.cursorToFrames(cursor, frameWriterFactory).withBaggage(closeable); return Optional.of(frames.map(frame -> new FrameSignaturePair(frame, modifiedRowSignature))); } diff --git a/processing/src/main/java/org/apache/druid/segment/RowWalker.java b/processing/src/main/java/org/apache/druid/segment/RowWalker.java index d6241f197e85..f55245b3bca3 100644 --- a/processing/src/main/java/org/apache/druid/segment/RowWalker.java +++ b/processing/src/main/java/org/apache/druid/segment/RowWalker.java @@ -26,14 +26,19 @@ import org.joda.time.DateTime; import javax.annotation.Nullable; +import java.io.Closeable; import java.io.IOException; import java.util.function.ToLongFunction; /** * Used by {@link RowBasedStorageAdapter} and {@link RowBasedCursor} to walk through rows. It allows multiple * {@link RowBasedCursor} to share the same underlying Iterable. + * + * The class creates a yielder from the sequence to iterate over the rows. However, it doesn't call the sequence's close + * after iterating over it. {@link #close()} should be called by the instantiators of the class to clear the resources + * held by the {@link #rowSequence} and the corresponding yielder created to iterate over it. */ -public class RowWalker +public class RowWalker implements Closeable { private final Sequence rowSequence; private final ToLongFunction timestampFunction; @@ -86,6 +91,7 @@ public void skipToDateTime(final DateTime timestamp, final boolean descending) } } + @Override public void close() { if (rowYielder != null) { diff --git a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java index 3eea9193cc37..49793fba4163 100644 --- a/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java +++ b/processing/src/main/java/org/apache/druid/segment/column/ColumnTypeFactory.java @@ -57,6 +57,8 @@ public static ColumnType ofType(TypeSignature type) switch (type.getElementType().getType()) { case LONG: return ColumnType.LONG_ARRAY; + case FLOAT: + return ColumnType.FLOAT_ARRAY; case DOUBLE: return ColumnType.DOUBLE_ARRAY; case STRING: diff --git a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java index a7de99905c39..3e2b46d5f681 100644 --- a/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java +++ b/processing/src/main/java/org/apache/druid/segment/join/table/FrameBasedIndexedTable.java @@ -298,7 +298,7 @@ private List createColumnReaders(RowSignature rowSignature) "Type for column [%s]", rowSignature.getColumnName(columnNumber) ); - columnReaders.add(FrameColumnReaders.create(columnNumber, columnType)); + columnReaders.add(FrameColumnReaders.create(rowSignature.getColumnName(columnNumber), columnNumber, columnType)); } return columnReaders; 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 a4fd0907066b..4e46a1a529a0 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 @@ -24,7 +24,6 @@ import org.apache.druid.error.DruidException; import org.apache.druid.java.util.common.ByteBufferUtils; import org.apache.druid.java.util.common.FileUtils; -import org.apache.druid.java.util.common.ISE; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.smoosh.FileSmoosher; import org.apache.druid.java.util.common.io.smoosh.SmooshedFileMapper; @@ -99,7 +98,7 @@ public int lookupString(@Nullable String value) // for strings because of this. if other type dictionary writers could potentially use multiple internal files // in the future, we should transition them to using this approach as well (or build a combination smoosher and // mapper so that we can have a mutable smoosh) - File stringSmoosh = FileUtils.createTempDir(name + "__stringTempSmoosh"); + File stringSmoosh = FileUtils.createTempDir(StringUtils.urlEncode(name) + "__stringTempSmoosh"); final String fileName = NestedCommonFormatColumnSerializer.getInternalFileName( name, NestedCommonFormatColumnSerializer.STRING_DICTIONARY_FILE_NAME @@ -127,7 +126,7 @@ public int lookupString(@Nullable String value) final byte[] bytes = StringUtils.toUtf8Nullable(value); final int index = stringDictionary.indexOf(bytes == null ? null : ByteBuffer.wrap(bytes)); if (index < 0) { - throw DruidException.defensive("Value not found in string dictionary"); + throw DruidException.defensive("Value not found in column[%s] string dictionary", name); } return index; } @@ -135,7 +134,7 @@ public int lookupString(@Nullable String value) public int lookupLong(@Nullable Long value) { if (longDictionary == null) { - Path longFile = makeTempFile(name + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME); + final Path longFile = makeTempFile(name + NestedCommonFormatColumnSerializer.LONG_DICTIONARY_FILE_NAME); longBuffer = mapWriter(longFile, longDictionaryWriter); longDictionary = FixedIndexed.read(longBuffer, TypeStrategies.LONG, ByteOrder.nativeOrder(), Long.BYTES).get(); // reset position @@ -143,7 +142,7 @@ public int lookupLong(@Nullable Long value) } final int index = longDictionary.indexOf(value); if (index < 0) { - throw DruidException.defensive("Value not found in long dictionary"); + throw DruidException.defensive("Value not found in column[%s] long dictionary", name); } return index + longOffset(); } @@ -151,15 +150,20 @@ public int lookupLong(@Nullable Long value) public int lookupDouble(@Nullable Double value) { if (doubleDictionary == null) { - Path doubleFile = makeTempFile(name + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME); + final Path doubleFile = makeTempFile(name + NestedCommonFormatColumnSerializer.DOUBLE_DICTIONARY_FILE_NAME); doubleBuffer = mapWriter(doubleFile, doubleDictionaryWriter); - doubleDictionary = FixedIndexed.read(doubleBuffer, TypeStrategies.DOUBLE, ByteOrder.nativeOrder(), Double.BYTES).get(); + doubleDictionary = FixedIndexed.read( + doubleBuffer, + TypeStrategies.DOUBLE, + ByteOrder.nativeOrder(), + Double.BYTES + ).get(); // reset position doubleBuffer.position(0); } final int index = doubleDictionary.indexOf(value); if (index < 0) { - throw DruidException.defensive("Value not found in double dictionary"); + throw DruidException.defensive("Value not found in column[%s] double dictionary", name); } return index + doubleOffset(); } @@ -167,7 +171,7 @@ public int lookupDouble(@Nullable Double value) public int lookupArray(@Nullable int[] value) { if (arrayDictionary == null) { - Path arrayFile = makeTempFile(name + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME); + final Path arrayFile = makeTempFile(name + NestedCommonFormatColumnSerializer.ARRAY_DICTIONARY_FILE_NAME); arrayBuffer = mapWriter(arrayFile, arrayDictionaryWriter); arrayDictionary = FrontCodedIntArrayIndexed.read(arrayBuffer, ByteOrder.nativeOrder()).get(); // reset position @@ -175,7 +179,7 @@ public int lookupArray(@Nullable int[] value) } final int index = arrayDictionary.indexOf(value); if (index < 0) { - throw DruidException.defensive("Value not found in array dictionary"); + throw DruidException.defensive("Value not found in column[%s] array dictionary", name); } return index + arrayOffset(); } @@ -239,7 +243,7 @@ private int arrayOffset() private Path makeTempFile(String name) { try { - return Files.createTempFile(name, ".tmp"); + return Files.createTempFile(StringUtils.urlEncode(name), null); } catch (IOException e) { throw new RuntimeException(e); @@ -315,7 +319,11 @@ public long write(ByteBuffer[] srcs) throws IOException public int addToOffset(long numBytesWritten) { if (numBytesWritten > bytesLeft()) { - throw new ISE("Wrote more bytes[%,d] than available[%,d]. Don't do that.", numBytesWritten, bytesLeft()); + throw DruidException.defensive( + "Wrote more bytes[%,d] than available[%,d]. Don't do that.", + numBytesWritten, + bytesLeft() + ); } currOffset += numBytesWritten; diff --git a/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java b/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java index c8b3ab31302e..15691cfc9c4c 100644 --- a/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java +++ b/processing/src/main/java/org/apache/druid/segment/nested/FieldTypeInfo.java @@ -151,35 +151,7 @@ public MutableTypeSet(byte types, boolean hasEmptyArray) public MutableTypeSet add(ColumnType type) { - switch (type.getType()) { - case STRING: - types |= STRING_MASK; - break; - case LONG: - types |= LONG_MASK; - break; - case DOUBLE: - types |= DOUBLE_MASK; - break; - case ARRAY: - Preconditions.checkNotNull(type.getElementType(), "ElementType must not be null"); - switch (type.getElementType().getType()) { - case STRING: - types |= STRING_ARRAY_MASK; - break; - case LONG: - types |= LONG_ARRAY_MASK; - break; - case DOUBLE: - types |= DOUBLE_ARRAY_MASK; - break; - default: - throw new ISE("Unsupported nested array type: [%s]", type.asTypeString()); - } - break; - default: - throw new ISE("Unsupported nested type: [%s]", type.asTypeString()); - } + types = FieldTypeInfo.add(types, type); return this; } @@ -207,7 +179,11 @@ public MutableTypeSet merge(byte other, boolean hasEmptyArray) @Nullable public ColumnType getSingleType() { - return FieldTypeInfo.getSingleType(types); + final ColumnType columnType = FieldTypeInfo.getSingleType(types); + if (hasEmptyArray && columnType != null && !columnType.isArray()) { + return null; + } + return columnType; } public boolean isEmpty() @@ -218,6 +194,10 @@ public boolean isEmpty() public byte getByteValue() { + final ColumnType singleType = FieldTypeInfo.getSingleType(types); + if (hasEmptyArray && singleType != null && !singleType.isArray()) { + return FieldTypeInfo.add(types, ColumnType.ofArray(singleType)); + } return types; } @@ -293,6 +273,40 @@ private static ColumnType getSingleType(byte types) } } + public static byte add(byte types, ColumnType type) + { + switch (type.getType()) { + case STRING: + types |= STRING_MASK; + break; + case LONG: + types |= LONG_MASK; + break; + case DOUBLE: + types |= DOUBLE_MASK; + break; + case ARRAY: + Preconditions.checkNotNull(type.getElementType(), "ElementType must not be null"); + switch (type.getElementType().getType()) { + case STRING: + types |= STRING_ARRAY_MASK; + break; + case LONG: + types |= LONG_ARRAY_MASK; + break; + case DOUBLE: + types |= DOUBLE_ARRAY_MASK; + break; + default: + throw new ISE("Unsupported nested array type: [%s]", type.asTypeString()); + } + break; + default: + throw new ISE("Unsupported nested type: [%s]", type.asTypeString()); + } + return types; + } + public static Set convertToSet(byte types) { final Set theTypes = Sets.newHashSetWithExpectedSize(4); diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java new file mode 100644 index 000000000000..6381138f62db --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleArrayFieldReaderTest.java @@ -0,0 +1,234 @@ +/* + * 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.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class DoubleArrayFieldReaderTest extends InitializedNullHandlingTest +{ + private static final long MEMORY_POSITION = 1; + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + public ColumnValueSelector writeSelector; + + private WritableMemory memory; + private FieldWriter fieldWriter; + + //CHECKSTYLE.OFF: Regexp + private static final Object[] DOUBLES_ARRAY_1 = new Object[]{ + Double.MAX_VALUE, + Double.MIN_VALUE, + null, + Double.POSITIVE_INFINITY, + Double.NEGATIVE_INFINITY, + Double.MIN_NORMAL, + null, + 0.0d, + 1.234234d, + Double.NaN, + -1.344234d, + 129123.123123, + -21312213.33, + null, + 1111.0, + 23.0, + null, + }; + + private static final Object[] DOUBLES_ARRAY_2 = new Object[]{ + null, + Double.MAX_VALUE, + 12.234234d, + -21.344234d, + Double.POSITIVE_INFINITY, + null, + Double.MIN_VALUE, + 129123.123123, + null, + 0.0d, + Double.MIN_NORMAL, + 1111.0, + Double.NaN, + Double.NEGATIVE_INFINITY, + null, + -2133.33, + 23.0, + null + }; + //CHECKSTYLE.ON: Regexp + + private static final List DOUBLES_LIST_1; + private static final List DOUBLES_LIST_2; + + static { + DOUBLES_LIST_1 = Arrays.stream(DOUBLES_ARRAY_1).map(val -> (Double) val).collect(Collectors.toList()); + DOUBLES_LIST_2 = Arrays.stream(DOUBLES_ARRAY_2).map(val -> (Double) val).collect(Collectors.toList()); + } + + @Before + public void setUp() + { + memory = WritableMemory.allocate(1000); + fieldWriter = NumericArrayFieldWriter.getDoubleArrayFieldWriter(writeSelector); + } + + @After + public void tearDown() + { + fieldWriter.close(); + } + + @Test + public void test_isNull_null() + { + writeToMemory(null, MEMORY_POSITION); + Assert.assertTrue(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_aValue() + { + writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); + Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + Assert.assertFalse(new DoubleArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_makeColumnValueSelector_null() + { + writeToMemory(null, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + Assert.assertTrue(readSelector.isNull()); + } + + @Test + public void test_makeColumnValueSelector_aValue() + { + writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(DOUBLES_LIST_1, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_multipleValues() + { + long sz = writeToMemory(DOUBLES_ARRAY_1, MEMORY_POSITION); + writeToMemory(DOUBLES_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); + + + final ColumnValueSelector readSelector = new DoubleArrayFieldReader().makeColumnValueSelector(memory, pointer); + + pointer.setPointer(0); + assertResults(DOUBLES_LIST_1, readSelector.getObject()); + + pointer.setPointer(1); + assertResults(DOUBLES_LIST_2, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.emptyList(), readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new DoubleArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.singletonList(null), readSelector.getObject()); + } + + private long writeToMemory(final Object value, final long initialPosition) + { + Mockito.when(writeSelector.getObject()).thenReturn(value); + + long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition); + if (bytesWritten < 0) { + throw new ISE("Could not write"); + } + return bytesWritten; + } + + private void assertResults(List expected, Object actual) + { + if (expected == null) { + Assert.assertNull(actual); + } + Assert.assertTrue(actual instanceof Object[]); + List actualList = new ArrayList<>(); + for (Object val : (Object[]) actual) { + actualList.add((Double) val); + } + + Assert.assertEquals(expected, actualList); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java index 2393a6a7c3c9..8736c73276cc 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/DoubleFieldReaderTest.java @@ -59,7 +59,7 @@ public class DoubleFieldReaderTest extends InitializedNullHandlingTest public void setUp() { memory = WritableMemory.allocate(1000); - fieldWriter = new DoubleFieldWriter(writeSelector); + fieldWriter = DoubleFieldWriter.forPrimitive(writeSelector); } @After @@ -72,14 +72,14 @@ public void tearDown() public void test_isNull_defaultOrNull() { writeToMemory(NullHandling.defaultDoubleValue()); - Assert.assertEquals(NullHandling.sqlCompatible(), new DoubleFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertEquals(NullHandling.sqlCompatible(), DoubleFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(5.1d); - Assert.assertFalse(new DoubleFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(DoubleFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test @@ -88,7 +88,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultDoubleValue()); final ColumnValueSelector readSelector = - new DoubleFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -103,7 +103,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5.1d); final ColumnValueSelector readSelector = - new DoubleFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + DoubleFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(5.1d, readSelector.getObject()); } @@ -114,7 +114,7 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultDoubleValue()); final DimensionSelector readSelector = - new DoubleFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -148,7 +148,7 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5.1d); final DimensionSelector readSelector = - new DoubleFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + DoubleFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -175,7 +175,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() writeToMemory(20.5d); final DimensionSelector readSelector = - new DoubleFieldReader().makeDimensionSelector( + DoubleFieldReader.forPrimitive().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java new file mode 100644 index 000000000000..e61e40db1cb1 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/FloatArrayFieldReaderTest.java @@ -0,0 +1,235 @@ +/* + * 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.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class FloatArrayFieldReaderTest extends InitializedNullHandlingTest +{ + private static final long MEMORY_POSITION = 1; + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + public ColumnValueSelector writeSelector; + + private WritableMemory memory; + private FieldWriter fieldWriter; + + //CHECKSTYLE.OFF: Regexp + private static final Object[] FLOATS_ARRAY_1 = new Object[]{ + null, + Float.MIN_VALUE, + Float.MAX_VALUE, + Float.MIN_NORMAL, + null, + Float.POSITIVE_INFINITY, + Float.NEGATIVE_INFINITY, + null, + 0.0f, + 1.234234f, + -1.344234f, + Float.NaN, + 129123.123123f, + null, + -21312213.33f, + 1111.0f, + null, + 23.0f + }; + + private static final Object[] FLOATS_ARRAY_2 = new Object[]{ + null, + Float.MAX_VALUE, + null, + Float.POSITIVE_INFINITY, + -0.0f, + 0.0f, + -1.234234f, + Float.MIN_VALUE, + 1.344234333f, + -129123.123123f, + 21312213.33f, + Float.NEGATIVE_INFINITY, + -1111.0f, + 1223.0f, + Float.MIN_NORMAL, + null, + Float.NaN, + null + }; + //CHECKSTYLE.ON: Regexp + + private static final List FLOATS_LIST_1; + private static final List FLOATS_LIST_2; + + static { + FLOATS_LIST_1 = Arrays.stream(FLOATS_ARRAY_1).map(val -> (Float) val).collect(Collectors.toList()); + FLOATS_LIST_2 = Arrays.stream(FLOATS_ARRAY_2).map(val -> (Float) val).collect(Collectors.toList()); + } + + @Before + public void setUp() + { + memory = WritableMemory.allocate(1000); + fieldWriter = NumericArrayFieldWriter.getFloatArrayFieldWriter(writeSelector); + } + + @After + public void tearDown() + { + fieldWriter.close(); + } + + @Test + public void test_isNull_null() + { + writeToMemory(null, MEMORY_POSITION); + Assert.assertTrue(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_aValue() + { + writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); + Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + Assert.assertFalse(new FloatArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_makeColumnValueSelector_null() + { + writeToMemory(null, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + Assert.assertTrue(readSelector.isNull()); + } + + @Test + public void test_makeColumnValueSelector_aValue() + { + writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(FLOATS_LIST_1, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_multipleValues() + { + long sz = writeToMemory(FLOATS_ARRAY_1, MEMORY_POSITION); + writeToMemory(FLOATS_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); + + + final ColumnValueSelector readSelector = new FloatArrayFieldReader().makeColumnValueSelector(memory, pointer); + + pointer.setPointer(0); + assertResults(FLOATS_LIST_1, readSelector.getObject()); + + pointer.setPointer(1); + assertResults(FLOATS_LIST_2, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.emptyList(), readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new FloatArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.singletonList(null), readSelector.getObject()); + } + + private long writeToMemory(final Object value, final long initialPosition) + { + Mockito.when(writeSelector.getObject()).thenReturn(value); + + long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition); + if (bytesWritten < 0) { + throw new ISE("Could not write"); + } + return bytesWritten; + } + + private void assertResults(List expected, Object actual) + { + if (expected == null) { + Assert.assertNull(actual); + } + Assert.assertTrue(actual instanceof Object[]); + List actualList = new ArrayList<>(); + for (Object val : (Object[]) actual) { + actualList.add((Float) val); + } + + Assert.assertEquals(expected, actualList); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java index e85225b99877..441858f0fd67 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/FloatFieldReaderTest.java @@ -59,7 +59,7 @@ public class FloatFieldReaderTest extends InitializedNullHandlingTest public void setUp() { memory = WritableMemory.allocate(1000); - fieldWriter = new FloatFieldWriter(writeSelector); + fieldWriter = FloatFieldWriter.forPrimitive(writeSelector); } @After @@ -72,14 +72,14 @@ public void tearDown() public void test_isNull_defaultOrNull() { writeToMemory(NullHandling.defaultFloatValue()); - Assert.assertEquals(NullHandling.sqlCompatible(), new FloatFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertEquals(NullHandling.sqlCompatible(), FloatFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(5.1f); - Assert.assertFalse(new FloatFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(FloatFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test @@ -88,7 +88,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultFloatValue()); final ColumnValueSelector readSelector = - new FloatFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -103,7 +103,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5.1f); final ColumnValueSelector readSelector = - new FloatFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + FloatFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(5.1f, readSelector.getObject()); } @@ -114,7 +114,7 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultFloatValue()); final DimensionSelector readSelector = - new FloatFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -148,7 +148,7 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5.1f); final DimensionSelector readSelector = - new FloatFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + FloatFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -175,7 +175,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() writeToMemory(20.5f); final DimensionSelector readSelector = - new FloatFieldReader().makeDimensionSelector( + FloatFieldReader.forPrimitive().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java new file mode 100644 index 000000000000..1e115f48e3c5 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/IndexArrayFieldPointer.java @@ -0,0 +1,56 @@ +/* + * 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.frame.field; + +import it.unimi.dsi.fastutil.longs.LongArrayList; + +import java.util.List; + +/** + * Stores the memory locations in an array, and spits out the value pointed to by the memory location by pointer, + * which is settable by the user + */ +public class IndexArrayFieldPointer implements ReadableFieldPointer +{ + private final LongArrayList indices; + private int pointer = 0; + + public IndexArrayFieldPointer(final List indices) + { + this.indices = new LongArrayList(indices); + } + + private int numIndices() + { + return indices.size(); + } + + public void setPointer(int newPointer) + { + assert newPointer >= 0 && newPointer < numIndices(); + this.pointer = newPointer; + } + + @Override + public long position() + { + return indices.getLong(pointer); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java new file mode 100644 index 000000000000..aa34cd6afaf3 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/LongArrayFieldReaderTest.java @@ -0,0 +1,211 @@ +/* + * 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.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.apache.druid.java.util.common.ISE; +import org.apache.druid.segment.ColumnValueSelector; +import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mock; +import org.mockito.Mockito; +import org.mockito.junit.MockitoJUnit; +import org.mockito.junit.MockitoRule; +import org.mockito.quality.Strictness; + +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.List; +import java.util.stream.Collectors; + +public class LongArrayFieldReaderTest extends InitializedNullHandlingTest +{ + private static final long MEMORY_POSITION = 1; + + @Rule + public MockitoRule mockitoRule = MockitoJUnit.rule().strictness(Strictness.STRICT_STUBS); + + @Mock + public ColumnValueSelector writeSelector; + + private WritableMemory memory; + private FieldWriter fieldWriter; + + private static final Object[] LONGS_ARRAY_1 = new Object[]{ + Long.MIN_VALUE, + Long.MAX_VALUE, + null, + 0L, + 123L, + -123L + }; + + private static final Object[] LONGS_ARRAY_2 = new Object[]{ + null, + 234L, + Long.MAX_VALUE, + null, + Long.MIN_VALUE, + 0L, + -234L, + null + }; + + private static final List LONGS_LIST_1; + private static final List LONGS_LIST_2; + + static { + LONGS_LIST_1 = Arrays.stream(LONGS_ARRAY_1).map(val -> (Long) val).collect(Collectors.toList()); + LONGS_LIST_2 = Arrays.stream(LONGS_ARRAY_2).map(val -> (Long) val).collect(Collectors.toList()); + } + + @Before + public void setUp() + { + memory = WritableMemory.allocate(1000); + fieldWriter = NumericArrayFieldWriter.getLongArrayFieldWriter(writeSelector); + } + + @After + public void tearDown() + { + fieldWriter.close(); + } + + @Test + public void test_isNull_null() + { + writeToMemory(null, MEMORY_POSITION); + Assert.assertTrue(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_aValue() + { + writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); + Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_isNull_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + Assert.assertFalse(new LongArrayFieldReader().isNull(memory, MEMORY_POSITION)); + } + + @Test + public void test_makeColumnValueSelector_null() + { + writeToMemory(null, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + Assert.assertTrue(readSelector.isNull()); + } + + @Test + public void test_makeColumnValueSelector_aValue() + { + writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(LONGS_LIST_1, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_multipleValues() + { + long sz = writeToMemory(LONGS_ARRAY_1, MEMORY_POSITION); + writeToMemory(LONGS_ARRAY_2, MEMORY_POSITION + sz); + IndexArrayFieldPointer pointer = new IndexArrayFieldPointer(ImmutableList.of(MEMORY_POSITION, MEMORY_POSITION + sz)); + + + final ColumnValueSelector readSelector = new LongArrayFieldReader().makeColumnValueSelector(memory, pointer); + + pointer.setPointer(0); + assertResults(LONGS_LIST_1, readSelector.getObject()); + + pointer.setPointer(1); + assertResults(LONGS_LIST_2, readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_emptyArray() + { + writeToMemory(new Object[]{}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.emptyList(), readSelector.getObject()); + } + + @Test + public void test_makeColumnValueSelector_arrayWithSingleNullElement() + { + writeToMemory(new Object[]{null}, MEMORY_POSITION); + + final ColumnValueSelector readSelector = + new LongArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + + assertResults(Collections.singletonList(null), readSelector.getObject()); + } + + private long writeToMemory(final Object value, final long initialPosition) + { + Mockito.when(writeSelector.getObject()).thenReturn(value); + + long bytesWritten = fieldWriter.writeTo(memory, initialPosition, memory.getCapacity() - initialPosition); + if (bytesWritten < 0) { + throw new ISE("Could not write"); + } + return bytesWritten; + } + + private void assertResults(List expected, Object actual) + { + if (expected == null) { + Assert.assertNull(actual); + } + Assert.assertTrue(actual instanceof Object[]); + List actualList = new ArrayList<>(); + for (Object val : (Object[]) actual) { + actualList.add((Long) val); + } + + Assert.assertEquals(expected, actualList); + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java index 06e6c42d9e70..7e73a7cfdf03 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/LongFieldReaderTest.java @@ -59,7 +59,7 @@ public class LongFieldReaderTest extends InitializedNullHandlingTest public void setUp() { memory = WritableMemory.allocate(1000); - fieldWriter = new LongFieldWriter(writeSelector); + fieldWriter = LongFieldWriter.forPrimitive(writeSelector); } @After @@ -72,14 +72,14 @@ public void tearDown() public void test_isNull_defaultOrNull() { writeToMemory(NullHandling.defaultLongValue()); - Assert.assertEquals(NullHandling.sqlCompatible(), new LongFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertEquals(NullHandling.sqlCompatible(), LongFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(5L); - Assert.assertFalse(new LongFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(LongFieldReader.forPrimitive().isNull(memory, MEMORY_POSITION)); } @Test @@ -88,7 +88,7 @@ public void test_makeColumnValueSelector_defaultOrNull() writeToMemory(NullHandling.defaultLongValue()); final ColumnValueSelector readSelector = - new LongFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(!NullHandling.replaceWithDefault(), readSelector.isNull()); @@ -103,7 +103,7 @@ public void test_makeColumnValueSelector_aValue() writeToMemory(5L); final ColumnValueSelector readSelector = - new LongFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + LongFieldReader.forPrimitive().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(5L, readSelector.getObject()); } @@ -114,7 +114,7 @@ public void test_makeDimensionSelector_defaultOrNull() writeToMemory(NullHandling.defaultLongValue()); final DimensionSelector readSelector = - new LongFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -148,7 +148,7 @@ public void test_makeDimensionSelector_aValue() writeToMemory(5L); final DimensionSelector readSelector = - new LongFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + LongFieldReader.forPrimitive().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -175,7 +175,7 @@ public void test_makeDimensionSelector_aValue_extractionFn() writeToMemory(25L); final DimensionSelector readSelector = - new LongFieldReader().makeDimensionSelector( + LongFieldReader.forPrimitive().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java index 97370187522e..bde3f77480ee 100644 --- a/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java +++ b/processing/src/test/java/org/apache/druid/frame/field/StringFieldReaderTest.java @@ -81,24 +81,24 @@ public void tearDown() public void test_isNull_nullValue() { writeToMemory(Collections.singletonList(null)); - Assert.assertTrue(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertTrue(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_twoNullValues() { writeToMemory(Arrays.asList(null, null)); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_nullRow() { writeToMemory(null); - Assert.assertTrue(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertTrue(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertTrue(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertTrue(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test @@ -107,33 +107,33 @@ public void test_isNull_emptyString() writeToMemory(Collections.singletonList("")); Assert.assertEquals( NullHandling.replaceWithDefault(), - new StringFieldReader(false).isNull(memory, MEMORY_POSITION) + new StringFieldReader().isNull(memory, MEMORY_POSITION) ); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_aValue() { writeToMemory(Collections.singletonList("foo")); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_multiString() { writeToMemory(ImmutableList.of("foo", "bar")); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test public void test_isNull_multiStringIncludingNulls() { writeToMemory(Arrays.asList(null, "bar")); - Assert.assertFalse(new StringFieldReader(false).isNull(memory, MEMORY_POSITION)); - Assert.assertFalse(new StringFieldReader(true).isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringFieldReader().isNull(memory, MEMORY_POSITION)); + Assert.assertFalse(new StringArrayFieldReader().isNull(memory, MEMORY_POSITION)); } @Test @@ -142,9 +142,9 @@ public void test_makeColumnValueSelector_singleString_notArray() writeToMemory(Collections.singletonList("foo")); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals("foo", readSelector.getObject()); Assert.assertArrayEquals(new Object[]{"foo"}, (Object[]) readSelectorAsArray.getObject()); @@ -156,9 +156,9 @@ public void test_makeColumnValueSelector_multiString() writeToMemory(ImmutableList.of("foo", "bar")); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertEquals(ImmutableList.of("foo", "bar"), readSelector.getObject()); Assert.assertArrayEquals(new Object[]{"foo", "bar"}, (Object[]) readSelectorAsArray.getObject()); @@ -170,9 +170,9 @@ public void test_makeColumnValueSelector_null() writeToMemory(Collections.singletonList(null)); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertNull(readSelector.getObject()); Assert.assertArrayEquals(new Object[]{null}, (Object[]) readSelectorAsArray.getObject()); @@ -184,9 +184,9 @@ public void test_makeColumnValueSelector_empty() writeToMemory(Collections.emptyList()); final ColumnValueSelector readSelector = - new StringFieldReader(false).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); final ColumnValueSelector readSelectorAsArray = - new StringFieldReader(true).makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); + new StringArrayFieldReader().makeColumnValueSelector(memory, new ConstantFieldPointer(MEMORY_POSITION)); Assert.assertNull(readSelector.getObject()); Assert.assertArrayEquals(ObjectArrays.EMPTY_ARRAY, (Object[]) readSelectorAsArray.getObject()); @@ -199,7 +199,7 @@ public void test_makeDimensionSelector_multiString_asArray() final IllegalStateException e = Assert.assertThrows( IllegalStateException.class, - () -> new StringFieldReader(true).makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null) + () -> new StringArrayFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null) ); MatcherAssert.assertThat( @@ -214,7 +214,7 @@ public void test_makeDimensionSelector_multiString() writeToMemory(ImmutableList.of("foo", "bar")); final DimensionSelector readSelector = - new StringFieldReader(false).makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); + new StringFieldReader().makeDimensionSelector(memory, new ConstantFieldPointer(MEMORY_POSITION), null); // Data retrieval tests. final IndexedInts row = readSelector.getRow(); @@ -244,7 +244,7 @@ public void test_makeDimensionSelector_multiString_withExtractionFn() writeToMemory(ImmutableList.of("foo", "bar")); final DimensionSelector readSelector = - new StringFieldReader(false).makeDimensionSelector( + new StringFieldReader().makeDimensionSelector( memory, new ConstantFieldPointer(MEMORY_POSITION), new SubstringDimExtractionFn(1, null) diff --git a/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java b/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java new file mode 100644 index 000000000000..276e598ad367 --- /dev/null +++ b/processing/src/test/java/org/apache/druid/frame/field/TransformUtilsTest.java @@ -0,0 +1,187 @@ +/* + * 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.frame.field; + +import com.google.common.collect.ImmutableList; +import org.apache.datasketches.memory.WritableMemory; +import org.junit.Assert; +import org.junit.Test; + +import java.util.List; + +public class TransformUtilsTest +{ + + private final WritableMemory lhsMemory = WritableMemory.allocate(10); + private final WritableMemory rhsMemory = WritableMemory.allocate(10); + + private static final long MEMORY_LOCATION = 0; + + @Test + public void doubleTestWithoutNaN() + { + //CHECKSTYLE.OFF: Regexp + List values = + ImmutableList.of( + Double.MAX_VALUE, + Double.MIN_VALUE, + Double.POSITIVE_INFINITY, + Double.NEGATIVE_INFINITY, + Double.MIN_NORMAL, + 0.0d, + 1.234234d, + -1.344234d, + 129123.123123, + -21312213.33, + 1111.0, + 23.0 + ); + //CHECKSTYLE.ON: Regexp + + for (double value : values) { + Assert.assertEquals( + value, + TransformUtils.detransformToDouble(TransformUtils.transformFromDouble(value)), + 0.0 + ); + + } + + for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) { + for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) { + double lhs = values.get(lhsIndex); + double rhs = values.get(rhsIndex); + lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromDouble(lhs)); + rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromDouble(rhs)); + int byteCmp = byteComparison(Double.BYTES); + + if (byteCmp < 0) { + Assert.assertTrue(lhs < rhs); + } else if (byteCmp == 0) { + Assert.assertEquals(lhs, rhs, 0.0); + } else { + Assert.assertTrue(lhs > rhs); + } + + } + } + } + + @Test + public void longTest() + { + List values = + ImmutableList.of( + Long.MIN_VALUE, + Long.MAX_VALUE, + 0L, + 123L, + -123L + ); + + for (long value : values) { + Assert.assertEquals( + value, + TransformUtils.detransformToLong(TransformUtils.transformFromLong(value)) + ); + + } + + for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) { + for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) { + long lhs = values.get(lhsIndex); + long rhs = values.get(rhsIndex); + lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromLong(lhs)); + rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromLong(rhs)); + int byteCmp = byteComparison(Long.BYTES); + + if (byteCmp < 0) { + Assert.assertTrue(lhs < rhs); + } else if (byteCmp == 0) { + Assert.assertEquals(lhs, rhs); + } else { + Assert.assertTrue(lhs > rhs); + } + } + } + } + + @Test + public void floatTestWithoutNaN() + { + //CHECKSTYLE.OFF: Regexp + List values = + ImmutableList.of( + Float.MIN_VALUE, + Float.MAX_VALUE, + Float.MIN_NORMAL, + Float.POSITIVE_INFINITY, + Float.NEGATIVE_INFINITY, + 0.0f, + 1.234234f, + -1.344234f, + 129123.123123f, + -21312213.33f, + 1111.0f, + 23.0f + ); + //CHECKSTYLE.ON: Regexp + + for (float value : values) { + Assert.assertEquals( + value, + TransformUtils.detransformToFloat(TransformUtils.transformFromFloat(value)), + 0.0 + ); + + } + + for (int lhsIndex = 0; lhsIndex < values.size(); ++lhsIndex) { + for (int rhsIndex = lhsIndex; rhsIndex < values.size(); ++rhsIndex) { + float lhs = values.get(lhsIndex); + float rhs = values.get(rhsIndex); + lhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromFloat(lhs)); + rhsMemory.putLong(MEMORY_LOCATION, TransformUtils.transformFromFloat(rhs)); + int byteCmp = byteComparison(Long.BYTES); + + if (byteCmp < 0) { + Assert.assertTrue(lhs < rhs); + } else if (byteCmp == 0) { + Assert.assertEquals(lhs, rhs, 0.0); + } else { + Assert.assertTrue(lhs > rhs); + } + } + } + } + + private int byteComparison(int numBytes) + { + for (int i = 0; i < numBytes; ++i) { + byte lhsByte = lhsMemory.getByte(MEMORY_LOCATION + i); + byte rhsByte = rhsMemory.getByte(MEMORY_LOCATION + i); + final int cmp = (lhsByte & 0xFF) - (rhsByte & 0xFF); + if (cmp != 0) { + return cmp; + } + } + return 0; + } +} diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java index 342ccbefc48c..31b24825f959 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTest.java @@ -153,6 +153,33 @@ public void test_long() testWithDataset(FrameWriterTestData.TEST_LONGS); } + @Test + public void test_arrayLong() + { + // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those + // parameters + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); + testWithDataset(FrameWriterTestData.TEST_ARRAYS_LONG); + } + + @Test + public void test_arrayFloat() + { + // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those + // parameters + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); + testWithDataset(FrameWriterTestData.TEST_ARRAYS_FLOAT); + } + + @Test + public void test_arrayDouble() + { + // ARRAY can't be read or written for columnar frames, therefore skip the check if it encounters those + // parameters + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); + testWithDataset(FrameWriterTestData.TEST_ARRAYS_DOUBLE); + } + @Test public void test_float() { @@ -226,6 +253,14 @@ public void test_typePairs() // Test all possible arrangements of two different types. for (final FrameWriterTestData.Dataset dataset1 : FrameWriterTestData.DATASETS) { for (final FrameWriterTestData.Dataset dataset2 : FrameWriterTestData.DATASETS) { + if (dataset1.getType().isArray() && dataset1.getType().getElementType().isNumeric() + || dataset2.getType().isArray() && dataset2.getType().getElementType().isNumeric()) { + if (inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR) { + // Skip the check if any of the dataset is a numerical array and any of the input or the output frame type + // is COLUMNAR. + continue; + } + } final RowSignature signature = makeSignature(Arrays.asList(dataset1, dataset2)); final Sequence> rowSequence = unsortAndMakeRows(Arrays.asList(dataset1, dataset2)); @@ -265,6 +300,7 @@ public void test_typePairs() public void test_insufficientWriteCapacity() { // Test every possible capacity, up to the amount required to write all items from every list. + Assume.assumeFalse(inputFrameType == FrameType.COLUMNAR || outputFrameType == FrameType.COLUMNAR); final RowSignature signature = makeSignature(FrameWriterTestData.DATASETS); final Sequence> rowSequence = unsortAndMakeRows(FrameWriterTestData.DATASETS); final int totalRows = rowSequence.toList().size(); diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java index 3017f5b9ed48..a52c4d5efdd8 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWriterTestData.java @@ -136,6 +136,24 @@ public class FrameWriterTestData ).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList()) ); + public static final Dataset TEST_ARRAYS_LONG = new Dataset<>( + ColumnType.LONG_ARRAY, + Arrays.asList( + null, + ObjectArrays.EMPTY_ARRAY, + new Object[]{null}, + new Object[]{null, 6L, null, 5L, null}, + new Object[]{null, 6L, null, 5L, NullHandling.defaultLongValue()}, + new Object[]{null, 6L, null, 5L, 0L, -1L}, + new Object[]{null, 6L, null, 5L, 0L, -1L, Long.MIN_VALUE}, + new Object[]{null, 6L, null, 5L, 0L, -1L, Long.MAX_VALUE}, + new Object[]{5L}, + new Object[]{5L, 6L}, + new Object[]{5L, 6L, null}, + new Object[]{Long.MAX_VALUE, Long.MIN_VALUE} + ) + ); + public static final Dataset TEST_FLOATS = new Dataset<>( ColumnType.FLOAT, Stream.of( @@ -158,6 +176,28 @@ public class FrameWriterTestData ).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList()) ); + //CHECKSTYLE.OFF: Regexp + public static final Dataset TEST_ARRAYS_FLOAT = new Dataset<>( + ColumnType.FLOAT_ARRAY, + Arrays.asList( + null, + ObjectArrays.EMPTY_ARRAY, + new Object[]{null}, + new Object[]{null, 6.2f, null, 5.1f, null}, + new Object[]{null, 6.2f, null, 5.1f, NullHandling.defaultFloatValue()}, + new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f}, + new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f, Float.MIN_VALUE}, + new Object[]{null, 6.2f, null, 5.7f, 0.0f, -1.0f, Float.MAX_VALUE}, + new Object[]{Float.NEGATIVE_INFINITY, Float.MIN_VALUE}, + new Object[]{5.7f}, + new Object[]{5.7f, 6.2f}, + new Object[]{5.7f, 6.2f, null}, + new Object[]{Float.MAX_VALUE, Float.MIN_VALUE}, + new Object[]{Float.POSITIVE_INFINITY, Float.MIN_VALUE} + ) + ); + //CHECKSTYLE.ON: Regexp + public static final Dataset TEST_DOUBLES = new Dataset<>( ColumnType.DOUBLE, Stream.of( @@ -180,6 +220,28 @@ public class FrameWriterTestData ).sorted(Comparators.naturalNullsFirst()).collect(Collectors.toList()) ); + //CHECKSTYLE.OFF: Regexp + public static final Dataset TEST_ARRAYS_DOUBLE = new Dataset<>( + ColumnType.DOUBLE_ARRAY, + Arrays.asList( + null, + ObjectArrays.EMPTY_ARRAY, + new Object[]{null}, + new Object[]{null, 6.2d, null, 5.1d, null}, + new Object[]{null, 6.2d, null, 5.1d, NullHandling.defaultDoubleValue()}, + new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d}, + new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d, Double.MIN_VALUE}, + new Object[]{null, 6.2d, null, 5.7d, 0.0d, -1.0d, Double.MAX_VALUE}, + new Object[]{Double.NEGATIVE_INFINITY, Double.MIN_VALUE}, + new Object[]{5.7d}, + new Object[]{5.7d, 6.2d}, + new Object[]{5.7d, 6.2d, null}, + new Object[]{Double.MAX_VALUE, Double.MIN_VALUE}, + new Object[]{Double.POSITIVE_INFINITY, Double.MIN_VALUE} + ) + ); + //CHECKSTYLE.ON: Regexp + public static final Dataset TEST_COMPLEX = new Dataset<>( HyperUniquesAggregatorFactory.TYPE, Arrays.asList( @@ -200,6 +262,9 @@ public class FrameWriterTestData .add(TEST_STRINGS_SINGLE_VALUE) .add(TEST_STRINGS_MULTI_VALUE) .add(TEST_ARRAYS_STRING) + .add(TEST_ARRAYS_LONG) + .add(TEST_ARRAYS_FLOAT) + .add(TEST_ARRAYS_DOUBLE) .add(TEST_COMPLEX) .build(); diff --git a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java index 42ed468ec8c3..9d359eed05e0 100644 --- a/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java +++ b/processing/src/test/java/org/apache/druid/frame/write/FrameWritersTest.java @@ -75,42 +75,6 @@ public void test_columnar() Assert.assertEquals(ALLOCATOR_CAPACITY, factory.allocatorCapacity()); } - @Test - public void test_rowBased_unsupportedSortingColumnType() - { - final UnsupportedColumnTypeException e = Assert.assertThrows( - UnsupportedColumnTypeException.class, - () -> FrameWriters.makeFrameWriterFactory( - FrameType.ROW_BASED, - new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY), - RowSignature.builder().add("x", ColumnType.LONG_ARRAY).build(), - Collections.singletonList(new KeyColumn("x", KeyOrder.ASCENDING)) - ) - ); - - Assert.assertEquals("x", e.getColumnName()); - Assert.assertEquals(ColumnType.LONG_ARRAY, e.getColumnType()); - } - - @Test - public void test_rowBased_unsupportedNonSortingColumnType() - { - final FrameWriterFactory factory = FrameWriters.makeFrameWriterFactory( - FrameType.ROW_BASED, - new ArenaMemoryAllocatorFactory(ALLOCATOR_CAPACITY), - RowSignature.builder().add("x", ColumnType.LONG_ARRAY).build(), - Collections.emptyList() - ); - - final UnsupportedColumnTypeException e = Assert.assertThrows( - UnsupportedColumnTypeException.class, - () -> factory.newFrameWriter(new AllNullColumnSelectorFactory()) - ); - - Assert.assertEquals("x", e.getColumnName()); - Assert.assertEquals(ColumnType.LONG_ARRAY, e.getColumnType()); - } - @Test public void test_columnar_unsupportedColumnType() { diff --git a/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java index fbbc089255b5..e01c9459fa12 100644 --- a/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java +++ b/processing/src/test/java/org/apache/druid/query/FrameBasedInlineDataSourceSerializerTest.java @@ -32,6 +32,7 @@ import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.jackson.DefaultObjectMapper; import org.apache.druid.java.util.common.Intervals; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.guava.Sequence; import org.apache.druid.segment.Cursor; import org.apache.druid.segment.column.ColumnType; @@ -40,6 +41,7 @@ import org.junit.Assert; import org.junit.Test; +import java.io.Closeable; import java.util.ArrayList; public class FrameBasedInlineDataSourceSerializerTest @@ -124,10 +126,11 @@ private FrameBasedInlineDataSource convertToFrameBasedInlineDataSource( RowSignature rowSignature ) { - Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable( + Pair cursorAndCloseable = IterableRowsCursorHelper.getCursorFromIterable( inlineDataSource.getRows(), rowSignature ); + Cursor cursor = cursorAndCloseable.lhs; RowSignature modifiedRowSignature = FrameWriterUtils.replaceUnknownTypesWithNestedColumns(rowSignature); Sequence frames = FrameCursorUtils.cursorToFrames( cursor, @@ -139,7 +142,7 @@ private FrameBasedInlineDataSource convertToFrameBasedInlineDataSource( ) ); return new FrameBasedInlineDataSource( - frames.map(frame -> new FrameSignaturePair(frame, rowSignature)).toList(), + frames.map(frame -> new FrameSignaturePair(frame, rowSignature)).withBaggage(cursorAndCloseable.rhs).toList(), modifiedRowSignature ); } diff --git a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java index 1acaceabbd60..45f14b80976c 100644 --- a/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java +++ b/processing/src/test/java/org/apache/druid/query/IterableRowsCursorHelperTest.java @@ -48,7 +48,7 @@ public class IterableRowsCursorHelperTest @Test public void getCursorFromIterable() { - Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(rows, rowSignature); + Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(rows, rowSignature).lhs; testCursorMatchesRowSequence(cursor, rowSignature, rows); } @@ -56,7 +56,7 @@ public void getCursorFromIterable() public void getCursorFromSequence() { - Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(Sequences.simple(rows), rowSignature); + Cursor cursor = IterableRowsCursorHelper.getCursorFromSequence(Sequences.simple(rows), rowSignature).lhs; testCursorMatchesRowSequence(cursor, rowSignature, rows); } diff --git a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java index 0d91e7d5e001..fcf6720311f0 100644 --- a/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java +++ b/processing/src/test/java/org/apache/druid/query/scan/NestedDataScanQueryTest.java @@ -787,12 +787,12 @@ public void testIngestAndScanSegmentsRealtimeSchemaDiscoveryTypeGauntlet() throw Assert.assertEquals(resultsRealtime.size(), resultsSegments.size()); if (NullHandling.replaceWithDefault()) { Assert.assertEquals( - "[[1672531200000, null, 0, 0.0, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 2, 0.0, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, 0.0, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, 0, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, 0, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]", + "[[1672531200000, null, 0, 0.0, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}, v=[]}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 2, 0.0, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}, v=[]}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}, v=[]}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, 0.0, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}, v=[]}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, 0, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}, v=a}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}, v=b}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, 0, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]", resultsSegments.get(0).getEvents().toString() ); } else { Assert.assertEquals( - "[[1672531200000, null, null, null, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, , 2, null, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, null, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, null, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, null, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]", + "[[1672531200000, null, null, null, 1, 51, -0.13, 1, [], [51, -35], {a=700, b={x=g, y=1.1, z=[9, null, 9, 9]}, v=[]}, {x=400, y=[{l=[null], m=100, n=5}, {l=[a, b, c], m=a, n=1}], z={}}, null, [a, b], null, [2, 3], null, [null], null, [1, 0, 1], null, [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, , 2, null, 0, b, 1.1, b, 2, b, {a=200, b={x=b, y=1.1, z=[2, 4, 6]}, v=[]}, {x=10, y=[{l=[b, b, c], m=b, n=2}, [1, 2, 3]], z={a=[5.5], b=false}}, [a, b, c], [null, b], [2, 3], null, [3.3, 4.4, 5.5], [999.0, null, 5.5], [null, null, 2.2], [1, 1], [null, [null], []], [{x=3}, {x=4}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, a, 1, 1.0, 1, 1, 1, 1, 1, 1, {a=100, b={x=a, y=1.1, z=[1, 2, 3, 4]}, v=[]}, {x=1234, y=[{l=[a, b, c], m=a, n=1}, {l=[a, b, c], m=a, n=1}], z={a=[1.1, 2.2, 3.3], b=true}}, [a, b], [a, b], [1, 2, 3], [1, null, 3], [1.1, 2.2, 3.3], [1.1, 2.2, null], [a, 1, 2.2], [1, 0, 1], [[1, 2, null], [3, 4]], [{x=1}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, b, 4, 3.3, 1, 1, null, {}, 4, 1, {a=400, b={x=d, y=1.1, z=[3, 4]}, v=[]}, {x=1234, z={a=[1.1, 2.2, 3.3], b=true}}, [d, e], [b, b], [1, 4], [1], [2.2, 3.3, 4.0], null, [a, b, c], [null, 0, 1], [[1, 2], [3, 4], [5, 6, 7]], [{x=null}, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, c, null, 4.4, 1, hello, -1000, {}, [], hello, {a=500, b={x=e, z=[1, 2, 3, 4]}, v=a}, {x=11, y=[], z={a=[null], b=false}}, null, null, [1, 2, 3], [], [1.1, 2.2, 3.3], null, null, [0], null, [{x=1000}, {y=2000}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, d, 5, 5.9, 0, null, 3.33, a, 6, null, {a=600, b={x=f, y=1.1, z=[6, 7, 8, 9]}, v=b}, null, [a, b], null, null, [null, 2, 9], null, [999.0, 5.5, null], [a, 1, 2.2], [], [[1], [1, 2, null]], [{a=1}, {b=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1], [1672531200000, null, 3, 2.0, null, 3.0, 1.0, 3.3, 3, 3.0, {a=300}, {x=4.4, y=[{l=[], m=100, n=3}, {l=[a]}, {l=[b], n=[]}], z={a=[], b=true}}, [b, c], [d, null, b], [1, 2, 3, 4], [1, 2, 3], [1.1, 3.3], [null, 2.2, null], [1, null, 1], [1, null, 1], [[1], null, [1, 2, 3]], [null, {x=2}], null, hello, 1234, 1.234, {x=1, y=hello, z={a=1.1, b=1234, c=[a, b, c]}}, [a, b, c], [1, 2, 3], [1.1, 2.2, 3.3], [], {}, [null, null], [{}, {}, {}], [{a=b, x=1, y=1.3}], 1]]", resultsSegments.get(0).getEvents().toString() ); } diff --git a/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java index ed59f5f80652..6a244e9bae97 100644 --- a/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java +++ b/processing/src/test/java/org/apache/druid/segment/join/table/FrameBasedIndexedTableTest.java @@ -33,6 +33,7 @@ import org.apache.druid.frame.write.FrameWriterFactory; import org.apache.druid.frame.write.FrameWriters; import org.apache.druid.java.util.common.IAE; +import org.apache.druid.java.util.common.Pair; import org.apache.druid.java.util.common.StringUtils; import org.apache.druid.java.util.common.io.Closer; import org.apache.druid.query.FrameBasedInlineDataSource; @@ -42,10 +43,12 @@ import org.apache.druid.segment.column.ColumnType; import org.apache.druid.segment.column.RowSignature; import org.apache.druid.testing.InitializedNullHandlingTest; +import org.junit.After; import org.junit.Assert; import org.junit.Before; import org.junit.Test; +import java.io.Closeable; import java.io.IOException; import java.util.ArrayList; import java.util.Arrays; @@ -207,11 +210,13 @@ public class FrameBasedIndexedTableTest extends InitializedNullHandlingTest private FrameBasedInlineDataSource dataSource; private FrameBasedIndexedTable frameBasedIndexedTable; + private Pair cursorCloseablePair; @Before public void setup() { - Cursor cursor = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE); + cursorCloseablePair = IterableRowsCursorHelper.getCursorFromIterable(DATASOURCE_ROWS, ROW_SIGNATURE); + Cursor cursor = cursorCloseablePair.lhs; FrameWriterFactory frameWriterFactory = FrameWriters.makeFrameWriterFactory( FrameType.COLUMNAR, new SingleMemoryAllocatorFactory(HeapMemoryAllocator.unlimited()), @@ -226,7 +231,12 @@ public void setup() ); frameBasedIndexedTable = new FrameBasedIndexedTable(dataSource, KEY_COLUMNS, "test"); + } + @After + public void tearDown() throws IOException + { + cursorCloseablePair.rhs.close(); } @Test diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java index 80daa3549dcf..653b39ff9bf7 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedDataColumnSupplierTest.java @@ -171,7 +171,7 @@ public static void staticSetup() @Before public void setup() throws IOException { - final String fileNameBase = "test"; + final String fileNameBase = "test/column"; final String arrayFileNameBase = "array"; fileMapper = smooshify(fileNameBase, tempFolder.newFolder(), data); baseBuffer = fileMapper.mapFile(fileNameBase); diff --git a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java index 0e8d95cc57c9..33df1887ea54 100644 --- a/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java +++ b/processing/src/test/java/org/apache/druid/segment/nested/NestedFieldTypeInfoTest.java @@ -56,6 +56,23 @@ public void testSingleType() throws IOException } } + @Test + public void testSingleTypeWithEmptyArray() throws IOException + { + List supportedTypes = ImmutableList.of( + ColumnType.STRING, + ColumnType.LONG, + ColumnType.DOUBLE, + ColumnType.STRING_ARRAY, + ColumnType.LONG_ARRAY, + ColumnType.DOUBLE_ARRAY + ); + + for (ColumnType type : supportedTypes) { + testSingleTypeWithEmptyArray(type); + } + } + @Test public void testMultiType() throws IOException { @@ -137,6 +154,51 @@ private void testMultiType(Set columnTypes) throws IOException Assert.assertEquals(1, BUFFER.position()); } + private void testSingleTypeWithEmptyArray(ColumnType columnType) throws IOException + { + FieldTypeInfo.MutableTypeSet typeSet = new FieldTypeInfo.MutableTypeSet(); + typeSet.add(columnType); + typeSet.addUntypedArray(); + + if (columnType.isArray()) { + // arrays with empty arrays are still single type + Assert.assertEquals(columnType, typeSet.getSingleType()); + Assert.assertEquals(ImmutableSet.of(columnType), FieldTypeInfo.convertToSet(typeSet.getByteValue())); + + writeTypeSet(typeSet); + FieldTypeInfo info = new FieldTypeInfo(BUFFER); + Assert.assertEquals(0, BUFFER.position()); + + FieldTypeInfo.TypeSet roundTrip = info.getTypes(0); + Assert.assertEquals(columnType, roundTrip.getSingleType()); + + FieldTypeInfo info2 = FieldTypeInfo.read(BUFFER, 1); + Assert.assertEquals(info.getTypes(0), info2.getTypes(0)); + Assert.assertEquals(1, BUFFER.position()); + } else { + // scalar types become multi-type + Set columnTypes = ImmutableSet.of(columnType, ColumnType.ofArray(columnType)); + FieldTypeInfo.MutableTypeSet merge = new FieldTypeInfo.MutableTypeSet(); + merge.merge(new FieldTypeInfo.MutableTypeSet().add(columnType).getByteValue(), true); + + Assert.assertEquals(merge.getByteValue(), typeSet.getByteValue()); + Assert.assertNull(typeSet.getSingleType()); + Assert.assertEquals(columnTypes, FieldTypeInfo.convertToSet(typeSet.getByteValue())); + + writeTypeSet(typeSet); + FieldTypeInfo info = new FieldTypeInfo(BUFFER); + Assert.assertEquals(0, BUFFER.position()); + + FieldTypeInfo.TypeSet roundTrip = info.getTypes(0); + Assert.assertNull(roundTrip.getSingleType()); + Assert.assertEquals(columnTypes, FieldTypeInfo.convertToSet(roundTrip.getByteValue())); + + FieldTypeInfo info2 = FieldTypeInfo.read(BUFFER, 1); + Assert.assertEquals(info.getTypes(0), info2.getTypes(0)); + Assert.assertEquals(1, BUFFER.position()); + } + } + private static void writeTypeSet(FieldTypeInfo.MutableTypeSet typeSet) throws IOException { BUFFER.position(0); diff --git a/processing/src/test/resources/nested-all-types-test-data.json b/processing/src/test/resources/nested-all-types-test-data.json index 34d92b52ae82..b70c87646019 100644 --- a/processing/src/test/resources/nested-all-types-test-data.json +++ b/processing/src/test/resources/nested-all-types-test-data.json @@ -1,7 +1,7 @@ -{"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantNumeric": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "variantWithArrays": 1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantNumeric": 1.1, "variantEmptyObj":"b", "variantEmtpyArray":2, "variantWithArrays": "b", "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}, [1, 2, 3]], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantNumeric": 1.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "variantWithArrays": 3.0, "obj":{"a": 300}, "complexObj":{"x": 4.4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "1", "variantEmptyObj":{}, "variantEmtpyArray":4, "variantWithArrays": "1", "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantNumeric": -1000, "variantEmptyObj":{}, "variantEmtpyArray":[], "variantWithArrays": "hello", "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantNumeric": 3.33, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} -{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantNumeric": -0.13, "variantEmptyObj":1, "variantEmtpyArray":[], "variantWithArrays": [51, -35], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":"a", "long":1, "double":1.0, "bool": true, "variant": 1, "variantNumeric": 1, "variantEmptyObj":1, "variantEmtpyArray":1, "variantWithArrays": 1, "obj":{"a": 100, "b": {"x": "a", "y": 1.1, "z": [1, 2, 3, 4]}, "v": []}, "complexObj":{"x": 1234, "y": [{"l": ["a", "b", "c"], "m": "a", "n": 1},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["a", "b"], "arrayStringNulls": ["a", "b"], "arrayLong":[1, 2, 3], "arrayLongNulls":[1, null,3], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":[1.1, 2.2, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[true, false, true], "arrayNestedLong":[[1, 2, null], [3, 4]], "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":"", "long":2, "bool": false, "variant": "b", "variantNumeric": 1.1, "variantEmptyObj":"b", "variantEmtpyArray":2, "variantWithArrays": "b", "obj":{"a": 200, "b": {"x": "b", "y": 1.1, "z": [2, 4, 6]}, "v": []}, "complexObj":{"x": 10, "y": [{"l": ["b", "b", "c"], "m": "b", "n": 2}, [1, 2, 3]], "z": {"a": [5.5], "b": false}}, "arrayString": ["a", "b", "c"], "arrayStringNulls": [null, "b"], "arrayLong":[2, 3], "arrayDouble":[3.3, 4.4, 5.5], "arrayDoubleNulls":[999, null, 5.5], "arrayVariant":[null, null, 2.2], "arrayBool":[true, true], "arrayNestedLong":[null, [null], []], "arrayObject":[{"x": 3},{"x":4}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":"null", "long":3, "double":2.0, "variant": 3.0, "variantNumeric": 1.0, "variantEmptyObj":3.3, "variantEmtpyArray":3, "variantWithArrays": 3.0, "obj":{"a": 300}, "complexObj":{"x": 4.4, "y": [{"l": [], "m": 100, "n": 3},{"l": ["a"]}, {"l": ["b"], "n": []}], "z": {"a": [], "b": true}}, "arrayString": ["b", "c"], "arrayStringNulls": ["d", null, "b"], "arrayLong":[1, 2, 3, 4], "arrayLongNulls":[1, 2, 3], "arrayDouble":[1.1, 3.3], "arrayDoubleNulls":[null, 2.2, null], "arrayVariant":[1, null, 1], "arrayBool":[true, null, true], "arrayNestedLong":[[1], null, [1, 2, 3]], "arrayObject":[null,{"x":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":"b", "long":4, "double":3.3, "bool": true, "variant": "1", "variantEmptyObj":{}, "variantEmtpyArray":4, "variantWithArrays": "1", "obj":{"a": 400, "b": {"x": "d", "y": 1.1, "z": [3, 4]}, "v": []}, "complexObj":{"x": 1234, "z": {"a": [1.1, 2.2, 3.3], "b": true}}, "arrayString": ["d", "e"], "arrayStringNulls": ["b", "b"], "arrayLong":[1, 4], "arrayLongNulls":[1], "arrayDouble":[2.2, 3.3, 4.0], "arrayVariant":["a", "b", "c"], "arrayBool":[null, false, true], "arrayNestedLong":[[1, 2], [3, 4], [5, 6, 7]], "arrayObject":[{"x": null},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":"c", "long": null, "double":4.4, "bool": true, "variant": "hello", "variantNumeric": -1000, "variantEmptyObj":{}, "variantEmtpyArray":[], "variantWithArrays": "hello", "obj":{"a": 500, "b": {"x": "e", "z": [1, 2, 3, 4]}, "v": "a"}, "complexObj":{"x": 11, "y": [], "z": {"a": [null], "b": false}}, "arrayString": null, "arrayLong":[1, 2, 3], "arrayLongNulls":[], "arrayDouble":[1.1, 2.2, 3.3], "arrayDoubleNulls":null, "arrayBool":[false], "arrayObject":[{"x": 1000},{"y":2000}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":"d", "long":5, "double":5.9, "bool": false, "variantNumeric": 3.33, "variantEmptyObj":"a", "variantEmtpyArray":6, "obj":{"a": 600, "b": {"x": "f", "y": 1.1, "z": [6, 7, 8, 9]}, "v": "b"}, "arrayString": ["a", "b"], "arrayStringNulls": null, "arrayLongNulls":[null, 2, 9], "arrayDouble":null, "arrayDoubleNulls":[999, 5.5, null], "arrayVariant":["a", 1, 2.2], "arrayBool":[], "arrayNestedLong":[[1], [1, 2, null]], "arrayObject":[{"a": 1},{"b":2}], "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} +{"timestamp": "2023-01-01T00:00:00", "str":null, "double":null, "bool": true, "variant": 51, "variantNumeric": -0.13, "variantEmptyObj":1, "variantEmtpyArray":[], "variantWithArrays": [51, -35], "obj":{"a": 700, "b": {"x": "g", "y": 1.1, "z": [9, null, 9, 9]}, "v": []}, "complexObj":{"x": 400, "y": [{"l": [null], "m": 100, "n": 5},{"l": ["a", "b", "c"], "m": "a", "n": 1}], "z": {}}, "arrayStringNulls": ["a", "b"], "arrayLong":null, "arrayLongNulls":[2, 3], "arrayDoubleNulls":[null], "arrayVariant":null, "arrayBool":[true, false, true], "arrayNestedLong":null, "arrayObject":[{"x": 1},{"x":2}], "null": null, "cstr": "hello", "clong": 1234, "cdouble": 1.234, "cObj":{"x": 1, "y": "hello", "z": {"a": 1.1, "b": 1234, "c": ["a", "b", "c"]}}, "cstringArray": ["a", "b", "c"], "cLongArray": [1, 2, 3], "cDoubleArray": [1.1, 2.2, 3.3], "cEmptyArray":[], "cEmptyObj":{}, "cNullArray": [null, null], "cEmptyObjectArray": [{},{},{}], "cObjectArray": [{"a":"b", "x":1, "y":1.3}]} diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java index 16f642701724..3bb20bb3780e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/BaseCalciteQueryTest.java @@ -1170,7 +1170,7 @@ protected void skipVectorize() skipVectorize = true; } - protected void notMsqCompatible() + protected void msqIncompatible() { msqCompatible = false; } diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java index 8fc531d35ab6..e6a669b9c28b 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteArraysQueryTest.java @@ -87,7 +87,6 @@ public class CalciteArraysQueryTest extends BaseCalciteQueryTest @Test public void testSelectConstantArrayExpressionFromTable() { - notMsqCompatible(); testQuery( "SELECT ARRAY[1,2] as arr, dim1 FROM foo LIMIT 1", ImmutableList.of( @@ -169,7 +168,8 @@ public void testSelectNonConstantArrayExpressionFromTable() @Test public void testSelectNonConstantArrayExpressionFromTableForMultival() { - notMsqCompatible(); + // Produces nested string array, that MSQ can't infer from the selector + msqIncompatible(); final String sql = "SELECT ARRAY[CONCAT(dim3, 'word'),'up'] as arr, dim1 FROM foo LIMIT 5"; final Query scanQuery = newScanQueryBuilder() .dataSource(CalciteTests.DATASOURCE1) @@ -209,7 +209,6 @@ public void testSomeArrayFunctionsWithScanQuery() // Yes these outputs are strange sometimes, arrays are in a partial state of existence so end up a bit // stringy for now this is because virtual column selectors are coercing values back to stringish so that // multi-valued string dimensions can be grouped on. - notMsqCompatible(); List expectedResults; if (useDefault) { expectedResults = ImmutableList.of( @@ -389,7 +388,6 @@ public void testSomeArrayFunctionsWithScanQueryNoStringify() // which will still always be stringified to ultimately adhere to the varchar type // as array support increases in the engine this will likely change since using explict array functions should // probably kick it into an array - notMsqCompatible(); List expectedResults; if (useDefault) { expectedResults = ImmutableList.of( @@ -1021,7 +1019,6 @@ public void testArrayOffset() @Test public void testArrayGroupAsLongArray() { - notMsqCompatible(); // Cannot vectorize as we donot have support in native query subsytem for grouping on arrays cannotVectorize(); testQuery( @@ -1073,7 +1070,6 @@ public void testArrayGroupAsDoubleArray() { // Cannot vectorize as we donot have support in native query subsytem for grouping on arrays as keys cannotVectorize(); - notMsqCompatible(); testQuery( "SELECT ARRAY[d1], SUM(cnt) FROM druid.numfoo GROUP BY 1 ORDER BY 2 DESC", QUERY_CONTEXT_NO_STRINGIFY_ARRAY, @@ -1121,7 +1117,6 @@ public void testArrayGroupAsDoubleArray() @Test public void testArrayGroupAsFloatArray() { - notMsqCompatible(); // Cannot vectorize as we donot have support in native query subsytem for grouping on arrays as keys cannotVectorize(); testQuery( @@ -1612,7 +1607,6 @@ public void testArrayAggMultiValue() @Test public void testArrayAggNumeric() { - notMsqCompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_AGG(l1), ARRAY_AGG(DISTINCT l1), ARRAY_AGG(d1), ARRAY_AGG(DISTINCT d1), ARRAY_AGG(f1), ARRAY_AGG(DISTINCT f1) FROM numfoo", @@ -1749,7 +1743,6 @@ public void testArrayAggNumeric() @Test public void testArrayAggQuantile() { - notMsqCompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_QUANTILE(ARRAY_AGG(l1), 0.9) FROM numfoo", @@ -1793,7 +1786,9 @@ public void testArrayAggQuantile() @Test public void testArrayAggArrays() { - notMsqCompatible(); + // Produces nested array - ARRAY>, which frame writers don't support. A way to get this query + // to run would be to use nested columns. + msqIncompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_AGG(ARRAY[l1, l2]), ARRAY_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo", @@ -1890,7 +1885,6 @@ public void testArrayAggArrays() @Test public void testArrayConcatAggArrays() { - notMsqCompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_CONCAT_AGG(ARRAY[l1, l2]), ARRAY_CONCAT_AGG(DISTINCT ARRAY[l1, l2]) FROM numfoo", @@ -2039,7 +2033,6 @@ public void testArrayAggExpression() public void testArrayAggMaxBytes() { cannotVectorize(); - notMsqCompatible(); testQuery( "SELECT ARRAY_AGG(l1, 128), ARRAY_AGG(DISTINCT l1, 128) FROM numfoo", ImmutableList.of( @@ -2239,7 +2232,6 @@ public void testArrayAggGroupByArrayAggFromSubquery() @Test public void testArrayAggGroupByArrayAggOfLongsFromSubquery() { - notMsqCompatible(); requireMergeBuffers(3); cannotVectorize(); testQuery( @@ -2379,7 +2371,6 @@ public void testArrayAggGroupByArrayAggOfStringsFromSubquery() @Test public void testArrayAggGroupByArrayAggOfDoubleFromSubquery() { - notMsqCompatible(); requireMergeBuffers(3); cannotVectorize(); testQuery( @@ -2897,7 +2888,6 @@ public void testUnnestTwiceWithFiltersAndExpressions() @Test public void testUnnestThriceWithFiltersOnDimAndUnnestCol() { - notMsqCompatible(); cannotVectorize(); String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n" + " ( SELECT * FROM \n" @@ -2996,7 +2986,6 @@ public void testUnnestThriceWithFiltersOnDimAndUnnestCol() @Test public void testUnnestThriceWithFiltersOnDimAndAllUnnestColumns() { - notMsqCompatible(); cannotVectorize(); String sql = " SELECT dimZipf, dim3_unnest1, dim3_unnest2, dim3_unnest3 FROM \n" + " ( SELECT * FROM \n" diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java index bb1660f856a9..d1300ff19b24 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteJoinQueryTest.java @@ -695,7 +695,7 @@ public void testFilterAndGroupByLookupUsingJoinOperatorBackwards(Map queryContext) { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize JOIN operator. cannotVectorize(); @@ -1918,7 +1918,7 @@ public void testCommaJoinTableLookupTableMismatchedTypes(Map que // Regression test for https://github.com/apache/druid/issues/9646. // Empty-dataset aggregation queries in MSQ return an empty row, rather than a single row as SQL requires. - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize JOIN operator. cannotVectorize(); @@ -1974,7 +1974,7 @@ public void testCommaJoinTableLookupTableMismatchedTypes(Map que public void testJoinTableLookupTableMismatchedTypesWithoutComma(Map queryContext) { // Empty-dataset aggregation queries in MSQ return an empty row, rather than a single row as SQL requires. - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize JOIN operator. cannotVectorize(); @@ -2396,7 +2396,7 @@ public void testSelectOnLookupUsingLeftJoinOperator(Map queryCon public void testSelectOnLookupUsingRightJoinOperator(Map queryContext) { // MSQ refuses to do RIGHT join with broadcast. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim1, lookyloo.*\n" @@ -2438,7 +2438,7 @@ public void testSelectOnLookupUsingRightJoinOperator(Map queryCo public void testSelectOnLookupUsingFullJoinOperator(Map queryContext) { // MSQ refuses to do FULL join with broadcast. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim1, m1, cnt, lookyloo.*\n" @@ -3751,7 +3751,7 @@ public void testInnerJoinSubqueryWithSelectorFilter(Map queryCon { if (sortBasedJoin) { // Cannot handle the [l1.k = 'abc'] condition. - notMsqCompatible(); + msqIncompatible(); } // Cannot vectorize due to 'concat' expression. @@ -4172,7 +4172,7 @@ public void testSemiJoinWithOuterTimeExtractAggregateWithOrderBy() public void testJoinOnMultiValuedColumnShouldThrowException(Map queryContext) { // MSQ throws a slightly different error than QueryException. - notMsqCompatible(); + msqIncompatible(); final String query = "SELECT dim3, l.v from druid.foo f inner join lookup.lookyloo l on f.dim3 = l.k\n"; @@ -4189,7 +4189,7 @@ public void testJoinOnMultiValuedColumnShouldThrowException(Map public void testUnionAllTwoQueriesLeftQueryIsJoin(Map queryContext) { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); // Fully removing the join allows this query to vectorize. if (!isRewriteJoinToFilter(queryContext)) { @@ -4233,7 +4233,7 @@ public void testUnionAllTwoQueriesLeftQueryIsJoin(Map queryConte public void testUnionAllTwoQueriesRightQueryIsJoin(Map queryContext) { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); // Fully removing the join allows this query to vectorize. if (!isRewriteJoinToFilter(queryContext)) { @@ -4276,7 +4276,7 @@ public void testUnionAllTwoQueriesRightQueryIsJoin(Map queryCont public void testUnionAllTwoQueriesBothQueriesAreJoin() { // MSQ does not support UNION ALL. - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); testQuery( @@ -4982,7 +4982,7 @@ public void testVirtualColumnOnMVFilterJoinExpression(Map queryC // IndexedTableDimensionSelector. In native, this doesn't happen, because we don't have as much type information, // and we end up using IndexedTableColumnValueSelector instead. This is really a problem with // IndexedTableColumnSelectorFactory: it assumes strings are not multi-valued, even though they might be. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN druid.numfoo as foo2 " @@ -5038,7 +5038,7 @@ public void testVirtualColumnOnMVFilterMultiJoinExpression(Map q // IndexedTableDimensionSelector. In native, this doesn't happen, because we don't have as much type information, // and we end up using IndexedTableColumnValueSelector instead. This is really a problem with // IndexedTableColumnSelectorFactory: it assumes strings are not multi-valued, even though they might be. - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT foo1.dim3, foo2.dim3 FROM druid.numfoo as foo1 INNER JOIN " diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java index d440a6bb7182..5098343b5389 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteNestedDataQueryTest.java @@ -5667,7 +5667,7 @@ public void testScanAllTypesAuto() "1", "[]", "[51,-35]", - "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", + "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]},\"v\":[]}", "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", null, "[\"a\",\"b\"]", @@ -5705,7 +5705,7 @@ public void testScanAllTypesAuto() "\"b\"", "2", "b", - "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", + "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]},\"v\":[]}", "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}", "[\"a\",\"b\",\"c\"]", "[null,\"b\"]", @@ -5743,7 +5743,7 @@ public void testScanAllTypesAuto() "1", "1", "1", - "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", + "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]},\"v\":[]}", "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"a\",\"b\"]", "[\"a\",\"b\"]", @@ -5781,7 +5781,7 @@ public void testScanAllTypesAuto() "{}", "4", "1", - "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", + "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]},\"v\":[]}", "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"d\",\"e\"]", "[\"b\",\"b\"]", @@ -5819,7 +5819,7 @@ public void testScanAllTypesAuto() "{}", "[]", "hello", - "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", + "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]},\"v\":\"a\"}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, null, @@ -5857,7 +5857,7 @@ public void testScanAllTypesAuto() "\"a\"", "6", null, - "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", + "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]},\"v\":\"b\"}", null, "[\"a\",\"b\"]", null, @@ -5935,7 +5935,7 @@ public void testScanAllTypesAuto() "1", "[]", "[51,-35]", - "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]}}", + "{\"a\":700,\"b\":{\"x\":\"g\",\"y\":1.1,\"z\":[9,null,9,9]},\"v\":[]}", "{\"x\":400,\"y\":[{\"l\":[null],\"m\":100,\"n\":5},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{}}", null, "[\"a\",\"b\"]", @@ -5973,7 +5973,7 @@ public void testScanAllTypesAuto() "\"b\"", "2", "b", - "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]}}", + "{\"a\":200,\"b\":{\"x\":\"b\",\"y\":1.1,\"z\":[2,4,6]},\"v\":[]}", "{\"x\":10,\"y\":[{\"l\":[\"b\",\"b\",\"c\"],\"m\":\"b\",\"n\":2},[1,2,3]],\"z\":{\"a\":[5.5],\"b\":false}}", "[\"a\",\"b\",\"c\"]", "[null,\"b\"]", @@ -6011,7 +6011,7 @@ public void testScanAllTypesAuto() "1", "1", "1", - "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]}}", + "{\"a\":100,\"b\":{\"x\":\"a\",\"y\":1.1,\"z\":[1,2,3,4]},\"v\":[]}", "{\"x\":1234,\"y\":[{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1},{\"l\":[\"a\",\"b\",\"c\"],\"m\":\"a\",\"n\":1}],\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"a\",\"b\"]", "[\"a\",\"b\"]", @@ -6049,7 +6049,7 @@ public void testScanAllTypesAuto() "{}", "4", "1", - "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]}}", + "{\"a\":400,\"b\":{\"x\":\"d\",\"y\":1.1,\"z\":[3,4]},\"v\":[]}", "{\"x\":1234,\"z\":{\"a\":[1.1,2.2,3.3],\"b\":true}}", "[\"d\",\"e\"]", "[\"b\",\"b\"]", @@ -6087,7 +6087,7 @@ public void testScanAllTypesAuto() "{}", "[]", "hello", - "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]}}", + "{\"a\":500,\"b\":{\"x\":\"e\",\"z\":[1,2,3,4]},\"v\":\"a\"}", "{\"x\":11,\"y\":[],\"z\":{\"a\":[null],\"b\":false}}", null, null, @@ -6125,7 +6125,7 @@ public void testScanAllTypesAuto() "\"a\"", "6", null, - "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]}}", + "{\"a\":600,\"b\":{\"x\":\"f\",\"y\":1.1,\"z\":[6,7,8,9]},\"v\":\"b\"}", null, "[\"a\",\"b\"]", null, diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java index 35f85c5fbf2b..d67ed60c5527 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteQueryTest.java @@ -141,7 +141,7 @@ public class CalciteQueryTest extends BaseCalciteQueryTest @Test public void testInformationSchemaSchemata() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT DISTINCT SCHEMA_NAME FROM INFORMATION_SCHEMA.SCHEMATA", ImmutableList.of(), @@ -158,7 +158,7 @@ public void testInformationSchemaSchemata() @Test public void testInformationSchemaTables() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT TABLE_SCHEMA, TABLE_NAME, TABLE_TYPE, IS_JOINABLE, IS_BROADCAST\n" + "FROM INFORMATION_SCHEMA.TABLES\n" @@ -237,7 +237,7 @@ public void testInformationSchemaTables() @Test public void testInformationSchemaColumnsOnTable() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -259,7 +259,7 @@ public void testInformationSchemaColumnsOnTable() @Test public void testInformationSchemaColumnsOnForbiddenTable() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -290,7 +290,7 @@ public void testInformationSchemaColumnsOnForbiddenTable() @Test public void testInformationSchemaColumnsOnView() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -305,7 +305,7 @@ public void testInformationSchemaColumnsOnView() @Test public void testInformationSchemaColumnsOnAnotherView() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COLUMN_NAME, DATA_TYPE, IS_NULLABLE\n" + "FROM INFORMATION_SCHEMA.COLUMNS\n" @@ -322,7 +322,7 @@ public void testInformationSchemaColumnsOnAnotherView() @Test public void testCannotInsertWithNativeEngine() { - notMsqCompatible(); + msqIncompatible(); final DruidException e = Assert.assertThrows( DruidException.class, () -> testQuery( @@ -341,7 +341,7 @@ public void testCannotInsertWithNativeEngine() @Test public void testCannotReplaceWithNativeEngine() { - notMsqCompatible(); + msqIncompatible(); final DruidException e = Assert.assertThrows( DruidException.class, () -> testQuery( @@ -360,7 +360,7 @@ public void testCannotReplaceWithNativeEngine() @Test public void testAggregatorsOnInformationSchemaColumns() { - notMsqCompatible(); + msqIncompatible(); // Not including COUNT DISTINCT, since it isn't supported by BindableAggregate, and so it can't work. testQuery( "SELECT\n" @@ -381,7 +381,7 @@ public void testAggregatorsOnInformationSchemaColumns() @Test public void testFilterAggregatorFunctionsOnInformationSchemaRoutines() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + " COUNT(*)\n" @@ -397,7 +397,7 @@ public void testFilterAggregatorFunctionsOnInformationSchemaRoutines() @Test public void testFilterScalarFunctionsOnInformationSchemaRoutines() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + " COUNT(*)\n" @@ -413,7 +413,7 @@ public void testFilterScalarFunctionsOnInformationSchemaRoutines() @Test public void testNonExistentSchemaOnInformationSchemaRoutines() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT *\n" + "FROM INFORMATION_SCHEMA.ROUTINES\n" @@ -426,7 +426,7 @@ public void testNonExistentSchemaOnInformationSchemaRoutines() @Test public void testTopNLimitWrapping() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -631,7 +631,7 @@ public void testGroupBySingleColumnDescendingNoTopN() @Test public void testEarliestAggregators() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT " @@ -678,7 +678,7 @@ public void testEarliestAggregators() @Test public void testLatestVectorAggregators() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT " + "LATEST(cnt), LATEST(cnt + 1), LATEST(m1), LATEST(m1+1) " @@ -712,7 +712,7 @@ public void testLatestVectorAggregators() @Test public void testLatestAggregators() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT " @@ -857,7 +857,7 @@ public void testAnyAggregatorsOnHeapNumericNulls() @Test public void testLatestVectorAggregatorsOnTimeExpression() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT \n" + " LATEST_BY(m1, MILLIS_TO_TIMESTAMP(BITWISE_SHIFT_RIGHT(TIMESTAMP_TO_MILLIS(__time), 3)))\n" @@ -928,7 +928,7 @@ public void testAnyAggregatorsOffHeapNumericNulls() @Test public void testPrimitiveLatestInSubquery() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT SUM(val1), SUM(val2), SUM(val3) FROM (SELECT dim2, LATEST(m1) AS val1, LATEST(cnt) AS val2, LATEST(m2) AS val3 FROM foo GROUP BY dim2)", ImmutableList.of( @@ -976,7 +976,7 @@ public void testPrimitiveLatestInSubquery() @Test public void testPrimitiveLatestInSubqueryGroupBy() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, LATEST(m1) AS val1 FROM foo GROUP BY dim2", ImmutableList.of( @@ -1095,7 +1095,7 @@ public void testStringLatestByGroupByWithAlwaysFalseCondition() @Test public void testPrimitiveEarliestInSubquery() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT SUM(val1), SUM(val2), SUM(val3) FROM (SELECT dim2, EARLIEST(m1) AS val1, EARLIEST(cnt) AS val2, EARLIEST(m2) AS val3 FROM foo GROUP BY dim2)", @@ -1303,7 +1303,7 @@ public void testPrimitiveAnyInSubquery() @Test public void testStringEarliestSingleStringDim() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, EARLIEST(dim1,10) AS val FROM foo GROUP BY dim2", ImmutableList.of( @@ -1419,7 +1419,7 @@ public void testStringAnyInSubquery() @Test public void testEarliestAggregatorsNumericNulls() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT EARLIEST(l1), EARLIEST(d1), EARLIEST(f1) FROM druid.numfoo", @@ -1447,7 +1447,7 @@ public void testEarliestAggregatorsNumericNulls() @Test public void testLatestAggregatorsNumericNull() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT LATEST(l1), LATEST(d1), LATEST(f1) FROM druid.numfoo", ImmutableList.of( @@ -1478,7 +1478,7 @@ public void testLatestAggregatorsNumericNull() @Test public void testFirstLatestAggregatorsSkipNulls() { - notMsqCompatible(); + msqIncompatible(); final DimFilter filter; if (useDefault) { @@ -1592,7 +1592,7 @@ public void testAnyAggregatorsSkipNullsWithFilter() @Test public void testOrderByEarliestFloat() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { @@ -1639,7 +1639,7 @@ public void testOrderByEarliestFloat() @Test public void testOrderByEarliestDouble() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { @@ -1686,7 +1686,7 @@ public void testOrderByEarliestDouble() @Test public void testOrderByEarliestLong() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { @@ -1733,7 +1733,7 @@ public void testOrderByEarliestLong() @Test public void testOrderByLatestFloat() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -1780,7 +1780,7 @@ public void testOrderByLatestFloat() @Test public void testOrderByLatestDouble() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -1826,7 +1826,7 @@ public void testOrderByLatestDouble() @Test public void testOrderByLatestLong() { - notMsqCompatible(); + msqIncompatible(); List expected; if (NullHandling.replaceWithDefault()) { expected = ImmutableList.of( @@ -1884,7 +1884,7 @@ public void testOrderByAnyFloat() ); } else { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved - notMsqCompatible(); + msqIncompatible(); expected = ImmutableList.of( new Object[]{"2", 0.0f}, @@ -1935,7 +1935,7 @@ public void testOrderByAnyDouble() ); } else { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved - notMsqCompatible(); + msqIncompatible(); expected = ImmutableList.of( new Object[]{"2", 0.0}, @@ -1985,7 +1985,7 @@ public void testOrderByAnyLong() ); } else { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved - notMsqCompatible(); + msqIncompatible(); expected = ImmutableList.of( new Object[]{"2", 0L}, @@ -2066,7 +2066,7 @@ public void testGroupByOrdinal() @Ignore("Disabled since GROUP BY alias can confuse the validator; see DruidConformance::isGroupByAlias") public void testGroupByAndOrderByAlias() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT cnt AS theCnt, COUNT(*) FROM druid.foo GROUP BY theCnt ORDER BY theCnt ASC", ImmutableList.of( @@ -2449,7 +2449,7 @@ public void testHavingOnExactCountDistinct() @Test public void testExactCountDistinctWithFilter() { - notMsqCompatible(); + msqIncompatible(); final String sqlQuery = "SELECT COUNT(DISTINCT foo.dim1) FILTER(WHERE foo.cnt = 1), SUM(foo.cnt) FROM druid.foo"; // When useApproximateCountDistinct=false and useGroupingSetForExactDistinct=false, planning fails due // to a bug in the Calcite's rule (AggregateExpandDistinctAggregatesRule) @@ -2814,7 +2814,7 @@ public void testTopNWithSelectAndOrderByProjections() @Test public void testUnionAllQueries() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo", ImmutableList.of( @@ -2848,7 +2848,7 @@ public void testUnionAllQueries() @Test public void testUnionAllQueriesWithLimit() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT * FROM (" + "SELECT COUNT(*) FROM foo UNION ALL SELECT SUM(cnt) FROM foo UNION ALL SELECT COUNT(*) FROM foo" @@ -2877,7 +2877,7 @@ public void testUnionAllQueriesWithLimit() @Test public void testUnionAllDifferentTablesWithMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -2921,7 +2921,7 @@ public void testUnionAllDifferentTablesWithMapping() @Test public void testJoinUnionAllDifferentTablesWithMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -2985,7 +2985,7 @@ public void testUnionAllTablesColumnCountMismatch() @Test public void testUnionAllTablesColumnTypeMismatchFloatLong() { - notMsqCompatible(); + msqIncompatible(); // "m1" has a different type in foo and foo2 (float vs long), but this query is OK anyway because they can both // be implicitly cast to double. @@ -3095,7 +3095,7 @@ public void testUnionAllTablesWhenCastAndMappingIsRequired() @Test public void testUnionAllSameTableTwice() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3139,7 +3139,7 @@ public void testUnionAllSameTableTwice() @Test public void testUnionAllSameTableTwiceWithSameMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3197,7 +3197,7 @@ public void testUnionAllSameTableTwiceWithDifferentMapping() @Test public void testUnionAllSameTableThreeTimes() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3302,7 +3302,7 @@ public void testUnionAllThreeTablesColumnCountMismatch3() @Test public void testUnionAllSameTableThreeTimesWithSameMapping() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + "dim1, dim2, SUM(m1), COUNT(*)\n" @@ -3673,7 +3673,7 @@ public void testNullDoubleTopN() { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved if (NullHandling.sqlCompatible()) { - notMsqCompatible(); + msqIncompatible(); } List expected; if (useDefault) { @@ -3717,7 +3717,7 @@ public void testNullFloatTopN() { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved if (NullHandling.sqlCompatible()) { - notMsqCompatible(); + msqIncompatible(); } List expected; if (useDefault) { @@ -3761,7 +3761,7 @@ public void testNullLongTopN() { // Disabled test in MSQ with SQL-compatible mode till https://github.com/apache/druid/issues/13951 is resolved if (NullHandling.sqlCompatible()) { - notMsqCompatible(); + msqIncompatible(); } List expected; if (useDefault) { @@ -4199,7 +4199,7 @@ public void testColumnIsNull() @Test public void testGroupingWithNullInFilter() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM foo WHERE dim1 IN (NULL)", ImmutableList.of( @@ -4224,7 +4224,7 @@ public void testGroupingWithNullInFilter() @Test public void testGroupByNothingWithLiterallyFalseFilter() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE 1 = 0", ImmutableList.of( @@ -4250,7 +4250,7 @@ public void testGroupByNothingWithLiterallyFalseFilter() @Test public void testGroupByNothingWithImpossibleTimeFilter() { - notMsqCompatible(); + msqIncompatible(); // Regression test for https://github.com/apache/druid/issues/7671 testQuery( @@ -4321,7 +4321,7 @@ public void testGroupByOneColumnWithLiterallyFalseFilter() @Test public void testGroupByWithFilterMatchingNothing() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*), MAX(cnt) FROM druid.foo WHERE dim1 = 'foobar'", ImmutableList.of( @@ -4669,7 +4669,7 @@ public void testCountStarWithLongColumnFiltersForceRange() @Test public void testCountStarWithLongColumnFiltersOnFloatLiterals() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM druid.foo WHERE cnt > 1.1 and cnt < 100000001.0", ImmutableList.of( @@ -5659,7 +5659,7 @@ public void testCountStarWithDegenerateFilter() public void testCountStarWithNotOfDegenerateFilter() { - notMsqCompatible(); + msqIncompatible(); // HashJoinSegmentStorageAdapter is not vectorizable cannotVectorize(); @@ -5689,7 +5689,7 @@ public void testCountStarWithNotOfDegenerateFilter() @Test public void testUnplannableQueries() { - notMsqCompatible(); + msqIncompatible(); // All of these queries are unplannable because they rely on features Druid doesn't support. // This test is here to confirm that we don't fall back to Calcite's interpreter or enumerable implementation. // It's also here so when we do support these features, we can have "real" tests for these queries. @@ -5798,7 +5798,7 @@ public void testIsNotDistinctFromLiteral() @Test public void testArrayAggQueryOnComplexDatatypes() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); testQuery( "SELECT ARRAY_AGG(unique_dim1) FROM druid.foo", @@ -5961,7 +5961,7 @@ public void testCountStarWithTimeInIntervalFilterLosAngeles() @Test public void testCountStarWithTimeInIntervalFilterInvalidInterval() { - notMsqCompatible(); + msqIncompatible(); testQueryThrows( "SELECT COUNT(*) FROM druid.foo " + "WHERE TIME_IN_INTERVAL(__time, '2000-01-01/X')", @@ -5978,7 +5978,7 @@ public void testCountStarWithTimeInIntervalFilterInvalidInterval() @Test public void testCountStarWithTimeInIntervalFilterNonLiteral() { - notMsqCompatible(); + msqIncompatible(); testQueryThrows( "SELECT COUNT(*) FROM druid.foo " + "WHERE TIME_IN_INTERVAL(__time, dim1)", @@ -6559,7 +6559,7 @@ public void testTimeseriesWithTimeFilterOnLongColumnUsingMillisToTimestamp() @Test public void testCountDistinct() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT SUM(cnt), COUNT(distinct dim2), COUNT(distinct unique_dim1) FROM druid.foo", ImmutableList.of( @@ -6684,7 +6684,7 @@ public void testApproxCountDistinctWhenHllDisabled() { if (NullHandling.sqlCompatible()) { // Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950 - notMsqCompatible(); + msqIncompatible(); } // When HLL is disabled, APPROX_COUNT_DISTINCT is still approximate. @@ -6722,7 +6722,7 @@ public void testApproxCountDistinctBuiltin() { if (NullHandling.sqlCompatible()) { // Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950 - notMsqCompatible(); + msqIncompatible(); } testQuery( @@ -6921,7 +6921,7 @@ public void testMultipleExactCountDistinctWithGroupingAndOtherAggregatorsUsingJo @Test public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets() { - notMsqCompatible(); + msqIncompatible(); requireMergeBuffers(4); testQuery( PLANNER_CONFIG_NO_HLL.withOverrides( @@ -7004,7 +7004,7 @@ public void testMultipleExactCountDistinctWithGroupingUsingGroupingSets() @Test public void testApproxCountDistinct() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -7333,7 +7333,7 @@ public void testExactCountDistinctUsingSubquery() @Test public void testExactCountDistinctUsingSubqueryOnUnionAllTables() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT\n" + " SUM(cnt),\n" @@ -7485,7 +7485,7 @@ public void testQueryWithMoreThanMaxNumericInFilter() // skip in sql compatible mode, this plans to an OR filter with equality filter children... return; } - notMsqCompatible(); + msqIncompatible(); expectedException.expect(UOE.class); expectedException.expectMessage( "The number of values in the IN clause for [dim6] in query exceeds configured maxNumericFilter limit of [2] for INs. Cast [3] values of IN clause to String"); @@ -7746,7 +7746,7 @@ public void testCountDistinctArithmetic() { if (NullHandling.sqlCompatible()) { // Empty string isn't counted properly; see https://github.com/apache/druid/issues/13950 - notMsqCompatible(); + msqIncompatible(); } testQuery( @@ -8485,7 +8485,7 @@ public void testFilterOnTimeExtractWithMultipleDays() @Test public void testFilterOnTimeExtractWithVariousTimeUnits() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -8548,7 +8548,7 @@ public void testFilterOnTimeExtractWithVariousTimeUnits() @Test public void testFilterOnTimeFloorMisaligned() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT COUNT(*) FROM druid.foo " + "WHERE floor(__time TO month) = TIMESTAMP '2000-01-01 00:00:01'", @@ -8595,7 +8595,7 @@ public void testGroupByFloor() @Test public void testQueryWithSelectProjectAndIdentityProjectDoesNotRename() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); skipVectorize(); requireMergeBuffers(3); @@ -9068,7 +9068,7 @@ public void testCountDistinctOfLookup() @Test public void testGroupByExpressionFromLookup() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize direct queries on lookup tables. cannotVectorize(); @@ -9454,7 +9454,7 @@ public void testTimeseriesLosAngelesUsingTimeFloorConnectionLosAngeles() @Test public void testTimeseriesDontSkipEmptyBuckets() { - notMsqCompatible(); + msqIncompatible(); // Tests that query context parameters are passed through to the underlying query engine. Long defaultVal = NullHandling.replaceWithDefault() ? 0L : null; testQuery( @@ -9594,7 +9594,7 @@ public void testTimeseriesDescending() @Test public void testTimeseriesEmptyResultsAggregatorDefaultValues() { - notMsqCompatible(); + msqIncompatible(); // timeseries with all granularity have a single group, so should return default results for given aggregators testQuery( "SELECT\n" @@ -9700,7 +9700,7 @@ public void testTimeseriesEmptyResultsAggregatorDefaultValues() @Test public void testTimeseriesEmptyResultsAggregatorDefaultValuesNonVectorized() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); skipVectorize(); // timeseries with all granularity have a single group, so should return default results for given aggregators @@ -10016,7 +10016,7 @@ public void testGroupByAggregatorDefaultValues() @Test public void testGroupByAggregatorDefaultValuesNonVectorized() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); skipVectorize(); testQuery( @@ -10641,7 +10641,7 @@ public void testGroupByTimeFloorAndDimOnGroupByTimeFloorAndDim() @Test public void testGroupingSets() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -10706,7 +10706,7 @@ public void testGroupingSets() @Test public void testGroupingAggregatorDifferentOrder() { - notMsqCompatible(); + msqIncompatible(); requireMergeBuffers(3); testQuery( @@ -10770,7 +10770,7 @@ public void testGroupingAggregatorDifferentOrder() @Test public void testGroupingAggregatorWithPostAggregator() { - notMsqCompatible(); + msqIncompatible(); List resultList; if (NullHandling.sqlCompatible()) { resultList = ImmutableList.of( @@ -10829,7 +10829,7 @@ public void testGroupingAggregatorWithPostAggregator() @Test public void testGroupingSetsWithNumericDimension() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT cnt, COUNT(*)\n" + "FROM foo\n" @@ -10860,7 +10860,7 @@ public void testGroupingSetsWithNumericDimension() @Test public void testGroupByRollup() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -10919,7 +10919,7 @@ public void testGroupByRollup() @Test public void testGroupByRollupDifferentOrder() { - notMsqCompatible(); + msqIncompatible(); // Like "testGroupByRollup", but the ROLLUP exprs are in the reverse order. testQuery( "SELECT dim2, gran, SUM(cnt)\n" @@ -10975,7 +10975,7 @@ public void testGroupByRollupDifferentOrder() @Test public void testGroupByCube() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11037,7 +11037,7 @@ public void testGroupByCube() @Test public void testGroupingSetsWithDummyDimension() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11099,7 +11099,7 @@ public void testGroupingSetsWithDummyDimension() @Test public void testGroupingSetsNoSuperset() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11156,7 +11156,7 @@ public void testGroupingSetsNoSuperset() @Test public void testGroupingSetsWithOrderByDimension() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, gran, SUM(cnt)\n" + "FROM (SELECT FLOOR(__time TO MONTH) AS gran, COALESCE(dim2, '') dim2, cnt FROM druid.foo) AS x\n" @@ -11227,7 +11227,7 @@ public void testGroupingSetsWithOrderByDimension() @Test public void testGroupingSetsWithOrderByAggregator() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11296,7 +11296,7 @@ public void testGroupingSetsWithOrderByAggregator() @Test public void testGroupingSetsWithOrderByAggregatorWithLimit() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -11562,7 +11562,7 @@ public void testProjectAfterSort2() + " org.apache.calcite.sql.validate.SqlValidatorException: Column 'dim1' is ambiguous") public void testProjectAfterSort3() { - notMsqCompatible(); + msqIncompatible(); testQuery( "select dim1 from (select dim1, dim1, count(*) cnt from druid.foo group by dim1, dim1 order by cnt)", ImmutableList.of( @@ -11601,7 +11601,7 @@ public void testProjectAfterSort3() @Test public void testProjectAfterSort3WithoutAmbiguity() { - notMsqCompatible(); + msqIncompatible(); // This query is equivalent to the one in testProjectAfterSort3 but renames the second grouping column // to avoid the ambiguous name exception. The inner sort is also optimized out in Calcite 1.21. testQuery( @@ -12150,7 +12150,7 @@ public void testRequireTimeConditionLogicalValuePositive() @Test public void testRequireTimeConditionSimpleQueryNegative() { - notMsqCompatible(); + msqIncompatible(); expectedException.expect(CannotBuildQueryException.class); expectedException.expectMessage("__time column"); @@ -12171,7 +12171,7 @@ public void testRequireTimeConditionSimpleQueryNegative() @Test public void testRequireTimeConditionSubQueryNegative() { - notMsqCompatible(); + msqIncompatible(); expectedException.expect(CannotBuildQueryException.class); expectedException.expectMessage("__time column"); @@ -12191,7 +12191,7 @@ public void testRequireTimeConditionSubQueryNegative() @Test public void testRequireTimeConditionSemiJoinNegative() { - notMsqCompatible(); + msqIncompatible(); expectedException.expect(CannotBuildQueryException.class); expectedException.expectMessage("__time column"); @@ -12277,7 +12277,7 @@ public void testFilterLongDimension() @Test public void testTrigonometricFunction() { - notMsqCompatible(); + msqIncompatible(); testQuery( PLANNER_CONFIG_DEFAULT, QUERY_CONTEXT_DEFAULT, @@ -12576,7 +12576,7 @@ public void testLeftRightStringOperators() @Test public void testQueryContextOuterLimit() { - notMsqCompatible(); + msqIncompatible(); Map outerLimitContext = new HashMap<>(QUERY_CONTEXT_DEFAULT); outerLimitContext.put(PlannerContext.CTX_SQL_OUTER_LIMIT, 4); @@ -12887,7 +12887,7 @@ public void testTimeStampAddConversion() @Test public void testGroupingSetsWithLimit() { - notMsqCompatible(); + msqIncompatible(); testQuery( "SELECT dim2, gran, SUM(cnt)\n" + "FROM (SELECT FLOOR(__time TO MONTH) AS gran, COALESCE(dim2, '') dim2, cnt FROM druid.foo) AS x\n" @@ -12952,7 +12952,7 @@ public void testGroupingSetsWithLimit() @Test public void testGroupingSetsWithLimitOrderByGran() { - notMsqCompatible(); + msqIncompatible(); // Cannot vectorize due to virtual columns. cannotVectorize(); @@ -14276,7 +14276,7 @@ public void testGreatestFunctionForNumberWithIsNull() @Test public void testGreatestFunctionForStringWithIsNull() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); String query = "SELECT l1, LATEST(GREATEST(dim1, dim2)) IS NULL FROM druid.numfoo GROUP BY l1"; @@ -14367,7 +14367,7 @@ public void testSubqueryTypeMismatchWithLiterals() public void testTimeseriesQueryWithEmptyInlineDatasourceAndGranularity() { // TODO(gianm): this test does not actually test the below thing, b/c the timestamp_floor got baked in - notMsqCompatible(); + msqIncompatible(); //msqCompatible(); // the SQL query contains an always FALSE filter ('bar' = 'baz'), which optimizes the query to also remove time @@ -14457,7 +14457,7 @@ public void testComplexDecode() @Test public void testComplexDecodeAgg() { - notMsqCompatible(); + msqIncompatible(); cannotVectorize(); testQuery( "SELECT APPROX_COUNT_DISTINCT_BUILTIN(COMPLEX_DECODE_BASE64('hyperUnique',PARSE_JSON(TO_JSON_STRING(unique_dim1)))) from druid.foo", diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java index d39c9bf1388e..2ddc674eadda 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteSubqueryTest.java @@ -57,7 +57,6 @@ import org.apache.druid.sql.calcite.util.CalciteTests; import org.joda.time.DateTimeZone; import org.joda.time.Period; -import org.junit.Ignore; import org.junit.Test; import org.junit.runner.RunWith; import org.junit.runners.Parameterized; @@ -227,7 +226,6 @@ public void testExactCountDistinctOfSemiJoinResult() ); } - @Ignore("Merge buffers exceed the prescribed limit when the results are materialized as frames") @Test public void testTwoExactCountDistincts() { diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java index 963e1e0b23bc..f9d82e71dd7e 100644 --- a/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java +++ b/sql/src/test/java/org/apache/druid/sql/calcite/QueryTestRunner.java @@ -321,10 +321,15 @@ public static Pair> getResults( public static class VerifyResults implements QueryVerifyStep { protected final BaseExecuteQuery execStep; + protected final boolean verifyRowSignature; - public VerifyResults(BaseExecuteQuery execStep) + public VerifyResults( + BaseExecuteQuery execStep, + boolean verifyRowSignature + ) { this.execStep = execStep; + this.verifyRowSignature = verifyRowSignature; } @Override @@ -346,7 +351,9 @@ private void verifyResults(QueryResults queryResults) } QueryTestBuilder builder = execStep.builder(); - builder.expectedResultsVerifier.verifyRowSignature(queryResults.signature); + if (verifyRowSignature) { + builder.expectedResultsVerifier.verifyRowSignature(queryResults.signature); + } builder.expectedResultsVerifier.verify(builder.sql, results); } } @@ -667,7 +674,9 @@ public QueryTestRunner(QueryTestBuilder builder) verifySteps.add(new VerifyNativeQueries(finalExecStep)); } if (builder.expectedResultsVerifier != null) { - verifySteps.add(new VerifyResults(finalExecStep)); + // Don't verify the row signature when MSQ is running, since the broker receives the task id, and the signature + // would be {TASK:STRING} instead of the expected results signature + verifySteps.add(new VerifyResults(finalExecStep, !config.isRunningMSQ())); } if (!builder.customVerifications.isEmpty()) {