diff --git a/.github/workflows/reusable-unit-tests.yml b/.github/workflows/reusable-unit-tests.yml
index 291cb648b3e..3634da0e200 100644
--- a/.github/workflows/reusable-unit-tests.yml
+++ b/.github/workflows/reusable-unit-tests.yml
@@ -117,6 +117,28 @@ jobs:
MAVEN_PROJECTS: ${{ inputs.maven_projects }}
run: ./.github/scripts/unit_tests_script.sh
+ - name: Check for .hprof files on failure
+ if: ${{ failure() }}
+ id: check_for_heap_dump
+ run: |
+ if ls ${GITHUB_WORKSPACE}/target/*.hprof 1> /dev/null 2>&1; then
+ echo "found_hprof=true" >> "$GITHUB_ENV"
+ else
+ echo "found_hprof=false" >> "$GITHUB_ENV"
+ fi
+
+ - name: Collect tarball hprof dumps if they exist on failure
+ if: ${{ failure() && env.found_hprof == 'true' }}
+ run: |
+ tar cvzf ${RUNNER_TEMP}/hprof-dumps.tgz ${GITHUB_WORKSPACE}/target/*.hprof
+
+ - name: Upload hprof dumps to GitHub if they exist on failure
+ if: ${{ failure() && env.found_hprof == 'true' }}
+ uses: actions/upload-artifact@master
+ with:
+ name: Hprof-${{ inputs.group }} hprof dumps (Compile=jdk${{ inputs.build_jdk }}, Run=jdk${{ inputs.runtime_jdk }})
+ path: ${{ runner.temp }}/hprof-dumps.tgz
+
- name: set outputs on failure
id: set_outputs
if: ${{ failure() }}
diff --git a/benchmarks/pom.xml b/benchmarks/pom.xml
index 84b2ddf0d13..f2ffbe247ec 100644
--- a/benchmarks/pom.xml
+++ b/benchmarks/pom.xml
@@ -27,7 +27,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java
index cabd58a4b7e..008dcd9600a 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/DelimitedInputFormatBenchmark.java
@@ -132,7 +132,7 @@ public class DelimitedInputFormatBenchmark
@Setup(Level.Trial)
public void prepareFormat()
{
- format = new DelimitedInputFormat(fromHeader ? null : COLUMNS, null, "\t", null, fromHeader, fromHeader ? 0 : 1);
+ format = new DelimitedInputFormat(fromHeader ? null : COLUMNS, null, "\t", null, fromHeader, fromHeader ? 0 : 1, null);
}
@Benchmark
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
index d65d13a324d..cda80475f53 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionAggregationBenchmark.java
@@ -36,7 +36,7 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.QueryableIndex;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
+import org.apache.druid.segment.QueryableIndexCursorFactory;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.generator.GeneratorColumnSchema;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
@@ -161,9 +161,8 @@ public class ExpressionAggregationBenchmark
private double compute(final Function aggregatorFactory)
{
- final QueryableIndexStorageAdapter adapter = new QueryableIndexStorageAdapter(index);
-
- try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
final BufferAggregator bufferAggregator = aggregatorFactory.apply(cursor.getColumnSelectorFactory());
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java
index 06f0952b51f..6672edc5c42 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionFilterBenchmark.java
@@ -35,7 +35,7 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.QueryableIndex;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
+import org.apache.druid.segment.QueryableIndexCursorFactory;
import org.apache.druid.segment.column.ValueType;
import org.apache.druid.segment.generator.GeneratorColumnSchema;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
@@ -148,7 +148,9 @@ public class ExpressionFilterBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(expressionFilter.toFilter())
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
@@ -166,7 +168,9 @@ public class ExpressionFilterBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(nativeFilter.toFilter())
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("x");
while (!cursor.isDone()) {
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
index 8d79a611d0e..f4dce839cd5 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionSelectorBenchmark.java
@@ -42,9 +42,8 @@ import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.Cursors;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.QueryableIndex;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
+import org.apache.druid.segment.QueryableIndexCursorFactory;
import org.apache.druid.segment.QueryableIndexTimeBoundaryInspector;
-import org.apache.druid.segment.StorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.column.ColumnType;
@@ -160,7 +159,8 @@ public class ExpressionSelectorBenchmark
)
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
@@ -171,7 +171,8 @@ public class ExpressionSelectorBenchmark
@Benchmark
public void timeFloorUsingExtractionFn(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
final DimensionSelector selector = cursor
@@ -190,15 +191,15 @@ public class ExpressionSelectorBenchmark
@Benchmark
public void timeFloorUsingCursor(Blackhole blackhole)
{
- final StorageAdapter adapter = new QueryableIndexStorageAdapter(index);
- try (final CursorHolder cursorHolder = adapter.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
final CursorGranularizer granularizer = CursorGranularizer.create(
cursor,
QueryableIndexTimeBoundaryInspector.create(index),
Cursors.getTimeOrdering(index.getOrdering()),
Granularities.HOUR,
- adapter.getInterval()
+ index.getDataInterval()
);
final Sequence results =
Sequences.simple(granularizer.getBucketIterable())
@@ -241,7 +242,8 @@ public class ExpressionSelectorBenchmark
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final DimensionSelector selector = cursor.getColumnSelectorFactory().makeDimensionSelector(
DefaultDimensionSpec.of("v")
@@ -253,7 +255,8 @@ public class ExpressionSelectorBenchmark
@Benchmark
public void timeFormatUsingExtractionFn(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
final DimensionSelector selector = cursor
.getColumnSelectorFactory()
@@ -284,7 +287,8 @@ public class ExpressionSelectorBenchmark
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
@@ -307,7 +311,8 @@ public class ExpressionSelectorBenchmark
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final DimensionSelector selector = cursor
.getColumnSelectorFactory()
@@ -320,7 +325,8 @@ public class ExpressionSelectorBenchmark
@Benchmark
public void strlenUsingExtractionFn(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
final DimensionSelector selector = cursor
.getColumnSelectorFactory()
@@ -346,7 +352,8 @@ public class ExpressionSelectorBenchmark
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
@@ -368,7 +375,8 @@ public class ExpressionSelectorBenchmark
)
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
@@ -390,7 +398,8 @@ public class ExpressionSelectorBenchmark
)
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
@@ -412,7 +421,8 @@ public class ExpressionSelectorBenchmark
)
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
@@ -447,7 +457,8 @@ public class ExpressionSelectorBenchmark
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
@@ -476,7 +487,8 @@ public class ExpressionSelectorBenchmark
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
@@ -513,7 +525,8 @@ public class ExpressionSelectorBenchmark
)
.build();
- try (final CursorHolder cursorHolder = new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(index);
+ try (final CursorHolder cursorHolder = cursorFactory.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
final ColumnValueSelector selector = cursor.getColumnSelectorFactory().makeColumnValueSelector("v");
consumeLong(cursor, selector, blackhole);
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java
index 5604f7fe883..2e9ffb644d5 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/ExpressionVectorSelectorBenchmark.java
@@ -35,7 +35,7 @@ import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.QueryableIndex;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
+import org.apache.druid.segment.QueryableIndexCursorFactory;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
@@ -155,7 +155,7 @@ public class ExpressionVectorSelectorBenchmark
.setVirtualColumns(virtualColumns)
.build();
final CursorHolder cursorHolder = closer.register(
- new QueryableIndexStorageAdapter(index).makeCursorHolder(buildSpec)
+ new QueryableIndexCursorFactory(index).makeCursorHolder(buildSpec)
);
if (vectorize) {
VectorCursor cursor = cursorHolder.asVectorCursor();
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java
index 65dde3cb65a..2c0f2690198 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/FilterPartitionBenchmark.java
@@ -46,14 +46,14 @@ import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.segment.BaseLongColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
-import org.apache.druid.segment.StorageAdapter;
+import org.apache.druid.segment.QueryableIndexCursorFactory;
import org.apache.druid.segment.column.ColumnConfig;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.data.IndexedInts;
@@ -231,8 +231,8 @@ public class FilterPartitionBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void stringRead(Blackhole blackhole)
{
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, null)) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -243,8 +243,8 @@ public class FilterPartitionBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void longRead(Blackhole blackhole)
{
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, null)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, null)) {
final Cursor cursor = cursorHolder.asCursor();
readCursorLong(cursor, blackhole);
}
@@ -255,8 +255,8 @@ public class FilterPartitionBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void timeFilterNone(Blackhole blackhole)
{
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterNone)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterNone)) {
final Cursor cursor = cursorHolder.asCursor();
readCursorLong(cursor, blackhole);
}
@@ -267,8 +267,8 @@ public class FilterPartitionBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void timeFilterHalf(Blackhole blackhole)
{
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterHalf)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterHalf)) {
final Cursor cursor = cursorHolder.asCursor();
readCursorLong(cursor, blackhole);
}
@@ -279,8 +279,8 @@ public class FilterPartitionBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void timeFilterAll(Blackhole blackhole)
{
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, timeFilterAll)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, timeFilterAll)) {
final Cursor cursor = cursorHolder.asCursor();
readCursorLong(cursor, blackhole);
}
@@ -293,8 +293,8 @@ public class FilterPartitionBenchmark
{
Filter filter = new SelectorFilter("dimSequential", "199");
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -307,8 +307,8 @@ public class FilterPartitionBenchmark
{
Filter filter = new NoBitmapSelectorFilter("dimSequential", "199");
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -321,8 +321,8 @@ public class FilterPartitionBenchmark
{
Filter filter = new SelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter();
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -335,8 +335,8 @@ public class FilterPartitionBenchmark
{
Filter filter = new NoBitmapSelectorDimFilter("dimSequential", "super-199", JS_EXTRACTION_FN).toFilter();
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, filter)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, filter)) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -354,8 +354,8 @@ public class FilterPartitionBenchmark
)
);
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, andFilter)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, andFilter)) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -370,8 +370,8 @@ public class FilterPartitionBenchmark
Filter filter2 = new AndFilter(Arrays.asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar")));
Filter orFilter = new OrFilter(Arrays.asList(filter, filter2));
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, orFilter)) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, orFilter)) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -386,8 +386,8 @@ public class FilterPartitionBenchmark
Filter filter2 = new AndFilter(Arrays.asList(new SelectorFilter("dimMultivalEnumerated2", "Corundum"), new NoBitmapSelectorFilter("dimMultivalEnumerated", "Bar")));
Filter orFilter = new OrFilter(Arrays.asList(filter, filter2));
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(orFilter))) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, Filters.toCnf(orFilter))) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -425,8 +425,8 @@ public class FilterPartitionBenchmark
))
);
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, dimFilter3.toFilter())) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, dimFilter3.toFilter())) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
@@ -464,16 +464,16 @@ public class FilterPartitionBenchmark
))
);
- StorageAdapter sa = new QueryableIndexStorageAdapter(qIndex);
- try (final CursorHolder cursorHolder = makeCursorHolder(sa, Filters.toCnf(dimFilter3.toFilter()))) {
+ final QueryableIndexCursorFactory cursorFactory = new QueryableIndexCursorFactory(qIndex);
+ try (final CursorHolder cursorHolder = makeCursorHolder(cursorFactory, Filters.toCnf(dimFilter3.toFilter()))) {
final Cursor cursor = cursorHolder.asCursor();
readCursor(cursor, blackhole);
}
}
- private CursorHolder makeCursorHolder(StorageAdapter sa, Filter filter)
+ private CursorHolder makeCursorHolder(CursorFactory factory, Filter filter)
{
- return sa.makeCursorHolder(
+ return factory.makeCursorHolder(
CursorBuildSpec.builder()
.setFilter(filter)
.setInterval(schemaInfo.getDataInterval())
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
index 95d59856395..bbff131e867 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/GroupByTypeInterfaceBenchmark.java
@@ -378,7 +378,6 @@ public class GroupByTypeInterfaceBenchmark
final GroupingEngine groupingEngine = new GroupingEngine(
druidProcessingConfig,
configSupplier,
- bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@@ -387,7 +386,8 @@ public class GroupByTypeInterfaceBenchmark
factory = new GroupByQueryRunnerFactory(
groupingEngine,
- new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool)
+ new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool),
+ bufferPool
);
}
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java
index 952e9b188fe..e654b033dee 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/IndexedTableJoinCursorBenchmark.java
@@ -238,7 +238,7 @@ public class IndexedTableJoinCursorBenchmark
private CursorHolder makeCursorHolder()
{
- return hashJoinSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN);
+ return hashJoinSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN);
}
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
index c66ef6baf0a..d9a98b467d8 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/JoinAndLookupBenchmark.java
@@ -369,7 +369,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void baseSegment(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
+ try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "countryIsoCode"));
}
@@ -384,7 +384,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) {
+ try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "countryIsoCode"));
}
@@ -395,7 +395,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinLookupStringKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -411,7 +411,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -423,7 +423,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinLookupLongKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -439,7 +439,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.v"));
@@ -451,7 +451,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinIndexedTableLongKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -467,7 +467,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -479,7 +479,7 @@ public class JoinAndLookupBenchmark
@OutputTimeUnit(TimeUnit.MILLISECONDS)
public void joinIndexedTableStringKey(Blackhole blackhole)
{
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(CursorBuildSpec.FULL_SCAN)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -495,7 +495,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, "c.countryName"));
@@ -510,7 +510,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setVirtualColumns(lookupVirtualColumns)
.build();
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME));
@@ -527,7 +527,7 @@ public class JoinAndLookupBenchmark
.setFilter(filter)
.setVirtualColumns(lookupVirtualColumns)
.build();
- try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asStorageAdapter()
+ try (final CursorHolder cursorHolder = hashJoinLookupStringKeySegment.asCursorFactory()
.makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_CODE_TO_NAME));
@@ -542,7 +542,7 @@ public class JoinAndLookupBenchmark
final CursorBuildSpec buildSpec = CursorBuildSpec.builder()
.setVirtualColumns(lookupVirtualColumns)
.build();
- try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) {
+ try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME));
}
@@ -558,7 +558,7 @@ public class JoinAndLookupBenchmark
.setVirtualColumns(lookupVirtualColumns)
.setFilter(filter)
.build();
- try (final CursorHolder cursorHolder = baseSegment.asStorageAdapter().makeCursorHolder(buildSpec)) {
+ try (final CursorHolder cursorHolder = baseSegment.asCursorFactory().makeCursorHolder(buildSpec)) {
final Cursor cursor = cursorHolder.asCursor();
blackhole.consume(getLastValue(cursor, LOOKUP_COUNTRY_NUMBER_TO_NAME));
}
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java
index e059c8a7ea7..e76cfc5b425 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/compression/LongCompressionBenchmark.java
@@ -23,7 +23,7 @@ import com.google.common.base.Supplier;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.MappedByteBufferHandler;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
+import org.apache.druid.query.QueryContexts;
import org.apache.druid.segment.data.ColumnarLongs;
import org.apache.druid.segment.data.CompressedColumnarLongsSupplier;
import org.openjdk.jmh.annotations.Benchmark;
@@ -118,7 +118,7 @@ public class LongCompressionBenchmark
@Benchmark
public void readVectorizedContinuous(Blackhole bh)
{
- long[] vector = new long[QueryableIndexStorageAdapter.DEFAULT_VECTOR_SIZE];
+ long[] vector = new long[QueryContexts.DEFAULT_VECTOR_SIZE];
ColumnarLongs columnarLongs = supplier.get();
int count = columnarLongs.size();
for (int i = 0; i < count; i++) {
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java
index a8b90e56eea..25f9015de2b 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/frame/FrameChannelMergerBenchmark.java
@@ -21,6 +21,7 @@ package org.apache.druid.benchmark.frame;
import com.google.common.collect.ImmutableList;
import com.google.common.util.concurrent.ListenableFuture;
+import com.google.common.util.concurrent.ListeningExecutorService;
import com.google.common.util.concurrent.MoreExecutors;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.common.guava.FutureUtils;
@@ -203,6 +204,7 @@ public class FrameChannelMergerBenchmark
private final List sortKey = ImmutableList.of(new KeyColumn(KEY, KeyOrder.ASCENDING));
private List> channelFrames;
+ private ListeningExecutorService innerExec;
private FrameProcessorExecutor exec;
private List channels;
@@ -226,7 +228,7 @@ public class FrameChannelMergerBenchmark
frameReader = FrameReader.create(signature);
exec = new FrameProcessorExecutor(
- MoreExecutors.listeningDecorator(
+ innerExec = MoreExecutors.listeningDecorator(
Execs.singleThreaded(StringUtils.encodeForFormat(getClass().getSimpleName()))
)
);
@@ -284,7 +286,7 @@ public class FrameChannelMergerBenchmark
signature
);
final Sequence frameSequence =
- FrameSequenceBuilder.fromAdapter(segment.asStorageAdapter())
+ FrameSequenceBuilder.fromCursorFactory(segment.asCursorFactory())
.allocator(ArenaMemoryAllocator.createOnHeap(10_000_000))
.frameType(FrameType.ROW_BASED)
.frames();
@@ -335,8 +337,8 @@ public class FrameChannelMergerBenchmark
@TearDown(Level.Trial)
public void tearDown() throws Exception
{
- exec.getExecutorService().shutdownNow();
- if (!exec.getExecutorService().awaitTermination(1, TimeUnit.MINUTES)) {
+ innerExec.shutdownNow();
+ if (!innerExec.awaitTermination(1, TimeUnit.MINUTES)) {
throw new ISE("Could not terminate executor after 1 minute");
}
}
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java
index 6f0796f412f..56a41e0fb9b 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/indexing/IncrementalIndexReadBenchmark.java
@@ -35,6 +35,7 @@ import org.apache.druid.query.ordering.StringComparators;
import org.apache.druid.query.search.ContainsSearchQuerySpec;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.CursorBuildSpec;
+import org.apache.druid.segment.CursorFactory;
import org.apache.druid.segment.CursorHolder;
import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.data.IndexedInts;
@@ -44,8 +45,8 @@ import org.apache.druid.segment.generator.GeneratorSchemaInfo;
import org.apache.druid.segment.incremental.AppendableIndexSpec;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexCreator;
+import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
import org.apache.druid.segment.serde.ComplexMetrics;
import org.openjdk.jmh.annotations.Benchmark;
import org.openjdk.jmh.annotations.BenchmarkMode;
@@ -147,8 +148,8 @@ public class IncrementalIndexReadBenchmark
@OutputTimeUnit(TimeUnit.MICROSECONDS)
public void read(Blackhole blackhole)
{
- IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex);
- try (final CursorHolder cursorHolder = makeCursor(sa, null)) {
+ final CursorFactory cursorFactory = new IncrementalIndexCursorFactory(incIndex);
+ try (final CursorHolder cursorHolder = makeCursor(cursorFactory, null)) {
Cursor cursor = cursorHolder.asCursor();
List selectors = new ArrayList<>();
@@ -183,8 +184,8 @@ public class IncrementalIndexReadBenchmark
)
);
- IncrementalIndexStorageAdapter sa = new IncrementalIndexStorageAdapter(incIndex);
- try (final CursorHolder cursorHolder = makeCursor(sa, filter)) {
+ IncrementalIndexCursorFactory cursorFactory = new IncrementalIndexCursorFactory(incIndex);
+ try (final CursorHolder cursorHolder = makeCursor(cursorFactory, filter)) {
Cursor cursor = cursorHolder.asCursor();
List selectors = new ArrayList<>();
@@ -204,14 +205,14 @@ public class IncrementalIndexReadBenchmark
}
}
- private CursorHolder makeCursor(IncrementalIndexStorageAdapter sa, DimFilter filter)
+ private CursorHolder makeCursor(CursorFactory factory, DimFilter filter)
{
CursorBuildSpec.CursorBuildSpecBuilder builder = CursorBuildSpec.builder()
.setInterval(schemaInfo.getDataInterval());
if (filter != null) {
builder.setFilter(filter.toFilter());
}
- return sa.makeCursorHolder(builder.build());
+ return factory.makeCursorHolder(builder.build());
}
private static DimensionSelector makeDimensionSelector(Cursor cursor, String name)
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
index 24afa1e8477..8e0715e0fe5 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/CachingClusteredClientBenchmark.java
@@ -362,14 +362,13 @@ public class CachingClusteredClientBenchmark
final GroupingEngine groupingEngine = new GroupingEngine(
processingConfig,
configSupplier,
- bufferPool,
groupByResourcesReservationPool,
mapper,
mapper,
QueryRunnerTestHelper.NOOP_QUERYWATCHER
);
final GroupByQueryQueryToolChest toolChest = new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool);
- return new GroupByQueryRunnerFactory(groupingEngine, toolChest);
+ return new GroupByQueryRunnerFactory(groupingEngine, toolChest, bufferPool);
}
@TearDown(Level.Trial)
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
index e7220cc286d..5ab19b6235f 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/GroupByBenchmark.java
@@ -495,7 +495,6 @@ public class GroupByBenchmark
final GroupingEngine groupingEngine = new GroupingEngine(
druidProcessingConfig,
configSupplier,
- bufferPool,
groupByResourcesReservationPool,
TestHelper.makeJsonMapper(),
new ObjectMapper(new SmileFactory()),
@@ -504,7 +503,8 @@ public class GroupByBenchmark
factory = new GroupByQueryRunnerFactory(
groupingEngine,
- new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool)
+ new GroupByQueryQueryToolChest(groupingEngine, groupByResourcesReservationPool),
+ bufferPool
);
}
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java
index 06ea2aa9e19..0e38757f12b 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlBenchmark.java
@@ -57,8 +57,8 @@ import org.apache.druid.query.lookup.LookupExtractor;
import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.QueryableIndexCursorFactory;
import org.apache.druid.segment.QueryableIndexSegment;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.column.StringEncodingStrategy;
import org.apache.druid.segment.data.FrontCodedIndexed;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
@@ -694,8 +694,8 @@ public class SqlBenchmark
} else if (STORAGE_FRAME_ROW.equals(storageType)) {
walker.add(
descriptor,
- FrameTestUtil.adapterToFrameSegment(
- new QueryableIndexStorageAdapter(index),
+ FrameTestUtil.cursorFactoryToFrameSegment(
+ new QueryableIndexCursorFactory(index),
FrameType.ROW_BASED,
descriptor.getId()
)
@@ -703,8 +703,8 @@ public class SqlBenchmark
} else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) {
walker.add(
descriptor,
- FrameTestUtil.adapterToFrameSegment(
- new QueryableIndexStorageAdapter(index),
+ FrameTestUtil.cursorFactoryToFrameSegment(
+ new QueryableIndexCursorFactory(index),
FrameType.COLUMNAR,
descriptor.getId()
)
diff --git a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java
index 2b5db7413a2..977e8e8e42c 100644
--- a/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java
+++ b/benchmarks/src/test/java/org/apache/druid/benchmark/query/SqlWindowFunctionsBenchmark.java
@@ -55,8 +55,8 @@ import org.apache.druid.query.lookup.LookupExtractor;
import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndex;
+import org.apache.druid.segment.QueryableIndexCursorFactory;
import org.apache.druid.segment.QueryableIndexSegment;
-import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.column.StringEncodingStrategy;
import org.apache.druid.segment.generator.GeneratorBasicSchemas;
import org.apache.druid.segment.generator.GeneratorSchemaInfo;
@@ -159,6 +159,12 @@ public class SqlWindowFunctionsBenchmark
{
return 3;
}
+
+ @Override
+ public int intermediateComputeSizeBytes()
+ {
+ return 200_000_000;
+ }
};
@Setup(Level.Trial)
@@ -281,8 +287,8 @@ public class SqlWindowFunctionsBenchmark
} else if (STORAGE_FRAME_ROW.equals(storageType)) {
walker.add(
descriptor,
- FrameTestUtil.adapterToFrameSegment(
- new QueryableIndexStorageAdapter(index),
+ FrameTestUtil.cursorFactoryToFrameSegment(
+ new QueryableIndexCursorFactory(index),
FrameType.ROW_BASED,
descriptor.getId()
)
@@ -290,8 +296,8 @@ public class SqlWindowFunctionsBenchmark
} else if (STORAGE_FRAME_COLUMNAR.equals(storageType)) {
walker.add(
descriptor,
- FrameTestUtil.adapterToFrameSegment(
- new QueryableIndexStorageAdapter(index),
+ FrameTestUtil.cursorFactoryToFrameSegment(
+ new QueryableIndexCursorFactory(index),
FrameType.COLUMNAR,
descriptor.getId()
)
@@ -336,7 +342,8 @@ public class SqlWindowFunctionsBenchmark
{
final Map context = ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
- QueryContexts.MAX_SUBQUERY_BYTES_KEY, "auto"
+ QueryContexts.MAX_SUBQUERY_BYTES_KEY, "disabled",
+ QueryContexts.MAX_SUBQUERY_ROWS_KEY, -1
);
try (final DruidPlanner planner = plannerFactory.createPlannerForTesting(engine, sql, context)) {
final PlannerResult plannerResult = planner.plan();
@@ -420,4 +427,15 @@ public class SqlWindowFunctionsBenchmark
+ "GROUP BY dimUniform, dimSequential";
querySql(sql, blackhole);
}
+
+ @Benchmark
+ public void windowWithGroupbyTime(Blackhole blackhole)
+ {
+ String sql = "SELECT "
+ + "SUM(dimSequentialHalfNull) + SUM(dimHyperUnique), "
+ + "LAG(SUM(dimSequentialHalfNull + dimHyperUnique)) OVER (PARTITION BY dimUniform ORDER BY dimSequential) "
+ + "FROM foo "
+ + "GROUP BY __time, dimUniform, dimSequential";
+ querySql(sql, blackhole);
+ }
}
diff --git a/cloud/aws-common/pom.xml b/cloud/aws-common/pom.xml
index a41e21380e3..148a6dec54e 100644
--- a/cloud/aws-common/pom.xml
+++ b/cloud/aws-common/pom.xml
@@ -28,7 +28,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/cloud/gcp-common/pom.xml b/cloud/gcp-common/pom.xml
index 3819cf1a04d..d3ba20edd3f 100644
--- a/cloud/gcp-common/pom.xml
+++ b/cloud/gcp-common/pom.xml
@@ -28,7 +28,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/dev/chmod-heap-dumps.sh b/dev/chmod-heap-dumps.sh
new file mode 100755
index 00000000000..dbb9582224b
--- /dev/null
+++ b/dev/chmod-heap-dumps.sh
@@ -0,0 +1,19 @@
+#!/bin/bash -eux
+
+# 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.
+
+BASE_DIR=$(git rev-parse --show-toplevel)
+chmod 644 ${BASE_DIR}/target/*.hprof
diff --git a/distribution/pom.xml b/distribution/pom.xml
index 136935433e9..5aa71130f77 100644
--- a/distribution/pom.xml
+++ b/distribution/pom.xml
@@ -30,7 +30,7 @@
druidorg.apache.druid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT
@@ -458,6 +458,8 @@
org.apache.druid.extensions.contrib:druid-spectator-histogram-corg.apache.druid.extensions.contrib:druid-rabbit-indexing-service
+ -c
+ org.apache.druid.extensions.contrib:grpc-query
diff --git a/docs/assets/druid-architecture.svg b/docs/assets/druid-architecture.svg
index 8db6ec25d4a..9d0e67188fd 100644
--- a/docs/assets/druid-architecture.svg
+++ b/docs/assets/druid-architecture.svg
@@ -16,4 +16,4 @@
~ specific language governing permissions and limitations
~ under the License.
-->
-
\ No newline at end of file
+
\ No newline at end of file
diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index 64797c402c0..5fd31366ad0 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -616,9 +616,10 @@ the [HDFS input source](../ingestion/input-sources.md#hdfs-input-source).
You can set the following property to specify permissible protocols for
the [HTTP input source](../ingestion/input-sources.md#http-input-source).
-|Property|Possible values|Description|Default|
-|--------|---------------|-----------|-------|
-|`druid.ingestion.http.allowedProtocols`|List of protocols|Allowed protocols for the HTTP input source.|`["http", "https"]`|
+|Property| Possible values | Description |Default|
+|--------|------------------------------------------------------------------------------------------------------------------------------------------------------------------|-------------------------------------------------------------------------------------------------------------------------------------------|-------|
+|`druid.ingestion.http.allowedProtocols`| List of protocols | Allowed protocols for the HTTP input source. |`["http", "https"]`|
+|`druid.ingestion.http.allowedHeaders`| A list of permitted request headers for the HTTP input source. By default, the list is empty, which means no headers are allowed in the ingestion specification. |`[]`|
### External data access security configuration
@@ -1795,6 +1796,7 @@ This strategy can be enabled by setting `druid.query.scheduler.prioritization.st
|`druid.query.scheduler.prioritization.periodThreshold`|ISO duration threshold for how old data can be queried before automatically adjusting query priority.|none|
|`druid.query.scheduler.prioritization.durationThreshold`|ISO duration threshold for maximum duration a queries interval can span before the priority is automatically adjusted.|none|
|`druid.query.scheduler.prioritization.segmentCountThreshold`|Number threshold for maximum number of segments that can take part in a query before its priority is automatically adjusted.|none|
+|`druid.query.scheduler.prioritization.segmentRangeThreshold`|ISO duration threshold for maximum segment range a query can span before the priority is automatically adjusted.|none|
|`druid.query.scheduler.prioritization.adjustment`|Amount to reduce the priority of queries which cross any threshold.|none|
##### Laning strategies
diff --git a/docs/development/extensions-contrib/delta-lake.md b/docs/development/extensions-contrib/delta-lake.md
index 503fbfdc55d..88f3a2c77f3 100644
--- a/docs/development/extensions-contrib/delta-lake.md
+++ b/docs/development/extensions-contrib/delta-lake.md
@@ -51,9 +51,4 @@ java \
-c "org.apache.druid.extensions.contrib:druid-deltalake-extensions:"
```
-See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information.
-
-## Known limitations
-
-This extension relies on the Delta Kernel API and can only read from the latest Delta table snapshot. Ability to read from
-arbitrary snapshots is tracked [here](https://github.com/delta-io/delta/issues/2581).
\ No newline at end of file
+See [Loading community extensions](../../configuration/extensions.md#loading-community-extensions) for more information.
\ No newline at end of file
diff --git a/docs/development/extensions-core/druid-lookups.md b/docs/development/extensions-core/druid-lookups.md
index d6219b8c742..06283ec4d72 100644
--- a/docs/development/extensions-core/druid-lookups.md
+++ b/docs/development/extensions-core/druid-lookups.md
@@ -31,9 +31,9 @@ This module can be used side to side with other lookup module like the global ca
To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-lookups-cached-single` in the extensions load list.
:::info
- If using JDBC, you will need to add your database's client JAR files to the extension's directory.
+To use JDBC, you must add your database client JAR files to the extension's directory.
For Postgres, the connector JAR is already included.
- See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#installing-the-mysql-connector-library) or [MariaDB](./mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries.
+ See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#install-mysql-connectorj) or [MariaDB](./mysql.md#install-mariadb-connectorj) connector libraries.
Copy or symlink the downloaded file to `extensions/druid-lookups-cached-single` under the distribution root directory.
:::
diff --git a/docs/development/extensions-core/mysql.md b/docs/development/extensions-core/mysql.md
index bc6012dbb5a..a3678f65056 100644
--- a/docs/development/extensions-core/mysql.md
+++ b/docs/development/extensions-core/mysql.md
@@ -1,6 +1,6 @@
---
id: mysql
-title: "MySQL Metadata Store"
+title: "MySQL metadata store"
---
-This page provides a reference of Apache Druid® SQL functions in alphabetical order. For more details on a function, refer to the following:
-* [Aggregation functions](sql-aggregations.md)
-* [Array functions](sql-array-functions.md)
-* [JSON functions](sql-json-functions.md)
-* [Multi-value string functions](sql-multivalue-string-functions.md)
-* [Scalar functions](sql-scalar.md)
-* [Window functions](sql-window-functions.md)
-The examples on this page use the following example datasources:
-* `flight-carriers` using `FlightCarrierOnTime (1 month)`
-* `taxi-trips` using `NYC Taxi cabs (3 files)`
+
+This page provides a reference of all Druid SQL functions in alphabetical order.
+Click the linked function type for documentation on a particular function.
## ABS
+`ABS()`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the absolute value of a numeric expression.
-* **Syntax:** `ABS()`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example applies the ABS function to the `ArrDelay` column from the `flight-carriers` datasource.
-
-```sql
-SELECT
- "ArrDelay" AS "arrival_delay",
- ABS("ArrDelay") AS "absolute_arrival_delay"
-FROM "flight-carriers"
-WHERE "ArrDelay" < 0
-LIMIT 1
-```
-Returns the following:
-
-| `arrival_delay` | `absolute_arrival_delay` |
-| -- | -- |
-| `-27` | `27` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## ACOS
-Calculates the arc cosine (arccosine) of a numeric expression.
+`ACOS()`
-* **Syntax:** `ACOS(expr)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example calculates the arc cosine of `0`.
-
-```sql
-SELECT ACOS(0) AS "arc_cosine"
-```
-Returns the following:
-
-| `arc_cosine` |
-| -- |
-| `1.5707963267948966` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Calculates the arc cosine of a numeric expression.
## ANY_VALUE
@@ -107,7 +64,6 @@ Returns any value of the specified expression.
Counts distinct values of a regular column or a prebuilt sketch column.
-## APPROX_COUNT_DISTINCT_BUILTIN
`APPROX_COUNT_DISTINCT_BUILTIN(expr)`
**Function type:** [Aggregation](sql-aggregations.md)
@@ -295,72 +251,27 @@ Joins all elements of `arr` by the delimiter specified by `str`.
## ASIN
-Calculates the arc sine (arcsine) of a numeric expression.
+`ASIN()`
-* **Syntax:** `ASIN(expr)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example calculates the arc sine of `1`.
-
-```sql
-SELECT ASIN(1) AS "arc_sine"
-```
-Returns the following:
-
-| `arc_sine` |
-| -- |
-| `1.5707963267948966` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Calculates the arc sine of a numeric expression.
## ATAN
-Calculates the arc tangent (arctangent) of a numeric expression.
+`ATAN()`
-* **Syntax:** `ATAN(expr)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example calculates the arc tangent of `1`.
-
-```sql
-SELECT ATAN(1) AS "arc_tangent"
-```
-Returns the following:
-
-| `arc_tangent` |
-| -- |
-| `0.7853981633974483` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Calculates the arc tangent of a numeric expression.
## ATAN2
-Calculates the arc tangent (arctangent) of a specified x and y coordinate.
+`ATAN2(, )`
-* **Syntax:** `ATAN2(x, y)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example calculates the arc tangent of the coordinate `(1, -1)`
-
-```sql
-SELECT ATAN2(1,-1) AS "arc_tangent_2"
-```
-Returns the following:
-
-| `arc_tangent_2` |
-| -- |
-| `2.356194490192345` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Calculates the arc tangent of the two arguments.
## AVG
@@ -396,188 +307,67 @@ Performs a bitwise XOR operation on all input values.
## BITWISE_AND
-Returns the bitwise AND between two expressions: `expr1 & expr2`.
+`BITWISE_AND(expr1, expr2)`
-* **Syntax:** `BITWISE_AND(expr1, expr2)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example performs the bitwise AND operation `12 & 10`.
-
-```sql
-SELECT BITWISE_AND(12, 10) AS "bitwise_and"
-```
-Returns the following:
-
-| `bitwise_and` |
-| -- |
-| 8 |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Returns the bitwise AND between the two expressions, that is, `expr1 & expr2`.
## BITWISE_COMPLEMENT
-Returns the bitwise complement (bitwise not) for the expression: `~expr`.
+`BITWISE_COMPLEMENT(expr)`
-* **Syntax:** `BITWISE_COMPLEMENT(expr)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example performs the bitwise complement operation `~12`.
-
-```sql
-SELECT BITWISE_COMPLEMENT(12) AS "bitwise_complement"
-```
-Returns the following:
-
-| `bitwise_complement` |
-| -- |
-| -13 |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Returns the bitwise NOT for the expression, that is, `~expr`.
## BITWISE_CONVERT_DOUBLE_TO_LONG_BITS
-Converts the bits of an IEEE 754 floating-point double value to long.
+`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)`
-* **Syntax:**`BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example returns the IEEE 754 floating-point double representation of `255` as a long.
-
-```sql
-SELECT BITWISE_CONVERT_DOUBLE_TO_LONG_BITS(255) AS "ieee_754_double_to_long"
-```
-Returns the following:
-
-| `ieee_754_double_to_long` |
-| -- |
-| `4643176031446892544` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+Converts the bits of an IEEE 754 floating-point double value to a long.
## BITWISE_CONVERT_LONG_BITS_TO_DOUBLE
+`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Converts a long to the IEEE 754 floating-point double specified by the bits stored in the long.
-* **Syntax:**`BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example returns the long representation of `4643176031446892544` as an IEEE 754 floating-point double.
-
-```sql
-SELECT BITWISE_CONVERT_LONG_BITS_TO_DOUBLE(4643176031446892544) AS "long_to_ieee_754_double"
-```
-Returns the following:
-
-| `long_to_ieee_754_double` |
-| -- |
-| `255` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## BITWISE_OR
-Returns the bitwise OR between the two expressions: `expr1 | expr2`.
+`BITWISE_OR(expr1, expr2)`
-* **Syntax:** `BITWISE_OR(expr1, expr2)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example performs the bitwise OR operation `12 | 10`.
-
-```sql
-SELECT BITWISE_OR(12, 10) AS "bitwise_or"
-```
-Returns the following:
-
-| `bitwise_or` |
-| -- |
-| `14` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Returns the bitwise OR between the two expressions, that is, `expr1 | expr2`.
## BITWISE_SHIFT_LEFT
-Returns the bitwise left shift by x positions of an expr: `expr << x`.
+`BITWISE_SHIFT_LEFT(expr1, expr2)`
-* **Syntax:** `BITWISE_SHIFT_LEFT(expr, x)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example performs the bitwise SHIFT operation `2 << 3`.
-
-```sql
-SELECT BITWISE_SHIFT_LEFT(2, 3) AS "bitwise_shift_left"
-```
-Returns the following:
-
-| `bitwise_shift_left` |
-| -- |
-| `16` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Returns a bitwise left shift of expr1, that is, `expr1 << expr2`.
## BITWISE_SHIFT_RIGHT
-Returns the bitwise right shift by x positions of an expr: `expr >> x`.
+`BITWISE_SHIFT_RIGHT(expr1, expr2)`
-* **Syntax:** `BITWISE_SHIFT_RIGHT(expr, x)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example performs the bitwise SHIFT operation `16 >> 3`.
-
-```sql
-SELECT BITWISE_SHIFT_RIGHT(16, 3) AS "bitwise_shift_right"
-```
-Returns the following:
-
-| `bitwise_shift_right` |
-| -- |
-| `2` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Returns a bitwise right shift of expr1, that is, `expr1 >> expr2`.
## BITWISE_XOR
-Returns the bitwise exclusive OR between the two expressions: `expr1 ^ expr2`.
+`BITWISE_XOR(expr1, expr2)`
-* **Syntax:** `BITWISE_XOR(expr1, expr2)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example performs the bitwise XOR operation `12 ^ 10`.
-
-```sql
-SELECT BITWISE_XOR(12, 10) AS "bitwise_xor"
-```
-Returns the following:
-
-| `bitwise_xor` |
-| -- |
-| `6` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Returns the bitwise exclusive OR between the two expressions, that is, `expr1 ^ expr2`.
## BLOOM_FILTER
@@ -597,30 +387,11 @@ Returns true if the expression is contained in a Base64-serialized Bloom filter.
## BTRIM
-Trims characters from both the leading and trailing ends of an expression. Defaults `chars` to a space if none is provided.
+`BTRIM(, [])`
-* **Syntax:** `BTRIM(expr[, chars])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example trims the `_` characters from both ends of the string expression.
-
-```sql
-SELECT
- '___abc___' AS "original_string",
- BTRIM('___abc___', '_') AS "trim_both_ends"
-```
-
-Returns the following:
-
-| `original_string` | `trim_both_ends` |
-| -- | -- |
-| `___abc___` | `abc` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Trims characters from both the leading and trailing ends of an expression.
## CASE
@@ -648,49 +419,28 @@ Rounds up a timestamp by a given time unit.
## CEIL (numeric)
+`CEIL()`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the smallest integer value greater than or equal to the numeric expression.
-* **Syntax:** `CEIL()`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example applies the CEIL function to the `fare_amount` column from the `taxi-trips` datasource.
-
-```sql
-SELECT
- "fare_amount" AS "fare_amount",
- CEIL("fare_amount") AS "ceiling_fare_amount"
-FROM "taxi-trips"
-LIMIT 1
-```
-Returns the following:
-
-| `fare_amount` | `ceiling_fare_amount` |
-| -- | -- |
-| `21.25` | `22` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
## CHAR_LENGTH
+`CHAR_LENGTH(expr)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Alias for [`LENGTH`](#length).
-* **Syntax:** `CHAR_LENGTH(expr)`
-* **Function type:** Scalar, string
-
-[Learn more](sql-scalar.md#string-functions)
-
## CHARACTER_LENGTH
+`CHARACTER_LENGTH(expr)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Alias for [`LENGTH`](#length).
-* **Syntax:** `CHARACTER_LENGTH(expr)`
-* **Function type:** Scalar, string
-
-[Learn more](sql-scalar.md#string-functions)
-
-
## COALESCE
`COALESCE(expr, expr, ...)`
@@ -701,111 +451,36 @@ Returns the first non-null value.
## CONCAT
+`CONCAT(expr, expr...)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Concatenates a list of expressions.
-* **Syntax:** `CONCAT(expr[, expr,...])`
-* **Function type:** Scalar, string
-
-Example
-
-The following example concatenates the `OriginCityName` column from `flight-carriers`, the string ` to `, and the `DestCityName` column from `flight-carriers`.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- "DestCityName" AS "destination_city",
- CONCAT("OriginCityName", ' to ', "DestCityName") AS "concatenate_flight_details"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `origin_city` | `destination_city` | `concatenate_flight_details` |
-| -- | -- | -- |
-| `San Juan, PR` | `Washington, DC` | `San Juan, PR to Washington, DC` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
-
## CONTAINS_STRING
-Returns `true` if `str` is a substring of `expr`, case-sensitive. Otherwise returns `false`.
+`CONTAINS_STRING(, )`
-* **Syntax:** `CONTAINS_STRING(expr, str)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the substring `San`.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- CONTAINS_STRING("OriginCityName", 'San') AS "contains_string"
-FROM "flight-carriers"
-LIMIT 2
-```
-
-Returns the following:
-
-| `origin_city` | `contains_string` |
-| -- | -- |
-| `San Juan, PR` | `true` |
-| `Boston, MA` | `false` |
-
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Finds whether a string is in a given expression, case-sensitive.
## COS
+`COS()`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the trigonometric cosine of an angle expressed in radians.
-* **Syntax:** `COS(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example calculates the cosine of angle `PI/3` radians.
-
-```sql
-SELECT COS(PI / 3) AS "cosine"
-```
-Returns the following:
-
-| `cosine` |
-| -- |
-| `0.5000000000000001` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## COT
+`COT()`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the trigonometric cotangent of an angle expressed in radians.
-* **Syntax:** `COT(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example calculates the cotangent of angle `PI/3` radians.
-
-```sql
-SELECT COT(PI / 3) AS "cotangent"
-```
-Returns the following:
-
-| `cotangent` |
-| -- |
-| `0.577350269189626` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## COUNT
`COUNT([DISTINCT] expr)`
@@ -826,85 +501,28 @@ Returns the cumulative distribution of the current row within the window calcula
## CURRENT_DATE
-Returns the current date in UTC time, unless you specify a different timezone in the query context.
+`CURRENT_DATE`
-* **Syntax:** `CURRENT_DATE`
-* **Function type:** Scalar, date and time
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
-Example
-
-The following example returns the current date.
-
-```sql
-SELECT CURRENT_DATE AS "current_date"
-```
-
-Returns the following:
-
-| `current_date` |
-| -- |
-| `2024-08-14T00:00:00.000Z `|
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+Returns the current date in the connection's time zone.
## CURRENT_TIMESTAMP
-Returns the current timestamp in UTC time, unless you specify a different timezone in the query context.
+`CURRENT_TIMESTAMP`
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
-* **Syntax:** `CURRENT_TIMESTAMP`
-* **Function type:** Scalar, date and time
-
-Example
-
-The following example returns the current timestamp.
-
-```sql
-SELECT CURRENT_TIMESTAMP AS "current_timestamp"
-```
-
-Returns the following:
-
-| `current_timestamp` |
-| -- |
-| `2024-08-14T21:30:13.793Z` |
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+Returns the current timestamp in the connection's time zone.
## DATE_TRUNC
+`DATE_TRUNC(, )`
+
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
+
Rounds down a timestamp by a given time unit.
-* **Syntax:** `DATE_TRUNC(unit, timestamp_expr)`
-* **Function type:** Scalar, date and time
-
-Example
-
-The following example truncates a timestamp from the `__time` column from the `taxi-trips` datasource to the most recent `decade`.
-
-```sql
-SELECT
- "__time" AS "original_timestamp",
- DATE_TRUNC('decade', "__time") AS "truncate_timestamp"
-FROM "taxi-trips"
-LIMIT 1
-```
-
-Returns the following:
-
-| `original_timestamp` | `truncate_time` |
-| -- | -- |
-| `2013-08-01T08:14:37.000Z` | `2010-01-01T00:00:00.000Z` |
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
-
-
## DECODE_BASE64_COMPLEX
`DECODE_BASE64_COMPLEX(dataType, expr)`
@@ -915,54 +533,21 @@ Decodes a Base64-encoded string into a complex data type, where `dataType` is th
## DECODE_BASE64_UTF8
+`DECODE_BASE64_UTF8(expr)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
+
Decodes a Base64-encoded string into a UTF-8 encoded string.
-* **Syntax:** `DECODE_BASE64_UTF8(expr)`
-* **Function type:** Scalar, string
-
-Example
-
-The following example converts the base64 encoded string `SGVsbG8gV29ybGQhCg==` into an UTF-8 encoded string.
-
-```sql
-SELECT
- 'SGVsbG8gV29ybGQhCg==' AS "base64_encoding",
- DECODE_BASE64_UTF8('SGVsbG8gV29ybGQhCg==') AS "convert_to_UTF8_encoding"
-```
-
-Returns the following:
-
-| `base64_encoding` | `convert_to_UTF8_encoding` |
-| -- | -- |
-| `SGVsbG8gV29ybGQhCg==` | `Hello World!` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
-
## DEGREES
+`DEGREES()`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Converts an angle from radians to degrees.
-* **Syntax:** `DEGREES(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example converts an angle of `PI` radians to degrees
-
-```sql
-SELECT DEGREES(PI) AS "degrees"
-```
-Returns the following:
-
-| `degrees` |
-| -- |
-| `180` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## DENSE_RANK
`DENSE_RANK()`
@@ -973,37 +558,16 @@ Returns the rank for a row within a window without gaps. For example, if two row
## DIV
+`DIV(x, y)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Returns the result of integer division of `x` by `y`.
-* **Syntax:** `DIV(x, y)`
-* **Function type:** Scalar, numeric
-
-
-
:::info
-
- The `DIV` function is not implemented in Druid versions 30.0.0 or earlier. Consider using [`SAFE_DIVIDE`](./sql-functions.md/#safe_divide) instead.
-
+The `DIV` function is not implemented in Druid versions 30.0.0 or earlier. Consider using [`SAFE_DIVIDE`](./sql-functions.md#safe_divide) instead.
:::
-[Learn more](sql-scalar.md#numeric-functions)
-
## DS_CDF
`DS_CDF(expr, splitPoint0, splitPoint1, ...)`
@@ -1137,27 +701,12 @@ Returns the value of a numeric or string expression corresponding to the earlies
## EXP
+`EXP()`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates _e_ raised to the power of the numeric expression.
-* **Syntax:** `EXP()`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example calculates _e_ to the power of 1.
-
-```sql
-SELECT EXP(1) AS "exponential"
-```
-Returns the following:
-
-| `exponential` |
-| -- |
-| `2.7182818284590455` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## EXTRACT
`EXTRACT( FROM )`
@@ -1184,56 +733,19 @@ Rounds down a timestamp by a given time unit.
## FLOOR (numeric)
-Calculates the largest integer less than or equal to the numeric expression.
+`FLOOR()`
-* **Syntax:** `FLOOR()`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example applies the FLOOR function to the `fare_amount` column from the `taxi-trips` datasource.
-
-```sql
-SELECT
- "fare_amount" AS "fare_amount",
- FLOOR("fare_amount") AS "floor_fare_amount"
-FROM "taxi-trips"
-LIMIT 1
-```
-Returns the following:
-
-| `fare_amount` | `floor_fare_amount` |
-| -- | -- |
-| `21.25` | `21` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Calculates the largest integer value less than or equal to the numeric expression.
## GREATEST
-Returns the maximum value from the provided expressions. For information on how Druid interprets the arguments passed into the function, see [Reduction functions](sql-scalar.md#reduction-functions).
+`GREATEST([expr1, ...])`
-* **Syntax:** `GREATEST([expr1, ...])`
-* **Function type:** Scalar, reduction
-
-Example
-
-The following example returns the greatest value between the numeric constant `PI`, the integer number `4`, and the double `-5.0`. Druid interprets these arguments as DOUBLE data type.
-
-```sql
-SELECT GREATEST(PI, 4, -5.0) AS "greatest"
-```
-
-Returns the following:
-
-| `greatest` |
-| -- |
-| `4` |
-
-
-
-[Learn more](sql-scalar.md#reduction-functions)
+**Function type:** [Scalar, reduction](sql-scalar.md#reduction-functions)
+Returns the maximum value from the provided arguments.
## GROUPING
@@ -1277,107 +789,35 @@ Returns a union of HLL sketches.
## HUMAN_READABLE_BINARY_BYTE_FORMAT
-Converts an integer byte size into human-readable [IEC](https://en.wikipedia.org/wiki/Binary_prefix) format.
+`HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])`
-* **Syntax:** `HUMAN_READABLE_BINARY_BYTE_FORMAT(value[, precision])`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
- The following example converts `1000000` into IEC format.
-
- ```sql
- SELECT HUMAN_READABLE_BINARY_BYTE_FORMAT(1000000, 2) AS "iec_format"
- ```
-
- Returns the following:
-
- | `iec_format` |
- | -- |
- | `976.56 KiB` |
-
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Converts an integer byte size into human-readable IEC format.
## HUMAN_READABLE_DECIMAL_BYTE_FORMAT
-Converts a byte size into human-readable [SI](https://en.wikipedia.org/wiki/Binary_prefix) format.
+`HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])`
-* **Syntax:** `HUMAN_READABLE_DECIMAL_BYTE_FORMAT(value[, precision])`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example converts `1000000` into SI format.
-
-```sql
-SELECT HUMAN_READABLE_DECIMAL_BYTE_FORMAT(1000000, 2) AS "si_format"
-```
-
-Returns the following:
-
-|`si_format`|
-|--|
-|`1.00 MB`|
-
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Converts a byte size into human-readable SI format.
## HUMAN_READABLE_DECIMAL_FORMAT
+`HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Converts a byte size into human-readable SI format with single-character units.
-* **Syntax:** `HUMAN_READABLE_DECIMAL_FORMAT(value[, precision])`
-* **Function type:** Scalar, numeric
-
-Example
-
- The following example converts `1000000` into single character SI format.
-
-```sql
-SELECT HUMAN_READABLE_DECIMAL_FORMAT(1000000, 2) AS "single_character_si_format"
-```
-
-Returns the following:
-
-|`single_character_si_format`|
-|--|
-|`1.00 M`|
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## ICONTAINS_STRING
-Returns `true` if `str` is a substring of `expr`, case-insensitive. Otherwise returns `false`.
+`ICONTAINS_STRING(, str)`
-* **Syntax:** `ICONTAINS_STRING(expr, str)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns `true` if the `OriginCityName` column from the `flight-carriers` datasource contains the case-insensitive substring `san`.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- ICONTAINS_STRING("OriginCityName", 'san') AS "contains_case_insensitive_string"
-FROM "flight-carriers"
-LIMIT 2
-```
-
-Returns the following:
-
-| `origin_city` | `contains_case_insensitive_string` |
-| -- | -- |
-| `San Juan, PR` | `true` |
-| `Boston, MA` | `false` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Finds whether a string is in a given expression, case-insensitive.
## IPV4_MATCH
@@ -1419,6 +859,13 @@ Returns true if the IPv6 `address` belongs to the `subnet` literal, else false.
Returns an array of field names from `expr` at the specified `path`.
+## JSON_MERGE
+
+**Function type:** [JSON](sql-json-functions.md)
+
+`JSON_MERGE(expr1, expr2[, expr3 ...])`
+Merges two or more JSON `STRING` or `COMPLEX` into one. Preserves the rightmost value when there are key overlaps. Returning always a `COMPLEX` type.
+
## JSON_OBJECT
**Function type:** [JSON](sql-json-functions.md)
@@ -1501,269 +948,75 @@ If you do not supply an `offset`, returns the value evaluated at the row followi
## LEAST
-Returns the minimum value from the provided expressions. For information on how Druid interprets the arguments passed into the function, see [Reduction functions](sql-scalar.md#reduction-functions).
+`LEAST([expr1, ...])`
-* **Syntax:** `LEAST([expr1, ...])`
-* **Function type:** Scalar, reduction
-
-Example
-
-The following example returns the minimum value between the strings `apple`, `orange`, and `pear`. Druid interprets these arguments as STRING data type.
-
-```sql
-SELECT LEAST( 'apple', 'orange', 'pear') AS "least"
-```
-
-Returns the following:
-
-| `least` |
-| -- |
-| `apple` |
-
-
-
-[Learn more](sql-scalar.md#reduction-functions)
+**Function type:** [Scalar, reduction](sql-scalar.md#reduction-functions)
+Returns the minimum value from the provided arguments.
## LEFT
-Returns the `N` leftmost characters of an expression, where `N` is an integer value.
+`LEFT(expr, [length])`
-* **Syntax:** `LEFT(expr, N)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns the `3` leftmost characters of the expression `ABCDEFG`.
-
-```sql
-SELECT
- 'ABCDEFG' AS "expression",
- LEFT('ABCDEFG', 3) AS "leftmost_characters"
-```
-
-Returns the following:
-
-| `expression` | `leftmost_characters` |
-| -- | -- |
-| `ABCDEFG` | `ABC` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns the leftmost number of characters from an expression.
## LENGTH
-Returns the length of the expression in UTF-16 code units.
+`LENGTH(expr)`
-* **Syntax:** `LENGTH(expr)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns the character length of the `OriginCityName` column from the `flight-carriers` datasource.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city_name",
- LENGTH("OriginCityName") AS "city_name_length"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `origin_city_name` | `city_name_length` |
-| -- | -- |
-| `San Juan, PR` | `12` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns the length of the expression in UTF-16 encoding.
## LN
+`LN(expr)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the natural logarithm of the numeric expression.
-* **Syntax:** `LN()`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example applies the LN function to the `max_temperature` column from the `taxi-trips` datasource.
-
-```sql
-SELECT
- "max_temperature" AS "max_temperature",
- LN("max_temperature") AS "natural_log_max_temp"
-FROM "taxi-trips"
-LIMIT 1
-```
-
-Returns the following:
-
-| `max_temperature` | `natural_log_max_temp` |
-| -- | -- |
-| `76` | `4.330733340286331` |
-
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## LOG10
-Calculates the base-10 logarithm of the numeric expression.
+`LOG10(expr)`
-* **Syntax:** `LOG10()`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following example applies the LOG10 function to the `max_temperature` column from the `taxi-trips` datasource.
-
-```sql
-SELECT
- "max_temperature" AS "max_temperature",
- LOG10("max_temperature") AS "log10_max_temp"
-FROM "taxi-trips"
-LIMIT 1
-```
-Returns the following:
-
-| `max_temperature` | `log10_max_temp` |
-| -- | -- |
-| `76` | `1.8808135922807914` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Calculates the base-10 of the numeric expression.
## LOOKUP
-Searches for `expr` in a registered [query-time lookup table](lookups.md) named `lookupName` and returns the mapped value. If `expr` is null or not contained in the lookup, returns `defaultValue` if supplied, otherwise returns null.
+`LOOKUP(, [, ])`
-* **Syntax:** `LOOKUP(expr, lookupName[, defaultValue])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example uses a `map` type lookup table named `code_to_name`, which contains the following key-value pairs:
-
-```json
-{
- "SJU": "Luis Munoz Marin International Airport",
- "IAD": "Dulles International Airport"
-}
-```
-
-The example uses `code_to_name` to map the `Origin` column from the `flight-carriers` datasource to the corresponding full airport name. Returns `key not found` if no matching key exists in the lookup table.
-
-```sql
-SELECT
- "Origin" AS "origin_airport",
- LOOKUP("Origin", 'code_to_name','key not found') AS "full_airport_name"
-FROM "flight-carriers"
-LIMIT 2
-```
-
-Returns the following:
-
-| `origin_airport` | `full_airport_name` |
-| -- | -- |
-| `SJU` | `Luis Munoz Marin International Airport` |
-| `BOS` | `key not found` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Looks up the expression in a registered query-time lookup table.
## LOWER
+`LOWER(expr)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Returns the expression in lowercase.
-* **Syntax:** `LOWER(expr)`
-* **Function type:** Scalar, string
-
-Example
-
-The following example converts the `OriginCityName` column from the `flight-carriers` datasource to lowercase.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- LOWER("OriginCityName") AS "lowercase"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `origin_city` | `lowercase` |
-| -- | -- |
-`San Juan, PR` | `san juan, pr` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
-
-
## LPAD
-Returns a string of size `length` from `expr`. When the length of `expr` is less than `length`, left pads `expr` with `chars`, which defaults to the space character. Truncates `expr` to `length` if `length` is shorter than the length of `expr`.
+`LPAD(, , [])`
-* **Syntax:** `LPAD(expr, length[, chars])`
-* **Function type:** Scalar, string
-
-Example
-
-The following example left pads the value of `OriginStateName` from the `flight-carriers` datasource to return a total of 11 characters.
-
-```sql
-SELECT
- "OriginStateName" AS "origin_state",
- LPAD("OriginStateName", 11, '+') AS "add_left_padding"
-FROM "flight-carriers"
-LIMIT 3
-```
-
-Returns the following:
-
-| `origin_state` | `add_left_padding` |
-| -- | -- |
-| `Puerto Rico` | `Puerto Rico` |
-| `Massachusetts` | `Massachuset` |
-| `Florida` | `++++Florida` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+Returns the leftmost number of characters from an expression, optionally padded with the given characters.
## LTRIM
-Trims characters from the leading end of an expression. Defaults `chars` to a space if none is provided.
+`LTRIM(, [])`
-* **Syntax:** `LTRIM(expr[, chars])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example trims the `_` characters from the leading end of the string expression.
-
-```sql
-SELECT
- '___abc___' AS "original_string",
- LTRIM('___abc___', '_') AS "trim_leading_end_of_expression"
-```
-
-Returns the following:
-
-| `original_string` | `trim_leading_end_of_expression` |
-| -- | -- |
-| `___abc___` | `abc___` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Trims characters from the leading end of an expression.
## MAX
@@ -1791,26 +1044,11 @@ Returns the minimum value of a set of values.
## MOD
-Calculates x modulo y, or the remainder of x divided by y. Where x and y are numeric expressions.
+`MOD(x, y)`
-* **Syntax:** `MOD(x, y)`
-* **Function type:** Scalar, numeric
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
-Example
-
-The following calculates 78 MOD 10.
-
-```sql
-SELECT MOD(78, 10) as "modulo"
-```
-Returns the following:
-
-| `modulo` |
-| -- |
-| `8` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
+Calculates x modulo y, or the remainder of x divided by y.
## MV_APPEND
@@ -1966,31 +1204,11 @@ Parses `expr` into a `COMPLEX` object. This operator deserializes JSON val
## PARSE_LONG
-Converts a string into a long(BIGINT) with the given radix, or into DECIMAL(base 10) if a radix is not provided.
+`PARSE_LONG(, [])`
-* **Syntax:**`PARSE_LONG(string[, radix])`
-* **Function type:** Scalar, string
-
-Example
-
-The following example converts the string representation of the binary, radix 2, number `1100` into its long (BIGINT) equivalent.
-
-```sql
-SELECT
- '1100' AS "binary_as_string",
- PARSE_LONG('1110', 2) AS "bigint_value"
-```
-
-Returns the following:
-
-| `binary_as_string` | `bigint_value` |
-| -- | -- |
-| `1100` | `14` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+Converts a string into a BIGINT with the given base or into a DECIMAL data type if the base is not specified.
## PERCENT_RANK
@@ -2002,80 +1220,28 @@ Returns the relative rank of the row calculated as a percentage according to the
## POSITION
-Returns the one-based index position of a substring within an expression, optionally starting from a given one-based index. If `substring` is not found, returns 0.
+`POSITION( IN [FROM ])`
-* **Syntax**: `POSITION(substring IN expr [FROM startingIndex])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns the one-based index of the substring `PR` in the `OriginCityName` column from the `flight-carriers` datasource starting from index 5.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- POSITION('PR' IN "OriginCityName" FROM 5) AS "index"
-FROM "flight-carriers"
-LIMIT 2
-```
-
-Returns the following:
-
-| `origin_city` | `index` |
-| -- | -- |
-| `San Juan, PR` | `11` |
-| `Boston, MA` | `0` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns the one-based index position of a substring within an expression, optionally starting from a given one-based index.
## POWER
+`POWER(expr, power)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates a numerical expression raised to the specified power.
-* **Syntax:** `POWER(base, exponent)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example raises 5 to the power of 2.
-
-```sql
-SELECT POWER(5, 2) AS "power"
-```
-Returns the following:
-
-| `power` |
-| -- |
-| `25` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## RADIANS
+`RADIANS(expr)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Converts an angle from degrees to radians.
-* **Syntax:** `RADIANS(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example converts an angle of `180` degrees to radians
-
-```sql
-SELECT RADIANS(180) AS "radians"
-```
-Returns the following:
-
-| `radians` |
-| -- |
-| `3.141592653589793` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## RANK
`RANK()`
@@ -2086,227 +1252,69 @@ Returns the rank with gaps for a row within a window. For example, if two rows t
## REGEXP_EXTRACT
-Apply regular expression `pattern` to `expr` and extract the Nth capture group. If `N` is unspecified or zero, returns the first substring that matches the pattern. Returns `null` if there is no matching pattern.
+`REGEXP_EXTRACT(, , [])`
-* **Syntax:** `REGEXP_EXTRACT(expr, pattern[, N])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example uses regular expressions to find city names inside the `OriginCityName` column from the `flight-carriers` datasource by matching what comes before the comma.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- REGEXP_EXTRACT("OriginCityName", '([^,]+)', 0) AS "pattern_match"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `origin_city` | `pattern_match` |
-| -- | -- |
-| `San Juan, PR` | `San Juan`|
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Applies a regular expression to the string expression and returns the _n_th match.
## REGEXP_LIKE
-Returns `true` if the regular expression `pattern` finds a match in `expr`. Returns `false` otherwise.
+`REGEXP_LIKE(, )`
-* **Syntax:** `REGEXP_LIKE(expr, pattern)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns `true` when the `OriginCityName` column from `flight-carriers` has a city name containing a space.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- REGEXP_LIKE("OriginCityName", '[A-Za-z]+\s[A-Za-z]+') AS "pattern_found"
-FROM "flight-carriers"
-LIMIT 2
-```
-
-Returns the following:
-
-| `origin_city` | `pattern_found` |
-| -- | -- |
-| `San Juan, PR` | `true` |
-| `Boston, MA` | `false` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns true or false signifying whether the regular expression finds a match in the string expression.
## REGEXP_REPLACE
-Replaces all occurrences of a regular expression in a string expression with a replacement string. Refer to capture groups in the replacement string using `$group` syntax. For example: `$1` or `$2`.
+`REGEXP_REPLACE(, , )`
-* **Syntax:** `REGEXP_REPLACE(expr, pattern, replacement)`
-* **Function type:** Scalar, string
-
-Example
-
-The following example matches three consecutive words, where each word is its own capture group, and replaces the matched words with the word in the second capture group punctuated with exclamation marks.
-
-```sql
-SELECT
- 'foo bar baz' AS "original_string",
- REGEXP_REPLACE('foo bar baz', '([A-Za-z]+) ([A-Za-z]+) ([A-Za-z]+)' , '$2!') AS "modified_string"
-```
-
-Returns the following:
-
-| `original_string` | `modified_string` |
-| -- | -- |
-| `foo bar baz` | `bar!` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+Replaces all occurrences of a regular expression in a string expression with a replacement string. The replacement
+string may refer to capture groups using `$1`, `$2`, etc.
## REPEAT
-Repeats the string expression `N` times, where `N` is an integer.
+`REPEAT(, [])`
-* **Syntax:** `REPEAT(expr, N)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns the string expression `abc` repeated `3` times.
-
-```sql
-SELECT
- 'abc' AS "original_string",
- REPEAT('abc', 3) AS "with_repetition"
-```
-
-Returns the following:
-
-| `original_string` | `with_repetition` |
-| -- | -- |
-| `abc` | `abcabcabc` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Repeats the string expression an integer number of times.
## REPLACE
-Replaces instances of a substring with a replacement string in the given expression.
+`REPLACE(expr, pattern, replacement)`
-* **Syntax:** `REPLACE(expr, substring, replacement)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example replaces instances of the substring `abc` with `XYZ`.
-
-```sql
-SELECT
- 'abc 123 abc 123' AS "original_string",
- REPLACE('abc 123 abc 123', 'abc', 'XYZ') AS "modified_string"
-```
-
-Returns the following:
-
-| `original_string` | `modified_string` |
-| -- | -- |
-| `abc 123 abc 123` | `XYZ 123 XYZ 123` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Replaces a pattern with another string in the given expression.
## REVERSE
+`REVERSE(expr)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Reverses the given expression.
-* **Syntax:** `REVERSE(expr)`
-* **Function type:** Scalar, string
-
-Example
-
-The following example reverses the string expression `abc`.
-
-```sql
-SELECT
- 'abc' AS "original_string",
- REVERSE('abc') AS "reversal"
-```
-
-Returns the following:
-
-| `original_string` | `reversal` |
-| -- | -- |
-| `abc` | `cba` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
-
## RIGHT
-Returns the `N` rightmost characters of an expression, where `N` is an integer value.
+`RIGHT(expr, [length])`
-* **Syntax:** `RIGHT(expr, N)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns the `3` rightmost characters of the expression `ABCDEFG`.
-
-```sql
-SELECT
- 'ABCDEFG' AS "expression",
- RIGHT('ABCDEFG', 3) AS "rightmost_characters"
-```
-
-Returns the following:
-
-| `expression` | `rightmost_characters` |
-| -- | -- |
-| `ABCDEFG` | `EFG` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns the rightmost number of characters from an expression.
## ROUND
+`ROUND(expr[, digits])`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the rounded value for a numerical expression.
-* **Syntax:** `ROUND(expr[, digits])`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following applies the ROUND function to 0 decimal points on the `pickup_longitude` column from the `taxi-trips` datasource.
-
-```sql
-SELECT
- "pickup_longitude" AS "pickup_longitude",
- ROUND("pickup_longitude", 0) as "rounded_pickup_longitude"
-FROM "taxi-trips"
-WHERE "pickup_longitude" IS NOT NULL
-LIMIT 1
-```
-Returns the following:
-
-| `pickup_longitude` | `rounded_pickup_longitude` |
-| -- | -- |
-| `-73.9377670288086` | `-74` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## ROW_NUMBER
`ROW_NUMBER()`
@@ -2317,133 +1325,44 @@ Returns the number of the row within the window starting from 1.
## RPAD
-Returns a string of size `length` from `expr`. When the length of `expr` is less than `length`, right pads `expr` with `chars`, which defaults to the space character. Truncates `expr` to `length` if `length` is shorter than the length of `expr`.
+`RPAD(, , [])`
-* **Syntax:** `RPAD(expr, length[, chars])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example right pads the value of `OriginStateName` from the `flight-carriers` datasource to return a total of 11 characters.
-
-```sql
-SELECT
- "OriginStateName" AS "origin_state",
- RPAD("OriginStateName", 11, '+') AS "add_right_padding"
-FROM "flight-carriers"
-LIMIT 3
-```
-
-Returns the following:
-
-| `origin_state` | `add_right_padding` |
-| -- | -- |
-| `Puerto Rico` | `Puerto Rico` |
-| `Massachusetts` | `Massachuset` |
-| `Florida` | `Florida++++` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns the rightmost number of characters from an expression, optionally padded with the given characters.
## RTRIM
-Trims characters from the trailing end of an expression. Defaults `chars` to a space if none is provided.
+`RTRIM(, [])`
-* **Syntax:** `RTRIM(expr[, chars])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example trims the `_` characters from the trailing end of the string expression.
-
-```sql
-SELECT
- '___abc___' AS "original_string",
- RTRIM('___abc___', '_') AS "trim_end"
-```
-
-Returns the following:
-
-| `original_string` | `trim_end` |
-| -- | -- |
-| `___abc___` | `___abc` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Trims characters from the trailing end of an expression.
## SAFE_DIVIDE
+`SAFE_DIVIDE(x, y)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Returns `x` divided by `y`, guarded on division by 0.
-* **Syntax:** `SAFE_DIVIDE(x, y)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example calculates divisions of integer `78` by integer `10`.
-
-```sql
-SELECT SAFE_DIVIDE(78, 10) AS "safe_division"
-```
-
-Returns the following:
-
-|`safe_division`|
-|--|
-| `7` |
-
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## SIN
+`SIN(expr)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the trigonometric sine of an angle expressed in radians.
-* **Syntax:** `SIN(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example calculates the sine of angle `PI/3` radians.
-
-```sql
-SELECT SIN(PI / 3) AS "sine"
-```
-Returns the following:
-
-| `sine` |
-| -- |
-| `0.8660254037844386` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## SQRT
+`SQRT(expr)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the square root of a numeric expression.
-* **Syntax:** `SQRT()`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example calculates the square root of 25.
-
-```sql
-SELECT SQRT(25) AS "square_root"
-```
-Returns the following:
-
-| `square_root` |
-| -- |
-| `5` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## STDDEV
`STDDEV(expr)`
@@ -2487,33 +1406,11 @@ Splits `str1` into an array on the delimiter specified by `str2`, which is a reg
## STRING_FORMAT
-Returns a string formatted in the manner of Java's [String.format](https://docs.oracle.com/javase/8/docs/api/java/lang/String.html#format-java.lang.String-java.lang.Object...-).
+`STRING_FORMAT(pattern[, args...])`
-* **Syntax:** `STRING_FORMAT(pattern[, args...])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example uses Java String format to pass in `Flight_Number_Reporting_Airline` and `origin_airport` columns, from the `flight-carriers` datasource, as arguments into the string.
-
-```sql
-SELECT
- "Flight_Number_Reporting_Airline" AS "flight_number",
- "Origin" AS "origin_airport",
- STRING_FORMAT('Flight No.%d departing from %s', "Flight_Number_Reporting_Airline", "Origin") AS "departure_announcement"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `flight_number` | `origin_airport` | `departure_announcement` |
-| -- | -- | -- |
-| `314` | `SJU` | `Flight No.314 departing from SJU` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns a string formatted in accordance to Java's String.format method.
## STRING_TO_MV
@@ -2525,78 +1422,35 @@ Splits `str1` into an multi-value string on the delimiter specified by `str2`, w
## STRLEN
+`STRLEN(expr)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Alias for [`LENGTH`](#length).
-* **Syntax:** `STRLEN(expr)`
-* **Function type:** Scalar, string
-
-[Learn more](sql-scalar.md#string-functions)
-
## STRPOS
-Returns the one-based index position of a substring within an expression. If `substring` is not found, returns 0.
+`STRPOS(, )`
-* **Syntax:** `STRPOS(expr, substring)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example returns the one-based index position of `World`.
-
-```sql
-SELECT
- 'Hello World!' AS "original_string",
- STRPOS('Hello World!', 'World') AS "index"
-```
-
-Returns the following:
-
-| `original_string` | `index` |
-| -- | -- |
-| `Hello World!` | `7` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns the one-based index position of a substring within an expression.
## SUBSTR
+`SUBSTR(, , [])`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Alias for [`SUBSTRING`](#substring).
-* **Syntax:** `SUBSTR(expr, index[, length])`
-* **Function type:** Scalar, string
-
-[Learn more](sql-scalar.md#string-functions)
-
-
## SUBSTRING
-Returns a substring of the expression starting at a given one-based index. If `length` is omitted, extracts characters to the end of the string, otherwise returns a substring of `length` characters.
+`SUBSTRING(, , [])`
-* **Syntax:** `SUBSTRING(expr, index[, length])`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example extracts a substring from the string expression `abcdefghi` of length `3` starting at index `4`
-
-```sql
-SELECT
- 'abcdefghi' AS "original_string",
- SUBSTRING('abcdefghi', 4, 3) AS "substring"
-```
-
-Returns the following:
-
-| `original_string` | `substring` |
-| -- | -- |
-| `abcdefghi` | `def` |
-
-
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Returns a substring of the expression starting at a given one-based index.
## SUM
@@ -2608,27 +1462,12 @@ Calculates the sum of a set of values.
## TAN
+`TAN(expr)`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Calculates the trigonometric tangent of an angle expressed in radians.
-* **Syntax:** `TAN(expr)`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following example calculates the tangent of angle `PI/3` radians.
-
-```sql
-SELECT TAN(PI / 3) AS "tangent"
-```
-Returns the following:
-
-| `tangent` |
-| -- |
-| `1.7320508075688767` |
-
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## TDIGEST_GENERATE_SKETCH
`TDIGEST_GENERATE_SKETCH(expr, [compression])`
@@ -2647,33 +1486,12 @@ Returns the quantile for the specified fraction from a T-Digest sketch construct
## TEXTCAT
+`TEXTCAT(, )`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Concatenates two string expressions.
-* **Syntax:** `TEXTCAT(expr, expr)`
-* **Function type:** Scalar, string
-
-Example
-
-The following example concatenates the `OriginState` column from the `flight-carriers` datasource to `, USA`.
-
-```sql
-SELECT
- "OriginState" AS "origin_state",
- TEXTCAT("OriginState", ', USA') AS "concatenate_state_with_USA"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `origin_state` | `concatenate_state_with_USA` |
-| -- | -- |
-| `PR` | `PR, USA` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
-
## THETA_SKETCH_ESTIMATE
`THETA_SKETCH_ESTIMATE(expr)`
@@ -2716,121 +1534,35 @@ Returns a union of Theta sketches.
## TIME_CEIL
-Rounds up a timestamp to a given ISO 8601 time period. You can specify `origin` to provide a reference timestamp from which to start rounding. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`.
+`TIME_CEIL(, , [, []])`
-* **Syntax:** `TIME_CEIL(timestamp_expr, period[, origin[, timezone]])`
-* **Function type:** Scalar, date and time
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
-Example
-
-The following example rounds up the `__time` column from the `taxi-trips` datasource to the nearest 45th minute in reference to the timestamp `2013-08-01 08:0:00`.
-
-```sql
-SELECT
- "__time" AS "original_timestamp",
- TIME_CEIL("__time", 'PT45M', TIMESTAMP '2013-08-01 08:0:00') AS "time_ceiling"
-FROM "taxi-trips"
-LIMIT 2
-```
-
-Returns the following:
-
-| `original_timestamp` | `time_ceiling` |
-| -- | -- |
-| `2013-08-01T08:14:37.000Z` | `2013-08-01T08:45:00.000Z` |
-| `2013-08-01T09:13:00.000Z` | `2013-08-01T09:30:00.000Z` |
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+Rounds up a timestamp by a given time period, optionally from some reference time or timezone.
## TIME_EXTRACT
-Extracts the value of `unit` from the timestamp and returns it as a number. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`.
+`TIME_EXTRACT(, [, []])`
-* **Syntax:** `TIME_EXTRACT(timestamp_expr[, unit[, timezone]])`
-* **Function type:** Scalar, date and time
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
-Example
-
-The following example extracts the hour from the `__time` column in the `taxi-trips` datasource and offsets its timezone by `-04:00` hours.
-
-```sql
-SELECT
- "__time" AS "original_timestamp",
- TIME_EXTRACT("__time", 'hour', '-04:00') AS "extract_hour"
-FROM "taxi-trips"
-LIMIT 2
-```
-
-Returns the following:
-
-| `original_timestamp` | `extract_hour` |
-| -- | -- |
-| `2013-08-01T08:14:37.000Z` | `4` |
-| `2013-08-01T09:13:00.000Z` | `5` |
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+Extracts the value of some unit of the timestamp and returns the number.
## TIME_FLOOR
-Rounds down a timestamp to a given ISO 8601 time period. You can specify `origin` to provide a reference timestamp from which to start rounding. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`.
+`TIME_FLOOR(, , [, []])`
-* **Syntax:** `TIME_FLOOR(timestamp_expr, period[, origin[, timezone]])`
-* **Function type:** Scalar, date and time
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
-Example
-
-The following example rounds down the `__time` column from the `taxi-trips` datasource to the nearest 45th minute in reference to the timestamp `2013-08-01 08:0:00`.
-
-```sql
-SELECT
- "__time" AS "original_timestamp",
- TIME_FLOOR("__time", 'PT45M', TIMESTAMP '2013-08-01 08:0:00') AS "time_floor"
-FROM "taxi-trips"
-LIMIT 2
-```
-
-Returns the following:
-
-| `original_timestamp` | `time_floor` |
-| -- | -- |
-| `2013-08-01T08:14:37.000Z` | `2013-08-01T08:00:00.000Z` |
-| `2013-08-01T09:13:00.000Z` | `2013-08-01T08:45:00.000Z` |
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+Rounds down a timestamp by a given time period, optionally from some reference time or timezone.
## TIME_FORMAT
-Formats a timestamp as a string in a provided [Joda DateTimeFormat pattern](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html). If no pattern is provided, `pattern` defaults to ISO 8601. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`.
+`TIME_FORMAT(, [, []])`
-* **Syntax:** `TIME_FORMAT(timestamp_expr[, pattern[, timezone]])`
-* **Function type:** Scalar, date and time
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
-Example
-
-The following example formats the `__time` column from the `flight-carriers` datasource into a string format and offsets the result's timezone by `-05:00` hours.
-
-```sql
-SELECT
- "__time" AS "original_time",
-TIME_FORMAT( "__time", 'dd-MM-YYYY hh:mm aa zzz', '-05:00') AS "string"
-FROM "taxi-trips"
-LIMIT 1
-```
-
-Returns the following:
-
-| `original_time` | `string` |
-| -- | -- |
-| `2013-08-01T08:14:37.000Z` | `01-08-2013 03:14 AM -05:00` |
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+Formats a timestamp as a string.
## TIME_IN_INTERVAL
@@ -2842,62 +1574,19 @@ Returns whether a timestamp is contained within a particular interval, formatted
## TIME_PARSE
-Parses a string into a timestamp using a given [Joda DateTimeFormat pattern](http://www.joda.org/joda-time/apidocs/org/joda/time/format/DateTimeFormat.html). If no pattern is provided, `pattern` defaults to ISO 8601. Returns NULL if string cannot be parsed. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`.
+`TIME_PARSE(, [, []])`
-* **Syntax:** `TIME_PARSE(string_expr[, pattern[, timezone]])`
-* **Function type:** Scalar, date and time
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
-Example
-
-The following example parses the `FlightDate` STRING column from the `flight-carriers` datasource into a valid timestamp with an offset of `-05:00` hours.
-
-```sql
-SELECT
- "FlightDate" AS "original_string",
- TIME_PARSE("FlightDate", 'YYYY-MM-dd', '-05:00') AS "timestamp"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `original_string` | `timestamp` |
-| -- | -- |
-| `2005-11-01` | `2005-11-01T05:00:00.000Z` |
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+Parses a string into a timestamp.
## TIME_SHIFT
-Shifts a timestamp by a given number of time units. The `period` parameter can be any ISO 8601 period. The `step` parameter can be negative. If provided, `timezone` should be a time zone name like `America/Los_Angeles` or an offset like `-08:00`.
+`TIME_SHIFT(, , , [])`
-* **Syntax:** `TIME_SHIFT(timestamp_expr, period, step[, timezone])`
-* **Function type:** Scalar, date and time
-
-Example
-
-The following example shifts the `__time` column from the `taxi-trips` datasource back by 24 hours.
-
-```sql
-SELECT
- "__time" AS "original_timestamp",
- TIME_SHIFT("__time", 'PT1H', -24) AS "shift_back"
-FROM "taxi-trips"
-LIMIT 1
-```
-
-Returns the following:
-
-| `original_timestamp` | `shift_back` |
-| -- | -- |
-| `2013-08-01T08:14:37.000Z` | `2013-07-31T08:14:37.000Z` |
-
-
-
-[Learn more](sql-scalar.md#date-and-time-functions)
+**Function type:** [Scalar, date and time](sql-scalar.md#date-and-time-functions)
+Shifts a timestamp forwards or backwards by a given number of time units.
## TIMESTAMP_TO_MILLIS
@@ -2928,73 +1617,34 @@ Takes the difference between two timestamps, returning the results in the given
**Function type:** [JSON](sql-json-functions.md)
`TO_JSON_STRING(expr)`
+
Serializes `expr` into a JSON string.
## TRIM
-Trims the leading and/or trailing characters of an expression. Defaults `chars` to a space if none is provided. Defaults to `BOTH` if no directional argument is provided.
+`TRIM([BOTH|LEADING|TRAILING] [ FROM] expr)`
-* **Syntax:** `TRIM([BOTH|LEADING|TRAILING] [chars FROM] expr)`
-* **Function type:** Scalar, string
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
-Example
-
-The following example trims `_` characters from both ends of the string expression.
-
-```sql
-SELECT
- '___abc___' AS "original_string",
- TRIM( BOTH '_' FROM '___abc___') AS "trim_expression"
-```
-
-Returns the following:
-
-| `original_string` | `trim_expression` |
-| -- | -- |
-| `___abc___` | `abc` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
+Trims the leading or trailing characters of an expression.
## TRUNC
+`TRUNC(expr[, digits])`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Alias for [`TRUNCATE`](#truncate).
-* **Syntax:** `TRUNC(expr[, digits])`
-* **Function type:** Scalar, numeric
-
-[Learn more](sql-scalar.md#numeric-functions)
-
## TRUNCATE
+`TRUNCATE(expr[, digits])`
+
+**Function type:** [Scalar, numeric](sql-scalar.md#numeric-functions)
+
Truncates a numerical expression to a specific number of decimal digits.
-* **Syntax:** `TRUNCATE(expr[, digits])`
-* **Function type:** Scalar, numeric
-
-Example
-
-The following applies the TRUNCATE function to 1 decimal place on the `pickup_longitude` column from the `taxi-trips` datasource.
-
-```sql
-SELECT
- "pickup_longitude" as "pickup_longitude",
- TRUNCATE("pickup_longitude", 1) as "truncate_pickup_longitude"
-FROM "taxi-trips"
-WHERE "pickup_longitude" IS NOT NULL
-LIMIT 1
-```
-Returns the following:
-
-| `pickup_longitude` | `truncate_pickup_longitude` |
-| -- | -- |
-| `-73.9377670288086` | `-73.9` |
-
-
-
-[Learn more](sql-scalar.md#numeric-functions)
## TRY_PARSE_JSON
@@ -3014,33 +1664,12 @@ For more information, see [UNNEST](./sql.md#unnest).
## UPPER
+`UPPER(expr)`
+
+**Function type:** [Scalar, string](sql-scalar.md#string-functions)
+
Returns the expression in uppercase.
-* **Syntax:** `UPPER(expr)`
-* **Function type:** Scalar, string
-
-Example
-
-The following example converts the `OriginCityName` column from the `flight-carriers` datasource to uppercase.
-
-```sql
-SELECT
- "OriginCityName" AS "origin_city",
- UPPER("OriginCityName") AS "uppercase"
-FROM "flight-carriers"
-LIMIT 1
-```
-
-Returns the following:
-
-| `origin_city` | `uppercase` |
-| -- | -- |
-`San Juan, PR` | `SAN JUAN, PR` |
-
-
-
-[Learn more](sql-scalar.md#string-functions)
-
## VAR_POP
`VAR_POP(expr)`
@@ -3064,5 +1693,3 @@ Calculates the sample variance of a set of values.
**Function type:** [Aggregation](sql-aggregations.md)
Alias for [`VAR_SAMP`](#var_samp).
-
-
diff --git a/docs/querying/sql-json-functions.md b/docs/querying/sql-json-functions.md
index 199c568c29b..35b4f5e3769 100644
--- a/docs/querying/sql-json-functions.md
+++ b/docs/querying/sql-json-functions.md
@@ -38,6 +38,7 @@ You can use the following JSON functions to extract, transform, and create `COMP
| --- | --- |
|`JSON_KEYS(expr, path)`| Returns an array of field names from `expr` at the specified `path`.|
|`JSON_OBJECT(KEY expr1 VALUE expr2[, KEY expr3 VALUE expr4, ...])` | Constructs a new `COMPLEX` object. The `KEY` expressions must evaluate to string types. The `VALUE` expressions can be composed of any input type, including other `COMPLEX` values. `JSON_OBJECT` can accept colon-separated key-value pairs. The following syntax is equivalent: `JSON_OBJECT(expr1:expr2[, expr3:expr4, ...])`.|
+|`JSON_MERGE(expr1, expr2[, expr3 ...])`| Merges two or more JSON `STRING` or `COMPLEX` into one. Preserves the rightmost value when there are key overlaps. Returning always a `COMPLEX` type.|
|`JSON_PATHS(expr)`| Returns an array of all paths which refer to literal values in `expr` in JSONPath format. |
|`JSON_QUERY(expr, path)`| Extracts a `COMPLEX` value from `expr`, at the specified `path`. |
|`JSON_QUERY_ARRAY(expr, path)`| Extracts an `ARRAY>` value from `expr` at the specified `path`. If value is not an `ARRAY`, it gets translated into a single element `ARRAY` containing the value at `path`. The primary use of this function is to extract arrays of objects to use as inputs to other [array functions](./sql-array-functions.md).|
diff --git a/docs/querying/sql-metadata-tables.md b/docs/querying/sql-metadata-tables.md
index 4fbbfb2ff4c..b04b3f11e65 100644
--- a/docs/querying/sql-metadata-tables.md
+++ b/docs/querying/sql-metadata-tables.md
@@ -266,7 +266,7 @@ GROUP BY servers.server;
### TASKS table
-The tasks table provides information about active and recently-completed indexing tasks. For more information
+The tasks table provides information about active and recently completed tasks. For more information
check out the documentation for [ingestion tasks](../ingestion/tasks.md).
|Column|Type|Notes|
diff --git a/docs/querying/sql-scalar.md b/docs/querying/sql-scalar.md
index 28e889b8887..8a8c47160da 100644
--- a/docs/querying/sql-scalar.md
+++ b/docs/querying/sql-scalar.md
@@ -173,10 +173,10 @@ overhead.
|`MILLIS_TO_TIMESTAMP(millis_expr)`|Converts a number of milliseconds since the epoch (1970-01-01 00:00:00 UTC) into a timestamp.|
|`TIMESTAMP_TO_MILLIS(timestamp_expr)`|Converts a timestamp into a number of milliseconds since the epoch.|
|`EXTRACT(unit FROM timestamp_expr)`|Extracts a time part from `expr`, returning it as a number. Unit can be EPOCH, MICROSECOND, MILLISECOND, SECOND, MINUTE, HOUR, DAY (day of month), DOW (day of week), ISODOW (ISO day of week), DOY (day of year), WEEK (week of year), MONTH, QUARTER, YEAR, ISOYEAR, DECADE, CENTURY or MILLENNIUM. Units must be provided unquoted, like `EXTRACT(HOUR FROM __time)`.|
-|`FLOOR(timestamp_expr TO unit)`|Rounds down a timestamp, returning it as a new timestamp. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
-|`CEIL(timestamp_expr TO unit)`|Rounds up a timestamp, returning it as a new timestamp. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
-|`TIMESTAMPADD(unit, count, timestamp)`|Equivalent to `timestamp + count * INTERVAL '1' UNIT`.|
-|`TIMESTAMPDIFF(unit, timestamp1, timestamp2)`|Returns the (signed) number of `unit` between `timestamp1` and `timestamp2`. Unit can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
+|`FLOOR(timestamp_expr TO unit)`|Rounds down a timestamp, returning it as a new timestamp. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
+|`CEIL(timestamp_expr TO unit)`|Rounds up a timestamp, returning it as a new timestamp. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
+|`TIMESTAMPADD(unit, count, timestamp)`|Adds a `count` number of time `unit` to timestamp, equivalent to `timestamp + count * unit`. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
+|`TIMESTAMPDIFF(unit, timestamp1, timestamp2)`|Returns a signed number of `unit` between `timestamp1` and `timestamp2`. The `unit` parameter must be unquoted and can be SECOND, MINUTE, HOUR, DAY, WEEK, MONTH, QUARTER, or YEAR.|
## Reduction functions
diff --git a/examples/conf/druid/auto/_common/common.jvm.config b/examples/conf/druid/auto/_common/common.jvm.config
index fd74cf35897..0c488a2bcea 100644
--- a/examples/conf/druid/auto/_common/common.jvm.config
+++ b/examples/conf/druid/auto/_common/common.jvm.config
@@ -1,6 +1,5 @@
-server
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config b/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config
index 5df7d606725..00cfa42c630 100644
--- a/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config
+++ b/examples/conf/druid/cluster/master/coordinator-overlord/jvm.config
@@ -2,7 +2,6 @@
-Xms15g
-Xmx15g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/cluster/query/router/jvm.config b/examples/conf/druid/cluster/query/router/jvm.config
index a844c40b1ea..8fd91200735 100644
--- a/examples/conf/druid/cluster/query/router/jvm.config
+++ b/examples/conf/druid/cluster/query/router/jvm.config
@@ -1,7 +1,6 @@
-server
-Xms1g
-Xmx1g
--XX:+UseG1GC
-XX:MaxDirectMemorySize=128m
-XX:+ExitOnOutOfMemoryError
-Duser.timezone=UTC
diff --git a/examples/conf/druid/single-server/large/broker/jvm.config b/examples/conf/druid/single-server/large/broker/jvm.config
index 6c43c24dbb4..f554b705e10 100644
--- a/examples/conf/druid/single-server/large/broker/jvm.config
+++ b/examples/conf/druid/single-server/large/broker/jvm.config
@@ -3,7 +3,6 @@
-Xmx12g
-XX:MaxDirectMemorySize=11g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config
index 5df7d606725..00cfa42c630 100644
--- a/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config
+++ b/examples/conf/druid/single-server/large/coordinator-overlord/jvm.config
@@ -2,7 +2,6 @@
-Xms15g
-Xmx15g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/large/historical/jvm.config b/examples/conf/druid/single-server/large/historical/jvm.config
index 16e1f5d7825..8b105559b00 100644
--- a/examples/conf/druid/single-server/large/historical/jvm.config
+++ b/examples/conf/druid/single-server/large/historical/jvm.config
@@ -3,7 +3,6 @@
-Xmx16g
-XX:MaxDirectMemorySize=25g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/large/middleManager/jvm.config b/examples/conf/druid/single-server/large/middleManager/jvm.config
index 82597e0b956..69d307c3120 100644
--- a/examples/conf/druid/single-server/large/middleManager/jvm.config
+++ b/examples/conf/druid/single-server/large/middleManager/jvm.config
@@ -2,7 +2,6 @@
-Xms256m
-Xmx256m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/large/router/jvm.config b/examples/conf/druid/single-server/large/router/jvm.config
index a844c40b1ea..8fd91200735 100644
--- a/examples/conf/druid/single-server/large/router/jvm.config
+++ b/examples/conf/druid/single-server/large/router/jvm.config
@@ -1,7 +1,6 @@
-server
-Xms1g
-Xmx1g
--XX:+UseG1GC
-XX:MaxDirectMemorySize=128m
-XX:+ExitOnOutOfMemoryError
-Duser.timezone=UTC
diff --git a/examples/conf/druid/single-server/medium/broker/jvm.config b/examples/conf/druid/single-server/medium/broker/jvm.config
index a4bf3d91097..a2cbe790878 100644
--- a/examples/conf/druid/single-server/medium/broker/jvm.config
+++ b/examples/conf/druid/single-server/medium/broker/jvm.config
@@ -3,7 +3,6 @@
-Xmx8g
-XX:MaxDirectMemorySize=5g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config
index dbddd50ce8d..485137ad3d8 100644
--- a/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config
+++ b/examples/conf/druid/single-server/medium/coordinator-overlord/jvm.config
@@ -2,7 +2,6 @@
-Xms9g
-Xmx9g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/medium/historical/jvm.config b/examples/conf/druid/single-server/medium/historical/jvm.config
index f31ae92ba9f..891312f3c68 100644
--- a/examples/conf/druid/single-server/medium/historical/jvm.config
+++ b/examples/conf/druid/single-server/medium/historical/jvm.config
@@ -3,7 +3,6 @@
-Xmx8g
-XX:MaxDirectMemorySize=13g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/medium/middleManager/jvm.config b/examples/conf/druid/single-server/medium/middleManager/jvm.config
index 82597e0b956..69d307c3120 100644
--- a/examples/conf/druid/single-server/medium/middleManager/jvm.config
+++ b/examples/conf/druid/single-server/medium/middleManager/jvm.config
@@ -2,7 +2,6 @@
-Xms256m
-Xmx256m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/medium/router/jvm.config b/examples/conf/druid/single-server/medium/router/jvm.config
index 6fc9b4cdfef..ebecad4435e 100644
--- a/examples/conf/druid/single-server/medium/router/jvm.config
+++ b/examples/conf/druid/single-server/medium/router/jvm.config
@@ -1,7 +1,6 @@
-server
-Xms512m
-Xmx512m
--XX:+UseG1GC
-XX:MaxDirectMemorySize=128m
-XX:+ExitOnOutOfMemoryError
-Duser.timezone=UTC
diff --git a/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config b/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config
index cf72e5d2df9..5d666f6db23 100644
--- a/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config
+++ b/examples/conf/druid/single-server/micro-quickstart/broker/jvm.config
@@ -3,7 +3,6 @@
-Xmx512m
-XX:MaxDirectMemorySize=768m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config
index 8ab5708c320..aae520d8325 100644
--- a/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config
+++ b/examples/conf/druid/single-server/micro-quickstart/coordinator-overlord/jvm.config
@@ -2,7 +2,6 @@
-Xms256m
-Xmx256m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config b/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config
index 8747402e85a..79edb06e203 100644
--- a/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config
+++ b/examples/conf/druid/single-server/micro-quickstart/historical/jvm.config
@@ -3,7 +3,6 @@
-Xmx512m
-XX:MaxDirectMemorySize=1280m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config b/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config
index 30b023cfb8d..8a650fb358c 100644
--- a/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config
+++ b/examples/conf/druid/single-server/micro-quickstart/middleManager/jvm.config
@@ -2,7 +2,6 @@
-Xms64m
-Xmx64m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/micro-quickstart/router/jvm.config b/examples/conf/druid/single-server/micro-quickstart/router/jvm.config
index e74d6b1375f..b822f460179 100644
--- a/examples/conf/druid/single-server/micro-quickstart/router/jvm.config
+++ b/examples/conf/druid/single-server/micro-quickstart/router/jvm.config
@@ -1,7 +1,6 @@
-server
-Xms128m
-Xmx128m
--XX:+UseG1GC
-XX:MaxDirectMemorySize=128m
-XX:+ExitOnOutOfMemoryError
-Duser.timezone=UTC
diff --git a/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config b/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config
index 8551ec59fb0..93d33903541 100644
--- a/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config
+++ b/examples/conf/druid/single-server/nano-quickstart/broker/jvm.config
@@ -3,7 +3,6 @@
-Xmx512m
-XX:MaxDirectMemorySize=400m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config
index 8ab5708c320..aae520d8325 100644
--- a/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config
+++ b/examples/conf/druid/single-server/nano-quickstart/coordinator-overlord/jvm.config
@@ -2,7 +2,6 @@
-Xms256m
-Xmx256m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config b/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config
index 8551ec59fb0..93d33903541 100644
--- a/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config
+++ b/examples/conf/druid/single-server/nano-quickstart/historical/jvm.config
@@ -3,7 +3,6 @@
-Xmx512m
-XX:MaxDirectMemorySize=400m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config b/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config
index 30b023cfb8d..8a650fb358c 100644
--- a/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config
+++ b/examples/conf/druid/single-server/nano-quickstart/middleManager/jvm.config
@@ -2,7 +2,6 @@
-Xms64m
-Xmx64m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/nano-quickstart/router/jvm.config b/examples/conf/druid/single-server/nano-quickstart/router/jvm.config
index e74d6b1375f..b822f460179 100644
--- a/examples/conf/druid/single-server/nano-quickstart/router/jvm.config
+++ b/examples/conf/druid/single-server/nano-quickstart/router/jvm.config
@@ -1,7 +1,6 @@
-server
-Xms128m
-Xmx128m
--XX:+UseG1GC
-XX:MaxDirectMemorySize=128m
-XX:+ExitOnOutOfMemoryError
-Duser.timezone=UTC
diff --git a/examples/conf/druid/single-server/small/broker/jvm.config b/examples/conf/druid/single-server/small/broker/jvm.config
index 1017bce2ab5..1bc85dd7702 100644
--- a/examples/conf/druid/single-server/small/broker/jvm.config
+++ b/examples/conf/druid/single-server/small/broker/jvm.config
@@ -3,7 +3,6 @@
-Xmx4g
-XX:MaxDirectMemorySize=3g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config
index 34176680c41..2d9b10d9501 100644
--- a/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config
+++ b/examples/conf/druid/single-server/small/coordinator-overlord/jvm.config
@@ -2,7 +2,6 @@
-Xms4500m
-Xmx4500m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/small/historical/jvm.config b/examples/conf/druid/single-server/small/historical/jvm.config
index 21906078471..a567647d543 100644
--- a/examples/conf/druid/single-server/small/historical/jvm.config
+++ b/examples/conf/druid/single-server/small/historical/jvm.config
@@ -3,7 +3,6 @@
-Xmx4g
-XX:MaxDirectMemorySize=8g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/small/middleManager/jvm.config b/examples/conf/druid/single-server/small/middleManager/jvm.config
index a5862186437..c145532ba1e 100644
--- a/examples/conf/druid/single-server/small/middleManager/jvm.config
+++ b/examples/conf/druid/single-server/small/middleManager/jvm.config
@@ -2,7 +2,6 @@
-Xms128m
-Xmx128m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/small/router/jvm.config b/examples/conf/druid/single-server/small/router/jvm.config
index ad8a52de508..ebecad4435e 100644
--- a/examples/conf/druid/single-server/small/router/jvm.config
+++ b/examples/conf/druid/single-server/small/router/jvm.config
@@ -1,10 +1,8 @@
-server
-Xms512m
-Xmx512m
--XX:+UseG1GC
-XX:MaxDirectMemorySize=128m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/xlarge/broker/jvm.config b/examples/conf/druid/single-server/xlarge/broker/jvm.config
index f83ad0e18b6..b683fd7dd7a 100644
--- a/examples/conf/druid/single-server/xlarge/broker/jvm.config
+++ b/examples/conf/druid/single-server/xlarge/broker/jvm.config
@@ -3,7 +3,6 @@
-Xmx16g
-XX:MaxDirectMemorySize=12g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config b/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config
index f3ca0fdbb9e..68f9f93c60d 100644
--- a/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config
+++ b/examples/conf/druid/single-server/xlarge/coordinator-overlord/jvm.config
@@ -2,7 +2,6 @@
-Xms18g
-Xmx18g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/xlarge/historical/jvm.config b/examples/conf/druid/single-server/xlarge/historical/jvm.config
index 92553d3c8b4..6d91df0aef0 100644
--- a/examples/conf/druid/single-server/xlarge/historical/jvm.config
+++ b/examples/conf/druid/single-server/xlarge/historical/jvm.config
@@ -3,7 +3,6 @@
-Xmx24g
-XX:MaxDirectMemorySize=44g
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/xlarge/middleManager/jvm.config b/examples/conf/druid/single-server/xlarge/middleManager/jvm.config
index 82597e0b956..69d307c3120 100644
--- a/examples/conf/druid/single-server/xlarge/middleManager/jvm.config
+++ b/examples/conf/druid/single-server/xlarge/middleManager/jvm.config
@@ -2,7 +2,6 @@
-Xms256m
-Xmx256m
-XX:+ExitOnOutOfMemoryError
--XX:+UseG1GC
-Duser.timezone=UTC
-Dfile.encoding=UTF-8
-Djava.io.tmpdir=var/tmp
diff --git a/examples/conf/druid/single-server/xlarge/router/jvm.config b/examples/conf/druid/single-server/xlarge/router/jvm.config
index a844c40b1ea..8fd91200735 100644
--- a/examples/conf/druid/single-server/xlarge/router/jvm.config
+++ b/examples/conf/druid/single-server/xlarge/router/jvm.config
@@ -1,7 +1,6 @@
-server
-Xms1g
-Xmx1g
--XX:+UseG1GC
-XX:MaxDirectMemorySize=128m
-XX:+ExitOnOutOfMemoryError
-Duser.timezone=UTC
diff --git a/extensions-contrib/aliyun-oss-extensions/pom.xml b/extensions-contrib/aliyun-oss-extensions/pom.xml
index f83b0b5b339..bffaed8c065 100644
--- a/extensions-contrib/aliyun-oss-extensions/pom.xml
+++ b/extensions-contrib/aliyun-oss-extensions/pom.xml
@@ -28,7 +28,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java
index 207cddf90b4..12ad9fc8bf9 100644
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java
+++ b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java
@@ -537,7 +537,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest
InputSourceReader reader = inputSource.reader(
someSchema,
- new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
+ new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
temporaryFolder.newFolder()
);
@@ -584,7 +584,7 @@ public class OssInputSourceTest extends InitializedNullHandlingTest
InputSourceReader reader = inputSource.reader(
someSchema,
- new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
+ new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0, null),
temporaryFolder.newFolder()
);
diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml
index 95773a63918..f1ad243e37c 100644
--- a/extensions-contrib/ambari-metrics-emitter/pom.xml
+++ b/extensions-contrib/ambari-metrics-emitter/pom.xml
@@ -24,7 +24,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml
index f502562a816..7b5fc2be183 100644
--- a/extensions-contrib/cassandra-storage/pom.xml
+++ b/extensions-contrib/cassandra-storage/pom.xml
@@ -29,7 +29,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml
index 2fca74ef55b..01b006dd5d5 100644
--- a/extensions-contrib/cloudfiles-extensions/pom.xml
+++ b/extensions-contrib/cloudfiles-extensions/pom.xml
@@ -29,7 +29,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/extensions-contrib/compressed-bigdecimal/pom.xml b/extensions-contrib/compressed-bigdecimal/pom.xml
index d6c27d88812..358bc3b26c5 100644
--- a/extensions-contrib/compressed-bigdecimal/pom.xml
+++ b/extensions-contrib/compressed-bigdecimal/pom.xml
@@ -25,7 +25,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/extensions-contrib/ddsketch/pom.xml b/extensions-contrib/ddsketch/pom.xml
index 7318e953ff7..9c737a75c6a 100644
--- a/extensions-contrib/ddsketch/pom.xml
+++ b/extensions-contrib/ddsketch/pom.xml
@@ -22,7 +22,7 @@
druidorg.apache.druid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml4.0.0
diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml
index 977d7550947..fd88a5bca2c 100644
--- a/extensions-contrib/distinctcount/pom.xml
+++ b/extensions-contrib/distinctcount/pom.xml
@@ -29,7 +29,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
index c06b5649681..d3edf0827b3 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
@@ -35,8 +35,8 @@ import org.apache.druid.query.timeseries.TimeseriesResultValue;
import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.IncrementalIndex;
+import org.apache.druid.segment.incremental.IncrementalIndexCursorFactory;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.testing.InitializedNullHandlingTest;
import org.joda.time.DateTime;
@@ -104,7 +104,7 @@ public class DistinctCountTimeseriesQueryTest extends InitializedNullHandlingTes
final Iterable> results =
engine.process(
query,
- new IncrementalIndexStorageAdapter(index),
+ new IncrementalIndexCursorFactory(index),
new IncrementalIndexTimeBoundaryInspector(index),
new DefaultTimeseriesQueryMetrics()
).toList();
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
index ff5a0c31363..7e3690cc1cb 100644
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
+++ b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
@@ -33,13 +33,13 @@ import org.apache.druid.query.topn.TopNQuery;
import org.apache.druid.query.topn.TopNQueryBuilder;
import org.apache.druid.query.topn.TopNQueryEngine;
import org.apache.druid.query.topn.TopNResultValue;
-import org.apache.druid.segment.IncrementalIndexTimeBoundaryInspector;
+import org.apache.druid.segment.IncrementalIndexSegment;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.testing.InitializedNullHandlingTest;
+import org.apache.druid.timeline.SegmentId;
import org.joda.time.DateTime;
import org.junit.After;
import org.junit.Before;
@@ -133,8 +133,7 @@ public class DistinctCountTopNQueryTest extends InitializedNullHandlingTest
final Iterable> results =
engine.query(
query,
- new IncrementalIndexStorageAdapter(index),
- new IncrementalIndexTimeBoundaryInspector(index),
+ new IncrementalIndexSegment(index, SegmentId.dummy(QueryRunnerTestHelper.DATA_SOURCE)),
null
).toList();
diff --git a/extensions-contrib/dropwizard-emitter/pom.xml b/extensions-contrib/dropwizard-emitter/pom.xml
index b53950bf13f..028ac0eb48e 100644
--- a/extensions-contrib/dropwizard-emitter/pom.xml
+++ b/extensions-contrib/dropwizard-emitter/pom.xml
@@ -24,7 +24,7 @@
org.apache.druiddruid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml
diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml
index 71a167c315e..6a2b7cc0646 100644
--- a/extensions-contrib/druid-deltalake-extensions/pom.xml
+++ b/extensions-contrib/druid-deltalake-extensions/pom.xml
@@ -29,7 +29,7 @@
druidorg.apache.druid
- 31.0.0-SNAPSHOT
+ 32.0.0-SNAPSHOT../../pom.xml4.0.0
diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java
index 01a18e9bc85..c4c2f2668b0 100644
--- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java
+++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java
@@ -67,9 +67,9 @@ import java.util.stream.Collectors;
import java.util.stream.Stream;
/**
- * Input source to ingest data from a Delta Lake. This input source reads the latest snapshot from a Delta table
- * specified by {@code tablePath} parameter. If {@code filter} is specified, it's used at the Kernel level
- * for data pruning. The filtering behavior is as follows:
+ * Input source to ingest data from a Delta Lake. This input source reads the given {@code snapshotVersion} from a Delta
+ * table specified by {@code tablePath} parameter, or the latest snapshot if it's not specified.
+ * If {@code filter} is specified, it's used at the Kernel level for data pruning. The filtering behavior is as follows:
*