Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension


Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
21 changes: 14 additions & 7 deletions docs/operations/metrics.md
Original file line number Diff line number Diff line change
Expand Up @@ -85,6 +85,13 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`subquery/fallback/unknownReason/count`|Number of subqueries which cannot be materialized as frames due other reasons.|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
|`query/rowLimit/exceeded/count`|Number of queries whose inlined subquery results exceeded the given row limit|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
|`query/byteLimit/exceeded/count`|Number of queries whose inlined subquery results exceeded the given byte limit|This metric is only available if the `SubqueryCountStatsMonitor` module is included.| |
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be ideally 0, though a higher number isn't representative of a problem.|
|`mergeBuffer/used`|Number of merge buffers used from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
|`mergeBuffer/queries`|Number of groupBy queries that acquired a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
|`mergeBuffer/acquisitionTimeNs`|Total time in nanoseconds to acquire merge buffer for groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|
|`groupBy/spilledQueries`|Number of groupBy queries that have spilled onto the disk.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|
|`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|
|`groupBy/mergeDictionarySize`|Size of on-heap merge dictionary in bytes.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|

### Historical

Expand All @@ -102,6 +109,13 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be ideally 0, though a higher number isn't representative of a problem.|
|`mergeBuffer/used`|Number of merge buffers used from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
|`mergeBuffer/queries`|Number of groupBy queries that acquired a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
|`mergeBuffer/acquisitionTimeNs`|Total time in nanoseconds to acquire merge buffer for groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|
|`groupBy/spilledQueries`|Number of groupBy queries that have spilled onto the disk.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|
|`groupBy/spilledBytes`|Number of bytes spilled on the disk by the groupBy queries.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|
|`groupBy/mergeDictionarySize`|Size of on-heap merge dictionary in bytes.|This metric is only available if the `GroupByStatsMonitor` module is included.|Varies|

### Real-time

Expand All @@ -117,13 +131,6 @@ Most metric values reset each emission period, as specified in `druid.monitoring
|`query/failed/count`|Number of failed queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/interrupted/count`|Number of queries interrupted due to cancellation.|This metric is only available if the `QueryCountStatsMonitor` module is included.||
|`query/timeout/count`|Number of timed out queries.|This metric is only available if the `QueryCountStatsMonitor` module is included.||

### GroupBy query metrics

These metrics are reported from broker, historical and real-time nodes

|Metric|Description|Normal value|
|------|-----------|------------|
|`mergeBuffer/pendingRequests`|Number of requests waiting to acquire a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Should be ideally 0, though a higher number isn't representative of a problem.|
|`mergeBuffer/used`|Number of merge buffers used from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
|`mergeBuffer/queries`|Number of groupBy queries that acquired a batch of buffers from the merge buffer pool.|This metric is only available if the `GroupByStatsMonitor` module is included.|Depends on the number of groupBy queries needing merge buffers.|
Expand Down
2 changes: 1 addition & 1 deletion docs/querying/multi-value-dimensions.md
Original file line number Diff line number Diff line change
Expand Up @@ -252,7 +252,7 @@ results in:
```sql
SELECT label, tags
FROM "mvd_example_rollup"
WHERE tags = 't3'
WHERE label in ('row1','row2')
GROUP BY 1,2
```

Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,15 +19,17 @@

package org.apache.druid.msq.quidem;

import org.apache.druid.quidem.DruidQuidemCommandHandler;
import org.apache.druid.quidem.DruidQuidemTestBase;
import org.apache.druid.quidem.ProjectPathUtils;

import java.io.File;

public class MSQQuidemTest extends DruidQuidemTestBase
{
public MSQQuidemTest()
{
super();
super(new DruidQuidemCommandHandler());
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,8 +19,6 @@

package org.apache.druid.msq.test;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Injector;
import org.apache.druid.guice.IndexingServiceTuningConfigModule;
import org.apache.druid.guice.JoinableFactoryModule;
import org.apache.druid.initialization.DruidModule;
Expand All @@ -29,7 +27,6 @@
import org.apache.druid.msq.guice.MSQIndexingModule;
import org.apache.druid.msq.sql.MSQTaskSqlEngine;
import org.apache.druid.msq.test.CalciteMSQTestsHelper.MSQTestModule;
import org.apache.druid.server.QueryLifecycleFactory;
import org.apache.druid.sql.calcite.run.SqlEngine;
import org.apache.druid.sql.calcite.util.DruidModuleCollection;
import org.apache.druid.sql.calcite.util.SqlTestFramework.QueryComponentSupplier;
Expand Down Expand Up @@ -61,17 +58,15 @@ public DruidModule getCoreModule()
}

@Override
public SqlEngine createEngine(
QueryLifecycleFactory qlf,
ObjectMapper queryJsonMapper,
Injector injector)
public Class<? extends SqlEngine> getSqlEngineClass()
{
return injector.getInstance(MSQTaskSqlEngine.class);
return MSQTaskSqlEngine.class;
}

@Override
public Boolean isExplainSupported()
{
return false;
}

}
Original file line number Diff line number Diff line change
@@ -0,0 +1,32 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing,
* software distributed under the License is distributed on an
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
* KIND, either express or implied. See the License for the
* specific language governing permissions and limitations
* under the License.
*/

package org.apache.druid.msq.test;

import org.apache.druid.sql.calcite.MultiComponentSupplier;
import org.apache.druid.sql.calcite.MultiComponentSupplier.ComponentSuppliers;
import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier;

@ComponentSuppliers({
StandardComponentSupplier.class,
DartComponentSupplier.class,
StandardMSQComponentSupplier.class})
public class AllDruidEnginesComponentSupplier extends MultiComponentSupplier
{
}
Original file line number Diff line number Diff line change
Expand Up @@ -45,12 +45,8 @@
import org.apache.druid.msq.guice.MSQExternalDataSourceModule;
import org.apache.druid.msq.guice.MSQIndexingModule;
import org.apache.druid.msq.querykit.DataSegmentProvider;
import org.apache.druid.query.DruidProcessingConfig;
import org.apache.druid.query.ForwardingQueryProcessingPool;
import org.apache.druid.query.QueryProcessingPool;
import org.apache.druid.query.groupby.GroupByQueryConfig;
import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
import org.apache.druid.query.groupby.GroupingEngine;
import org.apache.druid.query.groupby.TestGroupByBuffers;
import org.apache.druid.segment.CompleteSegment;
import org.apache.druid.segment.TestIndex;
Expand Down Expand Up @@ -98,15 +94,6 @@ public void configure(Binder binder)
{
}).annotatedWith(Self.class).toInstance(ImmutableSet.of(NodeRole.PEON));

DruidProcessingConfig druidProcessingConfig = new DruidProcessingConfig()
{
@Override
public String getFormatString()
{
return "test";
}
};
binder.bind(DruidProcessingConfig.class).toInstance(druidProcessingConfig);
binder.bind(QueryProcessingPool.class)
.toInstance(new ForwardingQueryProcessingPool(Execs.singleThreaded("Test-runner-processing-pool")));

Expand Down Expand Up @@ -174,26 +161,13 @@ public Supplier<ResourceHolder<CompleteSegment>> fetchSegment(
CompleteSegment a = walker.getSegment(segmentId);
return () -> new ReferenceCountingResourceHolder<>(a, Closer.create());
}

}

@Provides
public DataServerQueryHandlerFactory provideDataServerQueryHandlerFactory()
{
return getTestDataServerQueryHandlerFactory();
}

@Provides
@LazySingleton
GroupingEngine getGroupingEngine(GroupByQueryConfig groupByQueryConfig, TestGroupByBuffers groupByBuffers)
{
GroupingEngine groupingEngine = GroupByQueryRunnerTest.makeQueryRunnerFactory(
groupByQueryConfig,
groupByBuffers
).getGroupingEngine();
return groupingEngine;
}

}

@Deprecated
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,9 +19,7 @@

package org.apache.druid.msq.test;

import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.inject.Binder;
import com.google.inject.Injector;
import com.google.inject.Provides;
import org.apache.druid.collections.NonBlockingPool;
import org.apache.druid.discovery.DruidNodeDiscoveryProvider;
Expand All @@ -41,7 +39,6 @@
import org.apache.druid.query.TestBufferPool;
import org.apache.druid.rpc.ServiceClientFactory;
import org.apache.druid.rpc.guice.ServiceClientModule;
import org.apache.druid.server.QueryLifecycleFactory;
import org.apache.druid.sql.avatica.DartDruidMeta;
import org.apache.druid.sql.avatica.DruidMeta;
import org.apache.druid.sql.calcite.TempDirProducer;
Expand Down Expand Up @@ -90,12 +87,9 @@ public DruidModule getOverrideModule()
}

@Override
public SqlEngine createEngine(
QueryLifecycleFactory qlf,
ObjectMapper queryJsonMapper,
Injector injector)
public Class<? extends SqlEngine> getSqlEngineClass()
{
return injector.getInstance(DartSqlEngine.class);
return DartSqlEngine.class;
}

static class DartTestCoreModule implements DruidModule
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -25,14 +25,23 @@
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.segment.AutoTypeColumnSchema;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;

public class DimensionSchemaUtilsTest
{

@Before
public void setup()
{
EmittingLogger.registerEmitter(new NoopServiceEmitter());
}

@Test
public void testSchemaScalars()
{
Expand Down Expand Up @@ -179,19 +188,28 @@ public void testSchemaMvdMode()
DruidException.class,
() -> DimensionSchemaUtils.createDimensionSchema("x", ColumnType.LONG_ARRAY, false, ArrayIngestMode.MVD)
);
Assert.assertEquals("Numeric arrays can only be ingested when 'arrayIngestMode' is set to 'array'. Current value of the parameter is[mvd]", t.getMessage());
Assert.assertEquals(
"Numeric arrays can only be ingested when 'arrayIngestMode' is set to 'array'. Current value of the parameter is[mvd]",
t.getMessage()
);

t = Assert.assertThrows(
DruidException.class,
() -> DimensionSchemaUtils.createDimensionSchema("x", ColumnType.DOUBLE_ARRAY, false, ArrayIngestMode.MVD)
);
Assert.assertEquals("Numeric arrays can only be ingested when 'arrayIngestMode' is set to 'array'. Current value of the parameter is[mvd]", t.getMessage());
Assert.assertEquals(
"Numeric arrays can only be ingested when 'arrayIngestMode' is set to 'array'. Current value of the parameter is[mvd]",
t.getMessage()
);

t = Assert.assertThrows(
DruidException.class,
() -> DimensionSchemaUtils.createDimensionSchema("x", ColumnType.FLOAT_ARRAY, false, ArrayIngestMode.MVD)
);
Assert.assertEquals("Numeric arrays can only be ingested when 'arrayIngestMode' is set to 'array'. Current value of the parameter is[mvd]", t.getMessage());
Assert.assertEquals(
"Numeric arrays can only be ingested when 'arrayIngestMode' is set to 'array'. Current value of the parameter is[mvd]",
t.getMessage()
);
}

@Test
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -62,7 +62,7 @@ public TypeReference<Integer> getReturnTypeReference()
public Integer perform(Task task, TaskActionToolbox toolbox)
{
return toolbox.getIndexerMetadataStorageCoordinator()
.markSegmentsAsUnusedWithinInterval(dataSource, interval);
.markSegmentsWithinIntervalAsUnused(dataSource, interval, null);
}

}
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.indexing.common.task.Task;

import java.util.Set;
Expand All @@ -42,6 +43,8 @@ public RetrieveUpgradedFromSegmentIdsAction(
{
this.dataSource = dataSource;
this.segmentIds = segmentIds;

IdUtils.getValidSegmentIds(dataSource, segmentIds);
}

@JsonProperty
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -22,6 +22,7 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.core.type.TypeReference;
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.indexing.common.task.Task;

import java.util.Set;
Expand All @@ -47,6 +48,8 @@ public RetrieveUpgradedToSegmentIdsAction(
{
this.dataSource = dataSource;
this.segmentIds = segmentIds;

IdUtils.getValidSegmentIds(dataSource, segmentIds);
}

@JsonProperty
Expand Down
Loading
Loading