Merge remote-tracking branch 'apache/master' into quidem-kttm

This commit is contained in:
Zoltan Haindrich 2024-09-25 08:39:54 +00:00
commit 528765c9b2
267 changed files with 5682 additions and 2806 deletions

View File

@ -78,7 +78,6 @@ import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.DruidOperatorTable;
import org.apache.druid.sql.calcite.planner.DruidPlanner;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.planner.PlannerResult;
import org.apache.druid.sql.calcite.run.SqlEngine;
@ -341,7 +340,6 @@ public class SqlWindowFunctionsBenchmark
public void querySql(String sql, Blackhole blackhole)
{
final Map<String, Object> context = ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
QueryContexts.MAX_SUBQUERY_BYTES_KEY, "disabled",
QueryContexts.MAX_SUBQUERY_ROWS_KEY, -1
);

View File

@ -933,7 +933,6 @@ Host: http://ROUTER_IP:ROUTER_PORT
"context": {
"__resultFormat": "array",
"__user": "allowAll",
"enableWindowing": true,
"executionMode": "async",
"finalize": true,
"maxNumTasks": 2,

View File

@ -396,6 +396,9 @@ Metric monitoring is an essential part of Druid operations. The following monito
|`org.apache.druid.java.util.metrics.CgroupCpuSetMonitor`|Reports CPU core/HT and memory node allocations as per the `cpuset` cgroup.|
|`org.apache.druid.java.util.metrics.CgroupDiskMonitor`|Reports disk statistic as per the blkio cgroup.|
|`org.apache.druid.java.util.metrics.CgroupMemoryMonitor`|Reports memory statistic as per the memory cgroup.|
|`org.apache.druid.java.util.metrics.CgroupV2CpuMonitor`| **EXPERIMENTAL** Reports CPU usage from `cpu.stat` file. Only applicable to `cgroupv2`.|
|`org.apache.druid.java.util.metrics.CgroupV2DiskMonitor`| **EXPERIMENTAL** Reports disk usage from `io.stat` file. Only applicable to `cgroupv2`.|
|`org.apache.druid.java.util.metrics.CgroupV2MemoryMonitor`| **EXPERIMENTAL** Reports memory usage from `memory.current` and `memory.max` files. Only applicable to `cgroupv2`.|
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical services. Available only on Historical services.|
|`org.apache.druid.server.metrics.SegmentStatsMonitor` | **EXPERIMENTAL** Reports statistics about segments on Historical services. Available only on Historical services. Not to be used when lazy loading is configured.|
|`org.apache.druid.server.metrics.QueryCountStatsMonitor`|Reports how many queries have been successful/failed/interrupted.|

View File

@ -31,10 +31,11 @@ Apache Iceberg is an open table format for huge analytic datasets. [IcebergInput
Iceberg manages most of its metadata in metadata files in the object storage. However, it is still dependent on a metastore to manage a certain amount of metadata.
Iceberg refers to these metastores as catalogs. The Iceberg extension lets you connect to the following Iceberg catalog types:
* REST-based catalog
* Hive metastore catalog
* Local catalog
Druid does not support AWS Glue and REST based catalogs yet.
Druid does not support AWS Glue catalog yet.
For a given catalog, Iceberg input source reads the table name from the catalog, applies the filters, and extracts all the underlying live data files up to the latest snapshot.
The data files can be in Parquet, ORC, or Avro formats. The data files typically reside in a warehouse location, which can be in HDFS, S3, or the local filesystem.
@ -110,6 +111,11 @@ Since the Hadoop AWS connector uses the `s3a` filesystem client, specify the war
The local catalog type can be used for catalogs configured on the local filesystem. Set the `icebergCatalog` type to `local`. You can use this catalog for demos or localized tests. It is not recommended for production use cases.
The `warehouseSource` is set to `local` because this catalog only supports reading from a local filesystem.
## REST catalog
To connect to an Iceberg REST Catalog server, configure the `icebergCatalog` type as `rest`. The Iceberg REST Open API spec gives catalogs greater control over the implementation and in most cases, the `warehousePath` does not have to be provided by the client.
Security credentials may be provided in the `catalogProperties` object.
## Downloading Iceberg extension
To download `druid-iceberg-extensions`, run the following command after replacing `<VERSION>` with the desired

View File

@ -1065,7 +1065,7 @@ The following is a sample spec for a S3 warehouse source:
### Catalog Object
The catalog object supports `local` and `hive` catalog types.
The catalog object supports `rest`, `hive` and `local` catalog types.
The following table lists the properties of a `local` catalog:
@ -1086,9 +1086,18 @@ The following table lists the properties of a `hive` catalog:
|catalogProperties|Map of any additional properties that needs to be attached to the catalog.|None|no|
|caseSensitive|Toggle case sensitivity for column names during Iceberg table reads.|true|no|
The following table lists the properties of a `rest` catalog:
|Property|Description|Default|Required|
|--------|-----------|-------|---------|
|type|Set this value to `rest`.|None|yes|
|catalogUri|The URI associated with the catalog's HTTP endpoint.|None|yes|
|catalogProperties|Map of any additional properties that needs to be attached to the catalog.|None|no|
### Iceberg filter object
This input source provides the following filters: `and`, `equals`, `interval`, and `or`. You can use these filters to filter out data files from a snapshot, reducing the number of files Druid has to ingest.
It is strongly recommended to apply filtering only on Iceberg partition columns. When filtering on non-partition columns, Iceberg filters may return rows that do not fully match the expression. To address this, it may help to define an additional filter in the [`transformSpec`](./ingestion-spec.md#transformspec) to remove residual rows.
`equals` Filter:

View File

@ -32,9 +32,6 @@ Window functions are an [experimental](../development/experimental.md) feature.
Development and testing are still at early stage. Feel free to try window functions and provide your feedback.
Windows functions are not currently supported by multi-stage-query engine so you cannot use them in SQL-based ingestion.
Set the context parameter `enableWindowing: true` to use window functions.
:::
Window functions in Apache Druid produce values based upon the relationship of one row within a window of rows to the other rows within the same window. A window is a group of related rows within a result set. For example, rows with the same value for a specific dimension.

View File

@ -35,7 +35,7 @@
<modelVersion>4.0.0</modelVersion>
<properties>
<iceberg.core.version>1.4.1</iceberg.core.version>
<iceberg.core.version>1.6.1</iceberg.core.version>
<hive.version>3.1.3</hive.version>
</properties>
<dependencies>
@ -258,10 +258,6 @@
<groupId>io.airlift</groupId>
<artifactId>aircompressor</artifactId>
</exclusion>
<exclusion>
<groupId>org.apache.httpcomponents.client5</groupId>
<artifactId>httpclient5</artifactId>
</exclusion>
</exclusions>
</dependency>

View File

@ -28,6 +28,7 @@ import org.apache.druid.iceberg.guice.HiveConf;
import org.apache.druid.iceberg.input.HiveIcebergCatalog;
import org.apache.druid.iceberg.input.IcebergInputSource;
import org.apache.druid.iceberg.input.LocalCatalog;
import org.apache.druid.iceberg.input.RestIcebergCatalog;
import org.apache.druid.initialization.DruidModule;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FileSystem;
@ -45,6 +46,7 @@ public class IcebergDruidModule implements DruidModule
.registerSubtypes(
new NamedType(HiveIcebergCatalog.class, HiveIcebergCatalog.TYPE_KEY),
new NamedType(LocalCatalog.class, LocalCatalog.TYPE_KEY),
new NamedType(RestIcebergCatalog.class, RestIcebergCatalog.TYPE_KEY),
new NamedType(IcebergInputSource.class, IcebergInputSource.TYPE_KEY)
)

View File

@ -32,7 +32,7 @@ import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.utils.DynamicConfigProviderUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.iceberg.BaseMetastoreCatalog;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.hive.HiveCatalog;
import javax.annotation.Nullable;
@ -45,7 +45,6 @@ import java.util.Map;
*/
public class HiveIcebergCatalog extends IcebergCatalog
{
public static final String DRUID_DYNAMIC_CONFIG_PROVIDER_KEY = "druid.dynamic.config.provider";
public static final String TYPE_KEY = "hive";
@JsonProperty
@ -62,7 +61,7 @@ public class HiveIcebergCatalog extends IcebergCatalog
private final Configuration configuration;
private BaseMetastoreCatalog hiveCatalog;
private Catalog hiveCatalog;
private static final Logger log = new Logger(HiveIcebergCatalog.class);
@ -88,7 +87,7 @@ public class HiveIcebergCatalog extends IcebergCatalog
}
@Override
public BaseMetastoreCatalog retrieveCatalog()
public Catalog retrieveCatalog()
{
if (hiveCatalog == null) {
hiveCatalog = setupCatalog();

View File

@ -25,7 +25,6 @@ import org.apache.druid.iceberg.filter.IcebergFilter;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.iceberg.BaseMetastoreCatalog;
import org.apache.iceberg.FileScanTask;
import org.apache.iceberg.TableScan;
import org.apache.iceberg.catalog.Catalog;
@ -46,9 +45,10 @@ import java.util.List;
@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = InputFormat.TYPE_PROPERTY)
public abstract class IcebergCatalog
{
public static final String DRUID_DYNAMIC_CONFIG_PROVIDER_KEY = "druid.dynamic.config.provider";
private static final Logger log = new Logger(IcebergCatalog.class);
public abstract BaseMetastoreCatalog retrieveCatalog();
public abstract Catalog retrieveCatalog();
public boolean isCaseSensitive()
{

View File

@ -23,7 +23,7 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.BaseMetastoreCatalog;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.hadoop.HadoopCatalog;
import javax.annotation.Nullable;
@ -46,7 +46,7 @@ public class LocalCatalog extends IcebergCatalog
@JsonProperty
private final Boolean caseSensitive;
private BaseMetastoreCatalog catalog;
private Catalog catalog;
@JsonCreator
public LocalCatalog(
@ -83,7 +83,7 @@ public class LocalCatalog extends IcebergCatalog
}
@Override
public BaseMetastoreCatalog retrieveCatalog()
public Catalog retrieveCatalog()
{
if (catalog == null) {
catalog = setupCatalog();

View File

@ -0,0 +1,108 @@
/*
* 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.iceberg.input;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.guice.annotations.Json;
import org.apache.druid.iceberg.guice.HiveConf;
import org.apache.druid.utils.DynamicConfigProviderUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.catalog.Catalog;
import org.apache.iceberg.catalog.SessionCatalog;
import org.apache.iceberg.rest.HTTPClient;
import org.apache.iceberg.rest.RESTCatalog;
import javax.annotation.Nullable;
import java.util.Map;
/**
* Catalog implementation for Iceberg REST catalogs.
*/
public class RestIcebergCatalog extends IcebergCatalog
{
public static final String TYPE_KEY = "rest";
@JsonProperty
private final String catalogUri;
@JsonProperty
private final Map<String, String> catalogProperties;
private final Configuration configuration;
private Catalog restCatalog;
@JsonCreator
public RestIcebergCatalog(
@JsonProperty("catalogUri") String catalogUri,
@JsonProperty("catalogProperties") @Nullable
Map<String, Object> catalogProperties,
@JacksonInject @Json ObjectMapper mapper,
@JacksonInject @HiveConf Configuration configuration
)
{
if (catalogUri == null) {
throw InvalidInput.exception("catalogUri cannot be null");
}
this.catalogUri = catalogUri;
this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap(
catalogProperties,
DRUID_DYNAMIC_CONFIG_PROVIDER_KEY,
mapper
);
this.configuration = configuration;
}
@Override
public Catalog retrieveCatalog()
{
if (restCatalog == null) {
restCatalog = setupCatalog();
}
return restCatalog;
}
public String getCatalogUri()
{
return catalogUri;
}
public Map<String, String> getCatalogProperties()
{
return catalogProperties;
}
private RESTCatalog setupCatalog()
{
RESTCatalog restCatalog = new RESTCatalog(
SessionCatalog.SessionContext.createEmpty(),
config -> HTTPClient.builder(config).uri(config.get(CatalogProperties.URI)).build()
);
restCatalog.setConf(configuration);
catalogProperties.put(CatalogProperties.URI, catalogUri);
restCatalog.initialize("rest", catalogProperties);
return restCatalog;
}
}

View File

@ -0,0 +1,99 @@
/*
* 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.iceberg.input;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.net.HttpHeaders;
import com.sun.net.httpserver.HttpServer;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.hadoop.conf.Configuration;
import org.apache.iceberg.rest.RESTCatalog;
import org.junit.After;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Test;
import java.io.IOException;
import java.io.OutputStream;
import java.net.InetSocketAddress;
import java.net.ServerSocket;
import java.nio.charset.StandardCharsets;
import java.util.HashMap;
public class RestCatalogTest
{
private final ObjectMapper mapper = new DefaultObjectMapper();
private int port = 0;
private HttpServer server = null;
private ServerSocket serverSocket = null;
@Before
public void setup() throws Exception
{
serverSocket = new ServerSocket(0);
port = serverSocket.getLocalPort();
serverSocket.close();
server = HttpServer.create(new InetSocketAddress("localhost", port), 0);
server.createContext(
"/v1/config", // API for catalog fetchConfig which is invoked on catalog initialization
(httpExchange) -> {
String payload = "{}";
byte[] outputBytes = payload.getBytes(StandardCharsets.UTF_8);
httpExchange.sendResponseHeaders(200, outputBytes.length);
OutputStream os = httpExchange.getResponseBody();
httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_TYPE, "application/octet-stream");
httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_LENGTH, String.valueOf(outputBytes.length));
httpExchange.getResponseHeaders().set(HttpHeaders.CONTENT_RANGE, "bytes 0");
os.write(outputBytes);
os.close();
}
);
server.start();
}
@Test
public void testCatalogCreate()
{
String catalogUri = "http://localhost:" + port;
RestIcebergCatalog testRestCatalog = new RestIcebergCatalog(
catalogUri,
new HashMap<>(),
mapper,
new Configuration()
);
RESTCatalog innerCatalog = (RESTCatalog) testRestCatalog.retrieveCatalog();
Assert.assertEquals("rest", innerCatalog.name());
Assert.assertNotNull(innerCatalog.properties());
Assert.assertNotNull(testRestCatalog.getCatalogProperties());
Assert.assertEquals(testRestCatalog.getCatalogUri(), innerCatalog.properties().get("uri"));
}
@After
public void tearDown() throws IOException
{
if (server != null) {
server.stop(0);
}
if (serverSocket != null) {
serverSocket.close();
}
}
}

View File

@ -39,7 +39,6 @@ import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.indexing.overlord.supervisor.Supervisor;
import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
@ -55,6 +54,7 @@ import org.apache.druid.timeline.DataSegment;
import org.joda.time.Duration;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
@ -257,7 +257,7 @@ public class MaterializedViewSupervisor implements Supervisor
}
@Override
public void reset(DataSourceMetadata dataSourceMetadata)
public void reset(@Nullable DataSourceMetadata dataSourceMetadata)
{
if (dataSourceMetadata == null) {
// if oldMetadata is different from spec, tasks and segments will be removed when reset.
@ -280,30 +280,6 @@ public class MaterializedViewSupervisor implements Supervisor
}
}
@Override
public void resetOffsets(DataSourceMetadata resetDataSourceMetadata)
{
throw new UnsupportedOperationException("Reset offsets not supported in MaterializedViewSupervisor");
}
@Override
public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata)
{
// do nothing
}
@Override
public LagStats computeLagStats()
{
throw new UnsupportedOperationException("Compute Lag Stats not supported in MaterializedViewSupervisor");
}
@Override
public int getActiveTaskGroupsCount()
{
throw new UnsupportedOperationException("Get Active Task Groups Count is not supported in MaterializedViewSupervisor");
}
/**
* Find intervals in which derived dataSource should rebuild the segments.
* Choose the latest intervals to create new HadoopIndexTask and submit it.

View File

@ -52,7 +52,6 @@ import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.concurrent.Callable;
public class MaterializedViewSupervisorSpecTest
{
@ -202,19 +201,6 @@ public class MaterializedViewSupervisorSpecTest
SupervisorTaskAutoScaler autoscaler = spec.createAutoscaler(supervisor);
Assert.assertNull(autoscaler);
Assert.assertThrows(UnsupportedOperationException.class, () -> supervisor.computeLagStats());
Assert.assertThrows(UnsupportedOperationException.class, () -> supervisor.getActiveTaskGroupsCount());
Callable<Integer> noop = new Callable<Integer>() {
@Override
public Integer call()
{
return -1;
}
};
}
catch (Exception e) {
ex = e;

View File

@ -333,39 +333,6 @@ public class MaterializedViewSupervisorTest
supervisor.run();
}
@Test
public void testResetOffsetsNotSupported()
{
MaterializedViewSupervisorSpec suspended = new MaterializedViewSupervisorSpec(
"base",
new DimensionsSpec(Collections.singletonList(new StringDimensionSchema("dim"))),
new AggregatorFactory[]{new LongSumAggregatorFactory("m1", "m1")},
HadoopTuningConfig.makeDefaultTuningConfig(),
null,
null,
null,
null,
null,
true,
objectMapper,
taskMaster,
taskStorage,
metadataSupervisorManager,
sqlSegmentsMetadataManager,
indexerMetadataStorageCoordinator,
new MaterializedViewTaskConfig(),
EasyMock.createMock(AuthorizerMapper.class),
EasyMock.createMock(ChatHandlerProvider.class),
new SupervisorStateManagerConfig()
);
MaterializedViewSupervisor supervisor = (MaterializedViewSupervisor) suspended.createSupervisor();
Assert.assertThrows(
"Reset offsets not supported in MaterializedViewSupervisor",
UnsupportedOperationException.class,
() -> supervisor.resetOffsets(null)
);
}
private List<DataSegment> createBaseSegments()
{
return Arrays.asList(

View File

@ -80,7 +80,6 @@ import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig;
import org.apache.druid.sql.calcite.TempDirProducer;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.util.CacheTestHelperModule.ResultCacheMode;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier;
@ -1213,7 +1212,6 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
public void testHllWithOrderedWindowing()
{
testBuilder()
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))
.sql(
"SELECT dim1,coalesce(cast(l1 as integer),-999),"
+ " HLL_SKETCH_ESTIMATE( DS_HLL(dim1) OVER ( ORDER BY l1 ), true)"
@ -1238,7 +1236,6 @@ public class HllSketchSqlAggregatorTest extends BaseCalciteQueryTest
{
for (int i = 0; i < 2; i++) {
testBuilder()
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))
.sql(
"SELECT "
+ " TIME_FLOOR(__time, 'P1D') as dayLvl,\n"

View File

@ -86,7 +86,7 @@ public class CatalogClient implements CatalogSource
@Override
public List<TableMetadata> tablesForSchema(String dbSchema)
{
String url = StringUtils.replace(SCHEMA_SYNC_PATH, "{schema}", dbSchema);
String url = StringUtils.replace(SCHEMA_SYNC_PATH, "{schema}", StringUtils.urlEncode(dbSchema));
List<TableMetadata> results = send(url, LIST_OF_TABLE_METADATA_TYPE);
// Not found for a list is an empty list.
@ -96,8 +96,8 @@ public class CatalogClient implements CatalogSource
@Override
public TableMetadata table(TableId id)
{
String url = StringUtils.replace(TABLE_SYNC_PATH, "{schema}", id.schema());
url = StringUtils.replace(url, "{name}", id.name());
String url = StringUtils.replace(TABLE_SYNC_PATH, "{schema}", StringUtils.urlEncode(id.schema()));
url = StringUtils.replace(url, "{name}", StringUtils.urlEncode(id.name()));
return send(url, TABLE_METADATA_TYPE);
}

View File

@ -44,6 +44,7 @@
<groupId>org.apache.druid.extensions</groupId>
<artifactId>druid-lookups-cached-global</artifactId>
<version>${project.parent.version}</version>
<scope>provided</scope>
</dependency>
<dependency>
<groupId>org.apache.druid</groupId>

View File

@ -0,0 +1,3 @@
{
"dependsOnDruidExtensions": ["druid-lookups-cached-global"]
}

View File

@ -633,7 +633,8 @@ public class ControllerImpl implements Controller
this.workerSketchFetcher = new WorkerSketchFetcher(
netClient,
workerManager,
queryKernelConfig.isFaultTolerant()
queryKernelConfig.isFaultTolerant(),
MultiStageQueryContext.getSketchEncoding(querySpec.getQuery().context())
);
closer.register(workerSketchFetcher::close);

View File

@ -32,6 +32,7 @@ import org.apache.druid.msq.indexing.error.MSQException;
import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.msq.rpc.SketchEncoding;
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import javax.annotation.Nullable;
@ -60,23 +61,25 @@ public class ExceptionWrappingWorkerClient implements WorkerClient
@Override
public ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshot(
String workerTaskId,
StageId stageId
StageId stageId,
SketchEncoding sketchEncoding
)
{
return wrap(workerTaskId, client, c -> c.fetchClusterByStatisticsSnapshot(workerTaskId, stageId));
return wrap(workerTaskId, client, c -> c.fetchClusterByStatisticsSnapshot(workerTaskId, stageId, sketchEncoding));
}
@Override
public ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshotForTimeChunk(
String workerTaskId,
StageId stageId,
long timeChunk
long timeChunk,
SketchEncoding sketchEncoding
)
{
return wrap(
workerTaskId,
client,
c -> c.fetchClusterByStatisticsSnapshotForTimeChunk(workerTaskId, stageId, timeChunk)
c -> c.fetchClusterByStatisticsSnapshotForTimeChunk(workerTaskId, stageId, timeChunk, sketchEncoding)
);
}

View File

@ -25,6 +25,7 @@ import org.apache.druid.frame.key.ClusterByPartitions;
import org.apache.druid.msq.counters.CounterSnapshotsTree;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.msq.rpc.SketchEncoding;
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import java.io.Closeable;
@ -47,7 +48,8 @@ public interface WorkerClient extends Closeable
*/
ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshot(
String workerId,
StageId stageId
StageId stageId,
SketchEncoding sketchEncoding
);
/**
@ -57,7 +59,8 @@ public interface WorkerClient extends Closeable
ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshotForTimeChunk(
String workerId,
StageId stageId,
long timeChunk
long timeChunk,
SketchEncoding sketchEncoding
);
/**

View File

@ -34,6 +34,7 @@ import org.apache.druid.msq.indexing.error.MSQFault;
import org.apache.druid.msq.indexing.error.WorkerRpcFailedFault;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.controller.ControllerQueryKernel;
import org.apache.druid.msq.rpc.SketchEncoding;
import org.apache.druid.msq.statistics.ClusterByStatisticsCollector;
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation;
@ -57,6 +58,7 @@ public class WorkerSketchFetcher implements AutoCloseable
private static final int DEFAULT_THREAD_COUNT = 4;
private final WorkerClient workerClient;
private final SketchEncoding sketchEncoding;
private final WorkerManager workerManager;
private final boolean retryEnabled;
@ -68,10 +70,12 @@ public class WorkerSketchFetcher implements AutoCloseable
public WorkerSketchFetcher(
WorkerClient workerClient,
WorkerManager workerManager,
boolean retryEnabled
boolean retryEnabled,
SketchEncoding sketchEncoding
)
{
this.workerClient = workerClient;
this.sketchEncoding = sketchEncoding;
this.executorService = Execs.multiThreaded(DEFAULT_THREAD_COUNT, "SketchFetcherThreadPool-%d");
this.workerManager = workerManager;
this.retryEnabled = retryEnabled;
@ -96,7 +100,7 @@ public class WorkerSketchFetcher implements AutoCloseable
executorService.submit(() -> {
fetchStatsFromWorker(
kernelActions,
() -> workerClient.fetchClusterByStatisticsSnapshot(taskId, stageId),
() -> workerClient.fetchClusterByStatisticsSnapshot(taskId, stageId, sketchEncoding),
taskId,
(kernel, snapshot) ->
kernel.mergeClusterByStatisticsCollectorForAllTimeChunks(stageId, workerNumber, snapshot),
@ -252,7 +256,8 @@ public class WorkerSketchFetcher implements AutoCloseable
() -> workerClient.fetchClusterByStatisticsSnapshotForTimeChunk(
taskId,
new StageId(stageId.getQueryId(), stageId.getStageNumber()),
timeChunk
timeChunk,
sketchEncoding
),
taskId,
(kernel, snapshot) -> kernel.mergeClusterByStatisticsCollectorForTimeChunk(

View File

@ -34,6 +34,7 @@ import org.apache.druid.frame.read.FrameReader;
import org.apache.druid.frame.util.SettableLongVirtualColumn;
import org.apache.druid.frame.write.FrameWriter;
import org.apache.druid.frame.write.FrameWriterFactory;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.java.util.common.Unit;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.msq.indexing.error.MSQException;
@ -54,6 +55,7 @@ import org.apache.druid.segment.ColumnValueSelector;
import org.apache.druid.segment.Cursor;
import org.apache.druid.segment.VirtualColumn;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.NullableTypeStrategy;
import org.apache.druid.segment.column.RowSignature;
@ -136,6 +138,7 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
if (segmentGranularityVirtualColumn != null) {
frameWriterVirtualColumns.add(segmentGranularityVirtualColumn);
}
frameWriterVirtualColumns.add(this.partitionBoostVirtualColumn);
this.frameWriterVirtualColumns = VirtualColumns.create(frameWriterVirtualColumns);
}
@ -451,6 +454,14 @@ public class WindowOperatorQueryFrameProcessor implements FrameProcessor<Object>
int match = 0;
for (String columnName : partitionColumnNames) {
int i = frameReader.signature().indexOf(columnName);
if (ColumnType.STRING.equals(frameReader.signature().getColumnType(columnName).get()) && (row1.get(i) instanceof List || row2.get(i) instanceof List)) {
// special handling to reject MVDs
throw new UOE(
"Encountered a multi value column [%s]. Window processing does not support MVDs. "
+ "Consider using UNNEST or MV_TO_ARRAY.",
columnName
);
}
if (typeStrategies[i].compare(row1.get(i), row2.get(i)) == 0) {
match++;
}

View File

@ -91,14 +91,15 @@ public abstract class BaseWorkerClientImpl implements WorkerClient
@Override
public ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshot(
String workerId,
StageId stageId
StageId stageId,
SketchEncoding sketchEncoding
)
{
String path = StringUtils.format(
"/keyStatistics/%s/%d?sketchEncoding=%s",
StringUtils.urlEncode(stageId.getQueryId()),
stageId.getStageNumber(),
WorkerResource.SketchEncoding.OCTET_STREAM
sketchEncoding
);
return getClient(workerId).asyncRequest(
@ -111,7 +112,8 @@ public abstract class BaseWorkerClientImpl implements WorkerClient
public ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshotForTimeChunk(
String workerId,
StageId stageId,
long timeChunk
long timeChunk,
SketchEncoding sketchEncoding
)
{
String path = StringUtils.format(
@ -119,7 +121,7 @@ public abstract class BaseWorkerClientImpl implements WorkerClient
StringUtils.urlEncode(stageId.getQueryId()),
stageId.getStageNumber(),
timeChunk,
WorkerResource.SketchEncoding.OCTET_STREAM
sketchEncoding
);
return getClient(workerId).asyncRequest(

View File

@ -0,0 +1,39 @@
/*
* 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.rpc;
import org.apache.druid.msq.statistics.serde.ClusterByStatisticsSnapshotSerde;
/**
* Determines the encoding of key collectors returned by {@link WorkerResource#httpFetchKeyStatistics} and
* {@link WorkerResource#httpFetchKeyStatisticsWithSnapshot}.
*/
public enum SketchEncoding
{
/**
* The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}.
*/
OCTET_STREAM,
/**
* The key collector is encoded as json
*/
JSON
}

View File

@ -373,19 +373,4 @@ public class WorkerResource
return Response.status(Response.Status.OK).entity(worker.getCounters()).build();
}
/**
* Determines the encoding of key collectors returned by {@link #httpFetchKeyStatistics} and
* {@link #httpFetchKeyStatisticsWithSnapshot}.
*/
public enum SketchEncoding
{
/**
* The key collector is encoded as a byte stream with {@link ClusterByStatisticsSnapshotSerde}.
*/
OCTET_STREAM,
/**
* The key collector is encoded as json
*/
JSON
}
}

View File

@ -38,6 +38,7 @@ import org.apache.druid.msq.indexing.destination.MSQSelectDestination;
import org.apache.druid.msq.indexing.error.MSQWarnings;
import org.apache.druid.msq.kernel.WorkerAssignmentStrategy;
import org.apache.druid.msq.rpc.ControllerResource;
import org.apache.druid.msq.rpc.SketchEncoding;
import org.apache.druid.msq.sql.MSQMode;
import org.apache.druid.query.QueryContext;
import org.apache.druid.query.QueryContexts;
@ -137,6 +138,9 @@ public class MultiStageQueryContext
public static final String CTX_CLUSTER_STATISTICS_MERGE_MODE = "clusterStatisticsMergeMode";
public static final String DEFAULT_CLUSTER_STATISTICS_MERGE_MODE = ClusterStatisticsMergeMode.SEQUENTIAL.toString();
public static final String CTX_SKETCH_ENCODING_MODE = "sketchEncoding";
public static final String DEFAULT_CTX_SKETCH_ENCODING_MODE = SketchEncoding.OCTET_STREAM.toString();
public static final String CTX_ROWS_PER_SEGMENT = "rowsPerSegment";
public static final int DEFAULT_ROWS_PER_SEGMENT = 3000000;
@ -273,6 +277,15 @@ public class MultiStageQueryContext
);
}
public static SketchEncoding getSketchEncoding(QueryContext queryContext)
{
return QueryContexts.getAsEnum(
CTX_SKETCH_ENCODING_MODE,
queryContext.getString(CTX_SKETCH_ENCODING_MODE, DEFAULT_CTX_SKETCH_ENCODING_MODE),
SketchEncoding.class
);
}
public static boolean isFinalizeAggregations(final QueryContext queryContext)
{
return queryContext.getBoolean(

View File

@ -39,7 +39,6 @@ import org.apache.druid.sql.calcite.QueryTestBuilder;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig;
import org.apache.druid.sql.calcite.TempDirProducer;
import org.apache.druid.sql.calcite.planner.PlannerCaptureHook;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.run.SqlEngine;
import org.junit.jupiter.api.Test;
@ -50,7 +49,6 @@ import java.util.Map;
public class MSQDrillWindowQueryTest extends DrillWindowQueryTest
{
private final Map<String, Object> queryContext = new HashMap<>(ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
PlannerCaptureHook.NEED_CAPTURE_HOOK, true,
QueryContexts.ENABLE_DEBUG, true,
MultiStageQueryContext.CTX_MAX_NUM_TASKS, 5
@ -175,6 +173,105 @@ public class MSQDrillWindowQueryTest extends DrillWindowQueryTest
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_47")
@Test
public void test_ntile_func_ntileFn_47()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_49")
@Test
public void test_ntile_func_ntileFn_49()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_50")
@Test
public void test_ntile_func_ntileFn_50()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_51")
@Test
public void test_ntile_func_ntileFn_51()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_52")
@Test
public void test_ntile_func_ntileFn_52()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_53")
@Test
public void test_ntile_func_ntileFn_53()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_54")
@Test
public void test_ntile_func_ntileFn_54()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_55")
@Test
public void test_ntile_func_ntileFn_55()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_56")
@Test
public void test_ntile_func_ntileFn_56()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_57")
@Test
public void test_ntile_func_ntileFn_57()
{
useSingleWorker();
windowQueryTest();
}
@Override
@DrillTest("ntile_func/ntileFn_58")
@Test
public void test_ntile_func_ntileFn_58()
{
useSingleWorker();
windowQueryTest();
}
/*
Queries having window functions can give multiple correct results because of using MixShuffleSpec in the previous stage.
So we want to use a single worker to get the same result everytime for such test cases.

View File

@ -22,6 +22,8 @@ package org.apache.druid.msq.exec;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.calcite.avatica.ColumnMetaData;
import org.apache.calcite.avatica.remote.TypedValue;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
@ -704,6 +706,71 @@ public class MSQReplaceTest extends MSQTestBase
.verifyResults();
}
@MethodSource("data")
@ParameterizedTest(name = "{index}:with context {0}")
public void testReplaceWithDynamicParameters(String contextName, Map<String, Object> context)
{
RowSignature rowSignature = RowSignature.builder()
.add("__time", ColumnType.LONG)
.add("m1", ColumnType.FLOAT)
.build();
testIngestQuery().setSql(
" REPLACE INTO foo OVERWRITE WHERE __time >= ? AND __time < ? "
+ "SELECT __time, m1 "
+ "FROM foo "
+ "WHERE __time >= ? AND __time < ? "
+ "PARTITIONED by DAY ")
.setDynamicParameters(ImmutableList.of(
TypedValue.ofLocal(ColumnMetaData.Rep.JAVA_SQL_TIMESTAMP, DateTimes.of("2000-01-02").getMillis()),
TypedValue.ofLocal(ColumnMetaData.Rep.JAVA_SQL_TIMESTAMP, DateTimes.of("2000-01-03").getMillis()),
TypedValue.ofLocal(ColumnMetaData.Rep.JAVA_SQL_TIMESTAMP, DateTimes.of("2000-01-02").getMillis()),
TypedValue.ofLocal(ColumnMetaData.Rep.JAVA_SQL_TIMESTAMP, DateTimes.of("2000-01-03").getMillis())
))
.setExpectedDataSource("foo")
.setExpectedDestinationIntervals(ImmutableList.of(Intervals.of(
"2000-01-02T00:00:00.000Z/2000-01-03T00:00:00.000Z")))
.setExpectedRowSignature(rowSignature)
.setQueryContext(context)
.setExpectedSegments(ImmutableSet.of(SegmentId.of(
"foo",
Intervals.of("2000-01-02T/P1D"),
"test",
0
)))
.setExpectedResultRows(ImmutableList.of(new Object[]{946771200000L, 2.0f}))
.setExpectedCountersForStageWorkerChannel(
CounterSnapshotMatcher
.with().totalFiles(1),
0, 0, "input0"
)
.setExpectedCountersForStageWorkerChannel(
CounterSnapshotMatcher
.with().rows(1).frames(1),
0, 0, "shuffle"
)
.setExpectedCountersForStageWorkerChannel(
CounterSnapshotMatcher
.with().rows(1).frames(1),
1, 0, "input0"
)
.setExpectedSegmentGenerationProgressCountersForStageWorker(
CounterSnapshotMatcher
.with().segmentRowsProcessed(1),
1, 0
)
.setExpectedLastCompactionState(
expectedCompactionState(
context,
Collections.emptyList(),
Collections.singletonList(new FloatDimensionSchema("m1")),
GranularityType.DAY,
Intervals.of("2000-01-02T/P1D")
)
)
.verifyResults();
}
@MethodSource("data")
@ParameterizedTest(name = "{index}:with context {0}")
public void testReplaceOnFoo1WithAllExtern(String contextName, Map<String, Object> context) throws IOException

View File

@ -22,6 +22,8 @@ package org.apache.druid.msq.exec;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.calcite.avatica.ColumnMetaData;
import org.apache.calcite.avatica.remote.TypedValue;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.InlineInputSource;
@ -409,6 +411,53 @@ public class MSQSelectTest extends MSQTestBase
.verifyResults();
}
@MethodSource("data")
@ParameterizedTest(name = "{index}:with context {0}")
public void testSelectWithDynamicParameters(String contextName, Map<String, Object> context)
{
RowSignature resultSignature = RowSignature.builder()
.add("cnt", ColumnType.LONG)
.add("dim1", ColumnType.STRING)
.build();
// Filter [__time >= timestamp '3000-01-01 00:00:00'] matches no segments at all.
testSelectQuery()
.setSql("select cnt,dim1 from foo where __time >= ?")
.setExpectedMSQSpec(
MSQSpec.builder()
.query(
newScanQueryBuilder()
.dataSource(CalciteTests.DATASOURCE1)
.intervals(
querySegmentSpec(
Intervals.utc(
DateTimes.of("3000").getMillis(),
Intervals.ETERNITY.getEndMillis()
)
)
)
.columns("cnt", "dim1")
.context(defaultScanQueryContext(context, resultSignature))
.build()
)
.columnMappings(ColumnMappings.identity(resultSignature))
.tuningConfig(MSQTuningConfig.defaultConfig())
.destination(isDurableStorageDestination(contextName, context)
? DurableStorageMSQDestination.INSTANCE
: TaskReportMSQDestination.INSTANCE)
.build()
)
.setDynamicParameters(
ImmutableList.of(
TypedValue.ofLocal(ColumnMetaData.Rep.JAVA_SQL_TIMESTAMP, DateTimes.of("3000-01-01").getMillis())
)
)
.setQueryContext(context)
.setExpectedRowSignature(resultSignature)
.setExpectedResultRows(ImmutableList.of())
.verifyResults();
}
@MethodSource("data")
@ParameterizedTest(name = "{index}:with context {0}")
public void testSelectOnFooWhereMatchesNoData(String contextName, Map<String, Object> context)

View File

@ -27,6 +27,7 @@ import org.apache.druid.java.util.common.ISE;
import org.apache.druid.msq.kernel.StageDefinition;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.controller.ControllerQueryKernel;
import org.apache.druid.msq.rpc.SketchEncoding;
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import org.apache.druid.msq.statistics.CompleteKeyStatisticsInformation;
import org.junit.After;
@ -101,13 +102,13 @@ public class WorkerSketchFetcherTest
final CountDownLatch latch = new CountDownLatch(TASK_IDS.size());
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true, SketchEncoding.OCTET_STREAM));
// When fetching snapshots, return a mock and add it to queue
doAnswer(invocation -> {
ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class);
return Futures.immediateFuture(snapshot);
}).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any());
}).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), any());
target.inMemoryFullSketchMerging((kernelConsumer) -> {
kernelConsumer.accept(kernel);
@ -124,13 +125,13 @@ public class WorkerSketchFetcherTest
doReturn(true).when(completeKeyStatisticsInformation).isComplete();
final CountDownLatch latch = new CountDownLatch(TASK_IDS.size());
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true, SketchEncoding.OCTET_STREAM));
// When fetching snapshots, return a mock and add it to queue
doAnswer(invocation -> {
ClusterByStatisticsSnapshot snapshot = mock(ClusterByStatisticsSnapshot.class);
return Futures.immediateFuture(snapshot);
}).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyLong());
}).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyLong(), any());
target.sequentialTimeChunkMerging(
(kernelConsumer) -> {
@ -152,7 +153,7 @@ public class WorkerSketchFetcherTest
{
doReturn(false).when(completeKeyStatisticsInformation).isComplete();
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true, SketchEncoding.OCTET_STREAM));
Assert.assertThrows(ISE.class, () -> target.sequentialTimeChunkMerging(
(ignore) -> {},
completeKeyStatisticsInformation,
@ -167,7 +168,7 @@ public class WorkerSketchFetcherTest
{
final CountDownLatch latch = new CountDownLatch(TASK_IDS.size());
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true, SketchEncoding.OCTET_STREAM));
workersWithFailedFetchParallel(ImmutableSet.of(TASK_1));
@ -196,7 +197,7 @@ public class WorkerSketchFetcherTest
doReturn(true).when(completeKeyStatisticsInformation).isComplete();
final CountDownLatch latch = new CountDownLatch(TASK_IDS.size());
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, true, SketchEncoding.OCTET_STREAM));
workersWithFailedFetchSequential(ImmutableSet.of(TASK_1));
CountDownLatch retryLatch = new CountDownLatch(1);
@ -223,7 +224,7 @@ public class WorkerSketchFetcherTest
public void test_InMemoryRetryDisabled_multipleFailures() throws InterruptedException
{
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false, SketchEncoding.OCTET_STREAM));
workersWithFailedFetchParallel(ImmutableSet.of(TASK_1, TASK_0));
@ -252,7 +253,7 @@ public class WorkerSketchFetcherTest
public void test_InMemoryRetryDisabled_singleFailure() throws InterruptedException
{
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false, SketchEncoding.OCTET_STREAM));
workersWithFailedFetchParallel(ImmutableSet.of(TASK_1));
@ -283,7 +284,7 @@ public class WorkerSketchFetcherTest
{
doReturn(true).when(completeKeyStatisticsInformation).isComplete();
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false, SketchEncoding.OCTET_STREAM));
workersWithFailedFetchSequential(ImmutableSet.of(TASK_1, TASK_0));
@ -315,7 +316,7 @@ public class WorkerSketchFetcherTest
public void test_SequentialRetryDisabled_singleFailure() throws InterruptedException
{
doReturn(true).when(completeKeyStatisticsInformation).isComplete();
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false));
target = spy(new WorkerSketchFetcher(workerClient, workerManager, false, SketchEncoding.OCTET_STREAM));
workersWithFailedFetchSequential(ImmutableSet.of(TASK_1));
@ -352,7 +353,7 @@ public class WorkerSketchFetcherTest
return Futures.immediateFailedFuture(new Exception("Task fetch failed :" + invocation.getArgument(0)));
}
return Futures.immediateFuture(snapshot);
}).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyLong());
}).when(workerClient).fetchClusterByStatisticsSnapshotForTimeChunk(any(), any(), anyLong(), any());
}
private void workersWithFailedFetchParallel(Set<String> failedTasks)
@ -363,7 +364,7 @@ public class WorkerSketchFetcherTest
return Futures.immediateFailedFuture(new Exception("Task fetch failed :" + invocation.getArgument(0)));
}
return Futures.immediateFuture(snapshot);
}).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any());
}).when(workerClient).fetchClusterByStatisticsSnapshot(any(), any(), any());
}
}

View File

@ -36,6 +36,7 @@ import com.google.inject.Module;
import com.google.inject.TypeLiteral;
import com.google.inject.util.Modules;
import com.google.inject.util.Providers;
import org.apache.calcite.avatica.remote.TypedValue;
import org.apache.druid.client.ImmutableSegmentLoadInfo;
import org.apache.druid.collections.ReferenceCountingResourceHolder;
import org.apache.druid.collections.ResourceHolder;
@ -175,7 +176,6 @@ import org.apache.druid.sql.calcite.external.LocalOperatorConversion;
import org.apache.druid.sql.calcite.planner.CalciteRulesManager;
import org.apache.druid.sql.calcite.planner.CatalogResolver;
import org.apache.druid.sql.calcite.planner.PlannerConfig;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.planner.PlannerFactory;
import org.apache.druid.sql.calcite.rel.DruidQuery;
import org.apache.druid.sql.calcite.run.SqlEngine;
@ -218,7 +218,6 @@ import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.HashSet;
import java.util.List;
@ -272,7 +271,6 @@ public class MSQTestBase extends BaseCalciteQueryTest
.put(MultiStageQueryContext.CTX_MAX_NUM_TASKS, 2)
.put(MSQWarnings.CTX_MAX_PARSE_EXCEPTIONS_ALLOWED, 0)
.put(MSQTaskQueryMaker.USER_KEY, "allowAll")
.put(PlannerContext.CTX_ENABLE_WINDOW_FNS, true)
.build();
public static final Map<String, Object> DURABLE_STORAGE_MSQ_CONTEXT =
@ -786,13 +784,13 @@ public class MSQTestBase extends BaseCalciteQueryTest
);
}
private String runMultiStageQuery(String query, Map<String, Object> context)
private String runMultiStageQuery(String query, Map<String, Object> context, List<TypedValue> parameters)
{
final DirectStatement stmt = sqlStatementFactory.directStatement(
new SqlQueryPlus(
query,
context,
Collections.emptyList(),
parameters,
CalciteTests.REGULAR_USER_AUTH_RESULT
)
);
@ -888,6 +886,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
protected String sql = null;
protected MSQControllerTask taskSpec = null;
protected Map<String, Object> queryContext = DEFAULT_MSQ_CONTEXT;
protected List<TypedValue> dynamicParameters = new ArrayList<>();
protected List<MSQResultsReport.ColumnAndType> expectedRowSignature = null;
protected MSQSpec expectedMSQSpec = null;
protected MSQTuningConfig expectedTuningConfig = null;
@ -926,6 +925,12 @@ public class MSQTestBase extends BaseCalciteQueryTest
return asBuilder();
}
public Builder setDynamicParameters(List<TypedValue> dynamicParameters)
{
this.dynamicParameters = dynamicParameters;
return asBuilder();
}
public Builder setExpectedRowSignature(List<MSQResultsReport.ColumnAndType> expectedRowSignature)
{
Preconditions.checkArgument(!expectedRowSignature.isEmpty(), "Row signature cannot be empty");
@ -1059,7 +1064,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
final Throwable e = Assert.assertThrows(
Throwable.class,
() -> runMultiStageQuery(sql, queryContext)
() -> runMultiStageQuery(sql, queryContext, dynamicParameters)
);
assertThat(e, expectedValidationErrorMatcher);
@ -1211,7 +1216,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
String controllerId;
if (sql != null) {
// Run the sql command.
controllerId = runMultiStageQuery(sql, queryContext);
controllerId = runMultiStageQuery(sql, queryContext, dynamicParameters);
} else {
// Run the task spec directly instead.
controllerId = TEST_CONTROLLER_TASK_ID;
@ -1428,7 +1433,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
try {
String controllerId;
if (sql != null) {
controllerId = runMultiStageQuery(sql, queryContext);
controllerId = runMultiStageQuery(sql, queryContext, dynamicParameters);
} else {
// Run the task spec directly instead.
controllerId = TEST_CONTROLLER_TASK_ID;
@ -1470,7 +1475,7 @@ public class MSQTestBase extends BaseCalciteQueryTest
Preconditions.checkArgument(sql == null || queryContext != null, "queryContext cannot be null");
try {
String controllerId = runMultiStageQuery(sql, queryContext);
String controllerId = runMultiStageQuery(sql, queryContext, dynamicParameters);
if (expectedMSQFault != null || expectedMSQFaultClass != null) {
MSQErrorReport msqErrorReport = getErrorReportOrThrow(controllerId);

View File

@ -29,6 +29,7 @@ import org.apache.druid.msq.exec.Worker;
import org.apache.druid.msq.exec.WorkerClient;
import org.apache.druid.msq.kernel.StageId;
import org.apache.druid.msq.kernel.WorkOrder;
import org.apache.druid.msq.rpc.SketchEncoding;
import org.apache.druid.msq.statistics.ClusterByStatisticsSnapshot;
import java.io.InputStream;
@ -54,7 +55,8 @@ public class MSQTestWorkerClient implements WorkerClient
@Override
public ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshot(
String workerTaskId,
StageId stageId
StageId stageId,
SketchEncoding sketchEncoding
)
{
return Futures.immediateFuture(inMemoryWorkers.get(workerTaskId).fetchStatisticsSnapshot(stageId));
@ -64,7 +66,8 @@ public class MSQTestWorkerClient implements WorkerClient
public ListenableFuture<ClusterByStatisticsSnapshot> fetchClusterByStatisticsSnapshotForTimeChunk(
String workerTaskId,
StageId stageId,
long timeChunk
long timeChunk,
SketchEncoding sketchEncoding
)
{
return Futures.immediateFuture(

View File

@ -20,7 +20,6 @@
package org.apache.druid.query.aggregation.variance.sql;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.inject.Injector;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.InputRow;
@ -34,7 +33,6 @@ import org.apache.druid.guice.DruidInjectorBuilder;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.query.Druids;
import org.apache.druid.query.QueryContexts;
import org.apache.druid.query.QueryRunnerFactoryConglomerate;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
@ -64,7 +62,6 @@ import org.apache.druid.sql.calcite.BaseCalciteQueryTest;
import org.apache.druid.sql.calcite.SqlTestFrameworkConfig;
import org.apache.druid.sql.calcite.TempDirProducer;
import org.apache.druid.sql.calcite.filtration.Filtration;
import org.apache.druid.sql.calcite.planner.PlannerContext;
import org.apache.druid.sql.calcite.run.EngineFeature;
import org.apache.druid.sql.calcite.util.CalciteTests;
import org.apache.druid.sql.calcite.util.SqlTestFramework.StandardComponentSupplier;
@ -715,10 +712,6 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest
"select dim4, dim5, mod(m1, 3), var_pop(mod(m1, 3)) over (partition by dim4 order by dim5) c\n"
+ "from numfoo\n"
+ "group by dim4, dim5, mod(m1, 3)")
.queryContext(ImmutableMap.of(
PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
QueryContexts.ENABLE_DEBUG, true
))
.expectedResults(ImmutableList.of(
new Object[]{"a", "aa", 1.0D, 0.0D},
new Object[]{"a", "ab", 2.0D, 0.25D},
@ -738,7 +731,6 @@ public class VarianceSqlAggregatorTest extends BaseCalciteQueryTest
DruidException e = assertThrows(
DruidException.class,
() -> testBuilder()
.queryContext(ImmutableMap.of(PlannerContext.CTX_ENABLE_WINDOW_FNS, true))
.sql("SELECT stddev(m1) OVER () from numfoo")
.run()
);

View File

@ -23,11 +23,13 @@ import org.apache.druid.indexing.overlord.DataSourceMetadata;
import org.apache.druid.indexing.overlord.supervisor.Supervisor;
import org.apache.druid.indexing.overlord.supervisor.SupervisorReport;
import org.apache.druid.indexing.overlord.supervisor.SupervisorStateManager;
import org.apache.druid.indexing.overlord.supervisor.autoscaler.LagStats;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.server.coordinator.AutoCompactionSnapshot;
import javax.annotation.Nullable;
/**
* Supervisor for compaction of a single datasource.
*/
@ -55,6 +57,13 @@ public class CompactionSupervisor implements Supervisor
if (supervisorSpec.isSuspended()) {
log.info("Suspending compaction for dataSource[%s].", dataSource);
scheduler.stopCompaction(dataSource);
} else if (!supervisorSpec.getValidationResult().isValid()) {
log.warn(
"Cannot start compaction supervisor for datasource[%s] since the compaction supervisor spec is invalid. "
+ "Reason[%s].",
dataSource,
supervisorSpec.getValidationResult().getReason()
);
} else {
log.info("Starting compaction for dataSource[%s].", dataSource);
scheduler.startCompaction(dataSource, supervisorSpec.getSpec());
@ -76,6 +85,13 @@ public class CompactionSupervisor implements Supervisor
snapshot = AutoCompactionSnapshot.builder(dataSource)
.withStatus(AutoCompactionSnapshot.AutoCompactionScheduleStatus.NOT_ENABLED)
.build();
} else if (!supervisorSpec.getValidationResult().isValid()) {
snapshot = AutoCompactionSnapshot.builder(dataSource)
.withMessage(StringUtils.format(
"Compaction supervisor spec is invalid. Reason[%s].",
supervisorSpec.getValidationResult().getReason()
))
.build();
} else {
snapshot = scheduler.getCompactionSnapshot(dataSource);
}
@ -90,41 +106,17 @@ public class CompactionSupervisor implements Supervisor
return State.SCHEDULER_STOPPED;
} else if (supervisorSpec.isSuspended()) {
return State.SUSPENDED;
} else if (!supervisorSpec.getValidationResult().isValid()) {
return State.INVALID_SPEC;
} else {
return State.RUNNING;
}
}
// Un-implemented methods used only by streaming supervisors
@Override
public void reset(DataSourceMetadata dataSourceMetadata)
public void reset(@Nullable DataSourceMetadata dataSourceMetadata)
{
throw new UnsupportedOperationException("Resetting not supported for 'autocompact' supervisors.");
}
@Override
public void resetOffsets(DataSourceMetadata resetDataSourceMetadata)
{
throw new UnsupportedOperationException("Resetting offsets not supported for 'autocompact' supervisors.");
}
@Override
public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata)
{
throw new UnsupportedOperationException("Checkpointing not supported for 'autocompact' supervisors.");
}
@Override
public LagStats computeLagStats()
{
throw new UnsupportedOperationException("Lag stats not supported for 'autocompact' supervisors.");
}
@Override
public int getActiveTaskGroupsCount()
{
throw new UnsupportedOperationException("Task groups not supported for 'autocompact' supervisors.");
// do nothing
}
public enum State implements SupervisorStateManager.State
@ -132,6 +124,7 @@ public class CompactionSupervisor implements Supervisor
SCHEDULER_STOPPED(true),
RUNNING(true),
SUSPENDED(true),
INVALID_SPEC(false),
UNHEALTHY(false);
private final boolean healthy;

View File

@ -23,7 +23,6 @@ import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.common.config.Configs;
import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexing.overlord.supervisor.SupervisorSpec;
import org.apache.druid.server.coordinator.CompactionConfigValidationResult;
import org.apache.druid.server.coordinator.DataSourceCompactionConfig;
@ -40,6 +39,7 @@ public class CompactionSupervisorSpec implements SupervisorSpec
private final boolean suspended;
private final DataSourceCompactionConfig spec;
private final CompactionScheduler scheduler;
private final CompactionConfigValidationResult validationResult;
@JsonCreator
public CompactionSupervisorSpec(
@ -48,14 +48,10 @@ public class CompactionSupervisorSpec implements SupervisorSpec
@JacksonInject CompactionScheduler scheduler
)
{
final CompactionConfigValidationResult validationResult = scheduler.validateCompactionConfig(spec);
if (!validationResult.isValid()) {
throw InvalidInput.exception("Compaction supervisor 'spec' is invalid. Reason[%s].", validationResult.getReason());
}
this.spec = spec;
this.suspended = Configs.valueOrDefault(suspended, false);
this.scheduler = scheduler;
this.validationResult = scheduler.validateCompactionConfig(spec);
}
@JsonProperty
@ -77,6 +73,11 @@ public class CompactionSupervisorSpec implements SupervisorSpec
return ID_PREFIX + spec.getDataSource();
}
public CompactionConfigValidationResult getValidationResult()
{
return validationResult;
}
@Override
public CompactionSupervisor createSupervisor()
{

View File

@ -202,7 +202,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler
} else {
return ClientCompactionRunnerInfo.validateCompactionConfig(
compactionConfig,
compactionConfigSupplier.get().getEngine()
supervisorConfig.getEngine()
);
}
}
@ -272,7 +272,7 @@ public class OverlordCompactionScheduler implements CompactionScheduler
private synchronized void runCompactionDuty()
{
final CoordinatorRunStats stats = new CoordinatorRunStats();
duty.run(getLatestConfig(), getCurrentDatasourceTimelines(), stats);
duty.run(getLatestConfig(), getCurrentDatasourceTimelines(), supervisorConfig.getEngine(), stats);
// Emit stats only if emission period has elapsed
if (!sinceStatsEmitted.isRunning() || sinceStatsEmitted.hasElapsed(METRIC_EMISSION_PERIOD)) {
@ -309,7 +309,8 @@ public class OverlordCompactionScheduler implements CompactionScheduler
if (isRunning()) {
return new CompactionRunSimulator(statusTracker, overlordClient).simulateRunWithConfig(
getLatestConfig().withClusterConfig(updateRequest),
getCurrentDatasourceTimelines()
getCurrentDatasourceTimelines(),
supervisorConfig.getEngine()
);
} else {
return new CompactionSimulateResult(Collections.emptyMap());

View File

@ -22,6 +22,7 @@ package org.apache.druid.indexing.overlord.supervisor;
import com.google.common.base.Optional;
import com.google.common.base.Preconditions;
import com.google.inject.Inject;
import org.apache.druid.error.DruidException;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
@ -142,7 +143,8 @@ public class SupervisorManager
if (supervisor == null || supervisor.lhs == null) {
return false;
}
supervisor.lhs.handoffTaskGroupsEarly(taskGroupIds);
final SeekableStreamSupervisor streamSupervisor = requireSeekableStreamSupervisor(id, "handoff");
streamSupervisor.handoffTaskGroupsEarly(taskGroupIds);
return true;
}
@ -277,10 +279,11 @@ public class SupervisorManager
return false;
}
final SeekableStreamSupervisor streamSupervisor = requireSeekableStreamSupervisor(id, "reset");
if (resetDataSourceMetadata == null) {
supervisor.lhs.reset(null);
streamSupervisor.reset(null);
} else {
supervisor.lhs.resetOffsets(resetDataSourceMetadata);
streamSupervisor.resetOffsets(resetDataSourceMetadata);
}
SupervisorTaskAutoScaler autoscaler = autoscalers.get(id);
if (autoscaler != null) {
@ -303,7 +306,8 @@ public class SupervisorManager
Preconditions.checkNotNull(supervisor, "supervisor could not be found");
supervisor.lhs.checkpoint(taskGroupId, previousDataSourceMetadata);
final SeekableStreamSupervisor streamSupervisor = requireSeekableStreamSupervisor(supervisorId, "checkPoint");
streamSupervisor.checkpoint(taskGroupId, previousDataSourceMetadata);
return true;
}
catch (Exception e) {
@ -445,4 +449,21 @@ public class SupervisorManager
return true;
}
private SeekableStreamSupervisor requireSeekableStreamSupervisor(final String supervisorId, final String operation)
{
Pair<Supervisor, SupervisorSpec> supervisor = supervisors.get(supervisorId);
if (supervisor.lhs instanceof SeekableStreamSupervisor) {
return (SeekableStreamSupervisor) supervisor.lhs;
} else {
throw DruidException.forPersona(DruidException.Persona.USER)
.ofCategory(DruidException.Category.UNSUPPORTED)
.build(
"Operation[%s] is not supported by supervisor[%s] of type[%s].",
operation,
supervisorId,
supervisor.rhs.getType()
);
}
}
}

View File

@ -50,6 +50,7 @@ import org.apache.druid.server.security.ForbiddenException;
import org.apache.druid.server.security.Resource;
import org.apache.druid.server.security.ResourceAction;
import org.apache.druid.server.security.ResourceType;
import org.apache.druid.utils.CollectionUtils;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
@ -123,7 +124,7 @@ public class SupervisorResource
return asLeaderWithSupervisorManager(
manager -> {
Preconditions.checkArgument(
spec.getDataSources() != null && spec.getDataSources().size() > 0,
!CollectionUtils.isNullOrEmpty(spec.getDataSources()),
"No dataSources found to perform authorization checks"
);
final Set<ResourceAction> resourceActions;
@ -412,7 +413,7 @@ public class SupervisorResource
public Response handoffTaskGroups(@PathParam("id") final String id, @Nonnull final HandoffTaskGroupsRequest handoffTaskGroupsRequest)
{
List<Integer> taskGroupIds = handoffTaskGroupsRequest.getTaskGroupIds();
if (taskGroupIds == null || taskGroupIds.isEmpty()) {
if (CollectionUtils.isNullOrEmpty(taskGroupIds)) {
return Response.status(Response.Status.BAD_REQUEST)
.entity(ImmutableMap.of("error", "List of task groups to handoff can't be empty"))
.build();
@ -533,7 +534,7 @@ public class SupervisorResource
authorizerMapper
)
);
if (authorizedHistoryForId.size() > 0) {
if (!authorizedHistoryForId.isEmpty()) {
return Response.ok(authorizedHistoryForId).build();
}
}

View File

@ -627,6 +627,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
stillReading = !assignment.isEmpty();
SequenceMetadata<PartitionIdType, SequenceOffsetType> sequenceToCheckpoint = null;
AppenderatorDriverAddResult pushTriggeringAddResult = null;
for (OrderedPartitionableRecord<PartitionIdType, SequenceOffsetType, RecordType> record : records) {
final boolean shouldProcess = verifyRecordInRange(record.getPartitionId(), record.getSequenceNumber());
@ -677,6 +678,7 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
.getMaxTotalRowsOr(DynamicPartitionsSpec.DEFAULT_MAX_TOTAL_ROWS)
);
if (isPushRequired && !sequenceToUse.isCheckpointed()) {
pushTriggeringAddResult = addResult;
sequenceToCheckpoint = sequenceToUse;
}
isPersistRequired |= addResult.isPersistRequired();
@ -739,6 +741,10 @@ public abstract class SeekableStreamIndexTaskRunner<PartitionIdType, SequenceOff
if (System.currentTimeMillis() > nextCheckpointTime) {
sequenceToCheckpoint = getLastSequenceMetadata();
log.info("Next checkpoint time, updating sequenceToCheckpoint, SequenceToCheckpoint: [%s]", sequenceToCheckpoint);
}
if (pushTriggeringAddResult != null) {
log.info("Hit the row limit updating sequenceToCheckpoint, SequenceToCheckpoint: [%s], rowInSegment: [%s], TotalRows: [%s]", sequenceToCheckpoint, pushTriggeringAddResult.getNumRowsInSegment(), pushTriggeringAddResult.getTotalNumRowsInAppenderator());
}
if (sequenceToCheckpoint != null && stillReading) {

View File

@ -975,7 +975,6 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
this.taskClient = taskClientFactory.build(dataSource, taskInfoProvider, maxNumTasks, this.tuningConfig);
}
@Override
public int getActiveTaskGroupsCount()
{
return activelyReadingTaskGroups.values().size();
@ -1107,7 +1106,6 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
* @param resetDataSourceMetadata required datasource metadata with offsets to reset.
* @throws DruidException if any metadata attribute doesn't match the supervisor's.
*/
@Override
public void resetOffsets(@Nonnull DataSourceMetadata resetDataSourceMetadata)
{
if (resetDataSourceMetadata == null) {
@ -1975,7 +1973,11 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
return false;
}
@Override
/**
* Marks the given task groups as ready for segment hand-off irrespective of the task run times.
* In the subsequent run, the supervisor initiates segment publish and hand-off for these task groups and rolls over their tasks.
* taskGroupIds that are not valid or not actively reading are simply ignored.
*/
public void handoffTaskGroupsEarly(List<Integer> taskGroupIds)
{
addNotice(new HandoffTaskGroupsNotice(taskGroupIds));
@ -4170,7 +4172,15 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
return ioConfig;
}
@Override
/**
* The definition of checkpoint is not very strict as currently it does not affect data or control path.
* On this call Supervisor can potentially checkpoint data processed so far to some durable storage
* for example - Kafka/Kinesis Supervisor uses this to merge and handoff segments containing at least the data
* represented by {@param currentCheckpoint} DataSourceMetadata
*
* @param taskGroupId unique Identifier to figure out for which sequence to do checkpointing
* @param checkpointMetadata metadata for the sequence to currently checkpoint
*/
public void checkpoint(int taskGroupId, DataSourceMetadata checkpointMetadata)
{
Preconditions.checkNotNull(checkpointMetadata, "checkpointMetadata");
@ -4248,6 +4258,10 @@ public abstract class SeekableStreamSupervisor<PartitionIdType, SequenceOffsetTy
return activeTaskMap.build();
}
/**
* Computes maxLag, totalLag and avgLag
*/
public abstract LagStats computeLagStats();
/**
* creates a specific task IOConfig instance for Kafka/Kinesis

View File

@ -30,7 +30,6 @@ import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.indexing.overlord.config.TaskLockConfig;
import org.apache.druid.indexing.overlord.supervisor.SupervisorManager;
import org.apache.druid.java.util.common.concurrent.ScheduledExecutors;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import org.apache.druid.metadata.MetadataStorageConnectorConfig;
import org.apache.druid.metadata.MetadataStorageTablesConfig;
@ -104,16 +103,16 @@ public class TaskActionTestKit extends ExternalResource
}
};
taskLockbox = new TaskLockbox(taskStorage, metadataStorageCoordinator);
segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter());
segmentSchemaCache = new SegmentSchemaCache(NoopServiceEmitter.instance());
segmentsMetadataManager = new SqlSegmentsMetadataManager(
objectMapper,
Suppliers.ofInstance(new SegmentsMetadataManagerConfig()),
Suppliers.ofInstance(metadataStorageTablesConfig),
testDerbyConnector,
segmentSchemaCache,
CentralizedDatasourceSchemaConfig.create()
CentralizedDatasourceSchemaConfig.create(),
NoopServiceEmitter.instance()
);
final ServiceEmitter noopEmitter = new NoopServiceEmitter();
final TaskLockConfig taskLockConfig = new TaskLockConfig()
{
@Override
@ -137,10 +136,10 @@ public class TaskActionTestKit extends ExternalResource
taskLockbox,
taskLockConfig,
metadataStorageCoordinator,
noopEmitter,
NoopServiceEmitter.instance(),
ScheduledExecutors::fixed
),
noopEmitter,
NoopServiceEmitter.instance(),
EasyMock.createMock(SupervisorManager.class),
objectMapper
);

View File

@ -155,14 +155,15 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
segmentSchemaManager,
CentralizedDatasourceSchemaConfig.create()
);
segmentSchemaCache = new SegmentSchemaCache(new NoopServiceEmitter());
segmentSchemaCache = new SegmentSchemaCache(NoopServiceEmitter.instance());
segmentsMetadataManager = new SqlSegmentsMetadataManager(
objectMapper,
SegmentsMetadataManagerConfig::new,
derbyConnectorRule.metadataTablesConfigSupplier(),
derbyConnectorRule.getConnector(),
segmentSchemaCache,
CentralizedDatasourceSchemaConfig.create()
CentralizedDatasourceSchemaConfig.create(),
NoopServiceEmitter.instance()
);
lockbox = new TaskLockbox(taskStorage, storageCoordinator);
segmentCacheManagerFactory = new SegmentCacheManagerFactory(TestIndex.INDEX_IO, getObjectMapper());

View File

@ -82,20 +82,29 @@ public class CompactionSupervisorSpecTest
}
@Test
public void testInvalidSpecThrowsException()
public void testGetStatusWithInvalidSpec()
{
Mockito.when(scheduler.validateCompactionConfig(ArgumentMatchers.any()))
.thenReturn(CompactionConfigValidationResult.failure("bad spec"));
final DruidException exception = Assert.assertThrows(
DruidException.class,
() -> new CompactionSupervisorSpec(null, false, scheduler)
);
Assert.assertEquals(
"Compaction supervisor 'spec' is invalid. Reason[bad spec].",
exception.getMessage()
"Compaction supervisor spec is invalid. Reason[bad spec].", new CompactionSupervisorSpec(
new DataSourceCompactionConfig.Builder().forDataSource("datasource").build(),
false,
scheduler
).createSupervisor().getStatus().getPayload().getMessage()
);
}
@Test
public void testGetValidationResultForInvalidSpec()
{
Mockito.when(scheduler.validateCompactionConfig(ArgumentMatchers.any()))
.thenReturn(CompactionConfigValidationResult.failure("bad spec"));
CompactionConfigValidationResult validationResult = new CompactionSupervisorSpec(null, false, scheduler).getValidationResult();
Assert.assertFalse(validationResult.isValid());
Assert.assertEquals("bad spec", validationResult.getReason());
}
@Test
public void testGetIdAndDataSources()
{

View File

@ -120,7 +120,7 @@ public class OverlordCompactionSchedulerTest
serviceEmitter = new StubServiceEmitter();
segmentsMetadataManager = new TestSegmentsMetadataManager();
supervisorConfig = new CompactionSupervisorConfig(true);
supervisorConfig = new CompactionSupervisorConfig(true, null);
compactionConfig = DruidCompactionConfig.empty();
coordinatorOverlordServiceConfig = new CoordinatorOverlordServiceConfig(false, null);
@ -149,7 +149,7 @@ public class OverlordCompactionSchedulerTest
@Test
public void testStartStopWhenSchedulerIsEnabled()
{
supervisorConfig = new CompactionSupervisorConfig(true);
supervisorConfig = new CompactionSupervisorConfig(true, null);
Assert.assertFalse(scheduler.isRunning());
scheduler.start();
@ -168,7 +168,7 @@ public class OverlordCompactionSchedulerTest
@Test
public void testStartStopWhenScheduledIsDisabled()
{
supervisorConfig = new CompactionSupervisorConfig(false);
supervisorConfig = new CompactionSupervisorConfig(false, null);
initScheduler();
Assert.assertFalse(scheduler.isRunning());
@ -183,7 +183,7 @@ public class OverlordCompactionSchedulerTest
@Test
public void testSegmentsAreNotPolledWhenSchedulerIsDisabled()
{
supervisorConfig = new CompactionSupervisorConfig(false);
supervisorConfig = new CompactionSupervisorConfig(false, null);
initScheduler();
verifySegmentPolling(false);
@ -337,7 +337,7 @@ public class OverlordCompactionSchedulerTest
);
final CompactionSimulateResult simulateResult = scheduler.simulateRunWithConfigUpdate(
new ClusterCompactionConfig(null, null, null, null, null)
new ClusterCompactionConfig(null, null, null, null)
);
Assert.assertEquals(1, simulateResult.getCompactionStates().size());
final Table pendingCompactionTable = simulateResult.getCompactionStates().get(CompactionStatus.State.PENDING);
@ -362,7 +362,7 @@ public class OverlordCompactionSchedulerTest
scheduler.stopCompaction(TestDataSource.WIKI);
final CompactionSimulateResult simulateResultWhenDisabled = scheduler.simulateRunWithConfigUpdate(
new ClusterCompactionConfig(null, null, null, null, null)
new ClusterCompactionConfig(null, null, null, null)
);
Assert.assertTrue(simulateResultWhenDisabled.getCompactionStates().isEmpty());

View File

@ -44,9 +44,9 @@ import java.util.Map;
public class OverlordCompactionResourceTest
{
private static final CompactionSupervisorConfig SUPERVISOR_ENABLED
= new CompactionSupervisorConfig(true);
= new CompactionSupervisorConfig(true, null);
private static final CompactionSupervisorConfig SUPERVISOR_DISABLED
= new CompactionSupervisorConfig(false);
= new CompactionSupervisorConfig(false, null);
private CompactionScheduler scheduler;

View File

@ -23,6 +23,8 @@ import com.google.common.base.Optional;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.error.DruidException;
import org.apache.druid.error.DruidExceptionMatcher;
import org.apache.druid.indexing.common.TaskLockType;
import org.apache.druid.indexing.common.task.Tasks;
import org.apache.druid.indexing.overlord.DataSourceMetadata;
@ -41,6 +43,7 @@ import org.easymock.EasyMock;
import org.easymock.EasyMockRunner;
import org.easymock.EasyMockSupport;
import org.easymock.Mock;
import org.hamcrest.MatcherAssert;
import org.junit.Assert;
import org.junit.Before;
import org.junit.Rule;
@ -60,10 +63,10 @@ public class SupervisorManagerTest extends EasyMockSupport
private MetadataSupervisorManager metadataSupervisorManager;
@Mock
private Supervisor supervisor1;
private SeekableStreamSupervisor supervisor1;
@Mock
private Supervisor supervisor2;
private SeekableStreamSupervisor supervisor2;
@Mock
private Supervisor supervisor3;
@ -261,6 +264,33 @@ public class SupervisorManagerTest extends EasyMockSupport
verifyAll();
}
@Test
public void testHandoffTaskGroupsEarlyOnNonStreamSupervisor()
{
final Map<String, SupervisorSpec> existingSpecs = ImmutableMap.of(
"id3", new TestSupervisorSpec("id3", supervisor3)
);
EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs);
supervisor3.start();
replayAll();
manager.start();
MatcherAssert.assertThat(
Assert.assertThrows(DruidException.class, () -> manager.handoffTaskGroupsEarly("id3", ImmutableList.of(1))),
new DruidExceptionMatcher(
DruidException.Persona.USER,
DruidException.Category.UNSUPPORTED,
"general"
).expectMessageIs(
"Operation[handoff] is not supported by supervisor[id3] of type[TestSupervisorSpec]."
)
);
verifyAll();
}
@Test
public void testStartAlreadyStarted()
{
@ -359,6 +389,33 @@ public class SupervisorManagerTest extends EasyMockSupport
verifyAll();
}
@Test
public void testResetOnNonStreamSupervisor()
{
final Map<String, SupervisorSpec> existingSpecs = ImmutableMap.of(
"id3", new TestSupervisorSpec("id3", supervisor3)
);
EasyMock.expect(metadataSupervisorManager.getLatest()).andReturn(existingSpecs);
supervisor3.start();
replayAll();
manager.start();
MatcherAssert.assertThat(
Assert.assertThrows(DruidException.class, () -> manager.resetSupervisor("id3", null)),
new DruidExceptionMatcher(
DruidException.Persona.USER,
DruidException.Category.UNSUPPORTED,
"general"
).expectMessageIs(
"Operation[reset] is not supported by supervisor[id3] of type[TestSupervisorSpec]."
)
);
verifyAll();
}
@Test
public void testResetSupervisorWithSpecificOffsets()
{
@ -670,7 +727,7 @@ public class SupervisorManagerTest extends EasyMockSupport
@Override
public String getType()
{
return null;
return "TestSupervisorSpec";
}
@Override

View File

@ -118,7 +118,7 @@ public class ITAutoCompactionTest extends AbstractIndexerTest
@DataProvider(name = "engine")
public static Object[][] engine()
{
return new Object[][]{{CompactionEngine.NATIVE}, {CompactionEngine.MSQ}};
return new Object[][]{{CompactionEngine.NATIVE}};
}
@Inject

View File

@ -0,0 +1,49 @@
/*
* 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.guice;
import com.fasterxml.jackson.annotation.JsonProperty;
import javax.annotation.Nonnull;
import java.util.ArrayList;
import java.util.List;
public class DruidExtensionDependencies
{
@JsonProperty("dependsOnDruidExtensions")
private List<String> dependsOnDruidExtensions;
public DruidExtensionDependencies()
{
this.dependsOnDruidExtensions = new ArrayList<>();
}
public DruidExtensionDependencies(@Nonnull final List<String> dependsOnDruidExtensions)
{
this.dependsOnDruidExtensions = dependsOnDruidExtensions;
}
public List<String> getDependsOnDruidExtensions()
{
return dependsOnDruidExtensions;
}
}

View File

@ -24,7 +24,6 @@ import com.google.common.collect.Iterators;
import java.io.IOException;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.List;
@ -32,13 +31,13 @@ import java.util.List;
/**
* The ClassLoader that gets used when druid.extensions.useExtensionClassloaderFirst = true.
*/
public class ExtensionFirstClassLoader extends URLClassLoader
public class ExtensionFirstClassLoader extends StandardURLClassLoader
{
private final ClassLoader druidLoader;
public ExtensionFirstClassLoader(final URL[] urls, final ClassLoader druidLoader)
public ExtensionFirstClassLoader(final URL[] urls, final ClassLoader druidLoader, final List<ClassLoader> extensionDependencyClassLoaders)
{
super(urls, null);
super(urls, null, extensionDependencyClassLoaders);
this.druidLoader = Preconditions.checkNotNull(druidLoader, "druidLoader");
}
@ -60,8 +59,13 @@ public class ExtensionFirstClassLoader extends URLClassLoader
clazz = findClass(name);
}
catch (ClassNotFoundException e) {
try {
clazz = loadClassFromExtensionDependencies(name);
}
catch (ClassNotFoundException e2) {
// Try the Druid classloader. Will throw ClassNotFoundException if the class can't be loaded.
return druidLoader.loadClass(name);
clazz = druidLoader.loadClass(name);
}
}
}
@ -76,13 +80,18 @@ public class ExtensionFirstClassLoader extends URLClassLoader
@Override
public URL getResource(final String name)
{
final URL resourceFromExtension = super.getResource(name);
URL resourceFromExtension = super.getResource(name);
if (resourceFromExtension != null) {
return resourceFromExtension;
} else {
return druidLoader.getResource(name);
}
resourceFromExtension = getResourceFromExtensionsDependencies(name);
if (resourceFromExtension != null) {
return resourceFromExtension;
}
return druidLoader.getResource(name);
}
@Override
@ -90,6 +99,7 @@ public class ExtensionFirstClassLoader extends URLClassLoader
{
final List<URL> urls = new ArrayList<>();
Iterators.addAll(urls, Iterators.forEnumeration(super.getResources(name)));
addExtensionResources(name, urls);
Iterators.addAll(urls, Iterators.forEnumeration(druidLoader.getResources(name)));
return Iterators.asEnumeration(urls.iterator());
}

View File

@ -19,13 +19,19 @@
package org.apache.druid.guice;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableList;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import com.google.inject.Injector;
import org.apache.commons.io.FileUtils;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.checkerframework.checker.nullness.qual.MonotonicNonNull;
import javax.inject.Inject;
@ -34,18 +40,21 @@ import java.io.FilenameFilter;
import java.io.IOException;
import java.net.MalformedURLException;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.Collections;
import java.util.Enumeration;
import java.util.HashMap;
import java.util.HashSet;
import java.util.LinkedHashSet;
import java.util.List;
import java.util.Map;
import java.util.Optional;
import java.util.ServiceLoader;
import java.util.Set;
import java.util.concurrent.ConcurrentHashMap;
import java.util.jar.JarEntry;
import java.util.jar.JarFile;
import java.util.stream.Collectors;
/**
@ -59,19 +68,28 @@ import java.util.stream.Collectors;
public class ExtensionsLoader
{
private static final Logger log = new Logger(ExtensionsLoader.class);
public static final String DRUID_EXTENSION_DEPENDENCIES_JSON = "druid-extension-dependencies.json";
private final ExtensionsConfig extensionsConfig;
private final ConcurrentHashMap<Pair<File, Boolean>, URLClassLoader> loaders = new ConcurrentHashMap<>();
private final ObjectMapper objectMapper;
@GuardedBy("this")
private final HashMap<Pair<File, Boolean>, StandardURLClassLoader> loaders = new HashMap<>();
/**
* Map of loaded extensions, keyed by class (or interface).
*/
private final ConcurrentHashMap<Class<?>, Collection<?>> extensions = new ConcurrentHashMap<>();
@GuardedBy("this")
private final HashMap<Class<?>, Collection<?>> extensions = new HashMap<>();
@GuardedBy("this")
@MonotonicNonNull
private File[] extensionFilesToLoad;
@Inject
public ExtensionsLoader(ExtensionsConfig config)
public ExtensionsLoader(ExtensionsConfig config, ObjectMapper objectMapper)
{
this.extensionsConfig = config;
this.objectMapper = objectMapper;
}
public static ExtensionsLoader instance(Injector injector)
@ -92,6 +110,7 @@ public class ExtensionsLoader
*/
public <T> Collection<T> getLoadedImplementations(Class<T> clazz)
{
synchronized (this) {
@SuppressWarnings("unchecked")
Collection<T> retVal = (Collection<T>) extensions.get(clazz);
if (retVal == null) {
@ -99,6 +118,7 @@ public class ExtensionsLoader
}
return retVal;
}
}
/**
* @return a collection of implementations loaded.
@ -109,10 +129,12 @@ public class ExtensionsLoader
}
@VisibleForTesting
public Map<Pair<File, Boolean>, URLClassLoader> getLoadersMap()
public Map<Pair<File, Boolean>, StandardURLClassLoader> getLoadersMap()
{
synchronized (this) {
return loaders;
}
}
/**
* Look for implementations for the given class from both classpath and extensions directory, using {@link
@ -135,6 +157,7 @@ public class ExtensionsLoader
// In practice, it appears the only place this matters is with DruidModule:
// initialization gets the list of extensions, and two REST API calls later
// ask for the same list.
synchronized (this) {
Collection<?> modules = extensions.computeIfAbsent(
serviceClass,
serviceC -> new ServiceLoadingFromExtensions<>(serviceC).implsToLoad
@ -142,6 +165,7 @@ public class ExtensionsLoader
//noinspection unchecked
return (Collection<T>) modules;
}
}
public Collection<DruidModule> getModules()
{
@ -159,7 +183,7 @@ public class ExtensionsLoader
*
* @return an array of druid extension files that will be loaded by druid process
*/
public File[] getExtensionFilesToLoad()
public void initializeExtensionFilesToLoad()
{
final File rootExtensionsDir = new File(extensionsConfig.getDirectory());
if (rootExtensionsDir.exists() && !rootExtensionsDir.isDirectory()) {
@ -187,25 +211,98 @@ public class ExtensionsLoader
extensionsToLoad[i++] = extensionDir;
}
}
return extensionsToLoad == null ? new File[]{} : extensionsToLoad;
synchronized (this) {
extensionFilesToLoad = extensionsToLoad == null ? new File[]{} : extensionsToLoad;
}
}
public File[] getExtensionFilesToLoad()
{
synchronized (this) {
if (extensionFilesToLoad == null) {
initializeExtensionFilesToLoad();
}
return extensionFilesToLoad;
}
}
/**
* @param extension The File instance of the extension we want to load
*
* @return a URLClassLoader that loads all the jars on which the extension is dependent
* @return a StandardURLClassLoader that loads all the jars on which the extension is dependent
*/
public URLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst)
public StandardURLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst)
{
return loaders.computeIfAbsent(
Pair.of(extension, useExtensionClassloaderFirst),
k -> makeClassLoaderForExtension(k.lhs, k.rhs)
return getClassLoaderForExtension(extension, useExtensionClassloaderFirst, new ArrayList<>());
}
/**
* @param extension The File instance of the extension we want to load
* @param useExtensionClassloaderFirst Whether to return a StandardURLClassLoader that checks extension classloaders first for classes
* @param extensionDependencyStack If the extension is requested as a dependency of another extension, a list containing the
* dependency stack of the dependent extension (for checking circular dependencies). Otherwise
* this is a empty list.
* @return a StandardURLClassLoader that loads all the jars on which the extension is dependent
*/
public StandardURLClassLoader getClassLoaderForExtension(File extension, boolean useExtensionClassloaderFirst, List<String> extensionDependencyStack)
{
Pair<File, Boolean> classLoaderKey = Pair.of(extension, useExtensionClassloaderFirst);
synchronized (this) {
StandardURLClassLoader classLoader = loaders.get(classLoaderKey);
if (classLoader == null) {
classLoader = makeClassLoaderWithDruidExtensionDependencies(extension, useExtensionClassloaderFirst, extensionDependencyStack);
loaders.put(classLoaderKey, classLoader);
}
return classLoader;
}
}
private StandardURLClassLoader makeClassLoaderWithDruidExtensionDependencies(File extension, boolean useExtensionClassloaderFirst, List<String> extensionDependencyStack)
{
Optional<DruidExtensionDependencies> druidExtensionDependenciesOptional = getDruidExtensionDependencies(extension);
List<String> druidExtensionDependenciesList = druidExtensionDependenciesOptional.isPresent()
? druidExtensionDependenciesOptional.get().getDependsOnDruidExtensions()
: ImmutableList.of();
List<ClassLoader> extensionDependencyClassLoaders = new ArrayList<>();
for (String druidExtensionDependencyName : druidExtensionDependenciesList) {
Optional<File> extensionDependencyFileOptional = Arrays.stream(getExtensionFilesToLoad())
.filter(file -> file.getName().equals(druidExtensionDependencyName))
.findFirst();
if (!extensionDependencyFileOptional.isPresent()) {
throw new RE(
StringUtils.format(
"Extension [%s] depends on [%s] which is not a valid extension or not loaded.",
extension.getName(),
druidExtensionDependencyName
)
);
}
File extensionDependencyFile = extensionDependencyFileOptional.get();
if (extensionDependencyStack.contains(extensionDependencyFile.getName())) {
extensionDependencyStack.add(extensionDependencyFile.getName());
throw new RE(
StringUtils.format(
"Extension [%s] has a circular druid extension dependency. Dependency stack [%s].",
extensionDependencyStack.get(0),
extensionDependencyStack
)
);
}
extensionDependencyStack.add(extensionDependencyFile.getName());
extensionDependencyClassLoaders.add(
getClassLoaderForExtension(extensionDependencyFile, useExtensionClassloaderFirst, extensionDependencyStack)
);
}
private static URLClassLoader makeClassLoaderForExtension(
return makeClassLoaderForExtension(extension, useExtensionClassloaderFirst, extensionDependencyClassLoaders);
}
private static StandardURLClassLoader makeClassLoaderForExtension(
final File extension,
final boolean useExtensionClassloaderFirst
final boolean useExtensionClassloaderFirst,
final List<ClassLoader> extensionDependencyClassLoaders
)
{
final Collection<File> jars = FileUtils.listFiles(extension, new String[]{"jar"}, false);
@ -224,9 +321,9 @@ public class ExtensionsLoader
}
if (useExtensionClassloaderFirst) {
return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader());
return new ExtensionFirstClassLoader(urls, ExtensionsLoader.class.getClassLoader(), extensionDependencyClassLoaders);
} else {
return new URLClassLoader(urls, ExtensionsLoader.class.getClassLoader());
return new StandardURLClassLoader(urls, ExtensionsLoader.class.getClassLoader(), extensionDependencyClassLoaders);
}
}
@ -266,6 +363,45 @@ public class ExtensionsLoader
}
}
private Optional<DruidExtensionDependencies> getDruidExtensionDependencies(File extension)
{
final Collection<File> jars = FileUtils.listFiles(extension, new String[]{"jar"}, false);
DruidExtensionDependencies druidExtensionDependencies = null;
String druidExtensionDependenciesJarName = null;
for (File extensionFile : jars) {
try (JarFile jarFile = new JarFile(extensionFile.getPath())) {
Enumeration<JarEntry> entries = jarFile.entries();
while (entries.hasMoreElements()) {
JarEntry entry = entries.nextElement();
String entryName = entry.getName();
if (DRUID_EXTENSION_DEPENDENCIES_JSON.equals(entryName)) {
log.debug("Found extension dependency entry in jar [%s]", extensionFile.getPath());
if (druidExtensionDependenciesJarName != null) {
throw new RE(
StringUtils.format(
"The extension [%s] has multiple jars [%s] [%s] with dependencies in them. Each jar should be in a separate extension directory.",
extension.getName(),
druidExtensionDependenciesJarName,
jarFile.getName()
)
);
}
druidExtensionDependencies = objectMapper.readValue(
jarFile.getInputStream(entry),
DruidExtensionDependencies.class
);
druidExtensionDependenciesJarName = jarFile.getName();
}
}
}
catch (IOException e) {
throw new RE(e, "Failed to get dependencies for extension [%s]", extension);
}
}
return druidExtensionDependencies == null ? Optional.empty() : Optional.of(druidExtensionDependencies);
}
private class ServiceLoadingFromExtensions<T>
{
private final Class<T> serviceClass;
@ -293,17 +429,17 @@ public class ExtensionsLoader
for (File extension : getExtensionFilesToLoad()) {
log.debug("Loading extension [%s] for class [%s]", extension.getName(), serviceClass);
try {
final URLClassLoader loader = getClassLoaderForExtension(
final StandardURLClassLoader loader = getClassLoaderForExtension(
extension,
extensionsConfig.isUseExtensionClassloaderFirst()
);
log.info(
"Loading extension [%s], jars: %s",
"Loading extension [%s], jars: %s. Druid extension dependencies [%s]",
extension.getName(),
Arrays.stream(loader.getURLs())
.map(u -> new File(u.getPath()).getName())
.collect(Collectors.joining(", "))
.collect(Collectors.joining(", ")),
loader.getExtensionDependencyClassLoaders()
);
ServiceLoader.load(serviceClass, loader).forEach(impl -> tryAdd(impl, "local file system"));

View File

@ -0,0 +1,119 @@
/*
* 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.guice;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterators;
import java.io.IOException;
import java.net.URL;
import java.net.URLClassLoader;
import java.util.ArrayList;
import java.util.Enumeration;
import java.util.List;
/**
* The ClassLoader that gets used when druid.extensions.useExtensionClassloaderFirst = false.
*/
public class StandardURLClassLoader extends URLClassLoader
{
private final List<ClassLoader> extensionDependencyClassLoaders;
public StandardURLClassLoader(final URL[] urls, final ClassLoader druidLoader, final List<ClassLoader> extensionDependencyClassLoaders)
{
super(urls, druidLoader);
this.extensionDependencyClassLoaders = Preconditions.checkNotNull(extensionDependencyClassLoaders, "extensionDependencyClassLoaders");
}
@Override
protected Class<?> loadClass(final String name, final boolean resolve) throws ClassNotFoundException
{
Class<?> clazz;
try {
clazz = super.loadClass(name, resolve);
}
catch (ClassNotFoundException e) {
clazz = loadClassFromExtensionDependencies(name);
}
if (resolve) {
resolveClass(clazz);
}
return clazz;
}
@Override
public URL getResource(final String name)
{
URL resource = super.getResource(name);
if (resource != null) {
return resource;
}
return getResourceFromExtensionsDependencies(name);
}
@Override
public Enumeration<URL> getResources(final String name) throws IOException
{
final List<URL> urls = new ArrayList<>();
Iterators.addAll(urls, Iterators.forEnumeration(super.getResources(name)));
addExtensionResources(name, urls);
return Iterators.asEnumeration(urls.iterator());
}
protected URL getResourceFromExtensionsDependencies(final String name)
{
URL resourceFromExtension = null;
for (ClassLoader classLoader : extensionDependencyClassLoaders) {
resourceFromExtension = classLoader.getResource(name);
if (resourceFromExtension != null) {
break;
}
}
return resourceFromExtension;
}
protected Class<?> loadClassFromExtensionDependencies(final String name) throws ClassNotFoundException
{
for (ClassLoader classLoader : extensionDependencyClassLoaders) {
try {
return classLoader.loadClass(name);
}
catch (ClassNotFoundException ignored) {
}
}
throw new ClassNotFoundException();
}
protected void addExtensionResources(final String name, List<URL> urls) throws IOException
{
for (ClassLoader classLoader : extensionDependencyClassLoaders) {
Iterators.addAll(urls, Iterators.forEnumeration(classLoader.getResources(name)));
}
}
public List<ClassLoader> getExtensionDependencyClassLoaders()
{
return extensionDependencyClassLoaders;
}
}

View File

@ -38,12 +38,6 @@ public class CgroupCpuMonitor extends FeedDefiningMonitor
{
private static final Logger LOG = new Logger(CgroupCpuMonitor.class);
private static final Long DEFAULT_USER_HZ = 100L;
public static final String TOTAL_USAGE_METRIC = "cgroup/cpu/usage/total/percentage";
public static final String USER_USAGE_METRIC = "cgroup/cpu/usage/user/percentage";
public static final String SYS_USAGE_METRIC = "cgroup/cpu/usage/sys/percentage";
private static final String TOTAL = "total";
private static final String USER = "user";
private static final String SYSTEM = "system";
final CgroupDiscoverer cgroupDiscoverer;
final Map<String, String[]> dimensions;
private Long userHz;
@ -111,18 +105,18 @@ public class CgroupCpuMonitor extends FeedDefiningMonitor
final Map<String, Long> elapsedJiffies = jiffies.to(
"usage",
ImmutableMap.<String, Long>builder()
.put(USER, cpuSnapshot.getUserJiffies())
.put(SYSTEM, cpuSnapshot.getSystemJiffies())
.put(TOTAL, cpuSnapshot.getTotalJiffies())
.put(CgroupUtil.USER, cpuSnapshot.getUserJiffies())
.put(CgroupUtil.SYSTEM, cpuSnapshot.getSystemJiffies())
.put(CgroupUtil.TOTAL, cpuSnapshot.getTotalJiffies())
.build()
);
if (elapsedJiffies != null) {
double totalUsagePct = 100.0 * elapsedJiffies.get(TOTAL) / userHz / elapsedJiffiesSnapshotSecs;
double sysUsagePct = 100.0 * elapsedJiffies.get(SYSTEM) / userHz / elapsedJiffiesSnapshotSecs;
double userUsagePct = 100.0 * elapsedJiffies.get(USER) / userHz / elapsedJiffiesSnapshotSecs;
emitter.emit(builder.setMetric(TOTAL_USAGE_METRIC, totalUsagePct));
emitter.emit(builder.setMetric(SYS_USAGE_METRIC, sysUsagePct));
emitter.emit(builder.setMetric(USER_USAGE_METRIC, userUsagePct));
double totalUsagePct = 100.0 * elapsedJiffies.get(CgroupUtil.TOTAL) / userHz / elapsedJiffiesSnapshotSecs;
double sysUsagePct = 100.0 * elapsedJiffies.get(CgroupUtil.SYSTEM) / userHz / elapsedJiffiesSnapshotSecs;
double userUsagePct = 100.0 * elapsedJiffies.get(CgroupUtil.USER) / userHz / elapsedJiffiesSnapshotSecs;
emitter.emit(builder.setMetric(CgroupUtil.CPU_TOTAL_USAGE_METRIC, totalUsagePct));
emitter.emit(builder.setMetric(CgroupUtil.CPU_SYS_USAGE_METRIC, sysUsagePct));
emitter.emit(builder.setMetric(CgroupUtil.CPU_USER_USAGE_METRIC, userUsagePct));
}
}
return true;

View File

@ -64,10 +64,10 @@ public class CgroupDiskMonitor extends FeedDefiningMonitor
final Map<String, Long> stats = diff.to(
entry.getKey(),
ImmutableMap.<String, Long>builder()
.put("cgroup/disk/read/bytes", entry.getValue().getReadBytes())
.put("cgroup/disk/read/count", entry.getValue().getReadCount())
.put("cgroup/disk/write/bytes", entry.getValue().getWriteBytes())
.put("cgroup/disk/write/count", entry.getValue().getWriteCount())
.put(CgroupUtil.DISK_READ_BYTES_METRIC, entry.getValue().getReadBytes())
.put(CgroupUtil.DISK_READ_COUNT_METRIC, entry.getValue().getReadCount())
.put(CgroupUtil.DISK_WRITE_BYTES_METRIC, entry.getValue().getWriteBytes())
.put(CgroupUtil.DISK_WRITE_COUNT_METRIC, entry.getValue().getWriteCount())
.build()
);

View File

@ -60,7 +60,7 @@ public class CgroupMemoryMonitor extends FeedDefiningMonitor
public boolean doMonitor(ServiceEmitter emitter)
{
final Memory memory = new Memory(cgroupDiscoverer);
final Memory.MemoryStat stat = memory.snapshot();
final Memory.MemoryStat stat = memory.snapshot(memoryUsageFile(), memoryLimitFile());
final ServiceMetricEvent.Builder builder = builder();
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
emitter.emit(builder.setMetric("cgroup/memory/usage/bytes", stat.getUsage()));
@ -77,4 +77,14 @@ public class CgroupMemoryMonitor extends FeedDefiningMonitor
});
return true;
}
public String memoryUsageFile()
{
return "memory.usage_in_bytes";
}
public String memoryLimitFile()
{
return "memory.limit_in_bytes";
}
}

View File

@ -35,6 +35,17 @@ public class CgroupUtil
private static final Logger LOG = new Logger(CgroupUtil.class);
public static final String SPACE_MATCH = Pattern.quote(" ");
public static final String COMMA_MATCH = Pattern.quote(",");
public static final String TOTAL = "total";
public static final String USER = "user";
public static final String SYSTEM = "system";
public static final String CPU_TOTAL_USAGE_METRIC = "cgroup/cpu/usage/total/percentage";
public static final String CPU_USER_USAGE_METRIC = "cgroup/cpu/usage/user/percentage";
public static final String CPU_SYS_USAGE_METRIC = "cgroup/cpu/usage/sys/percentage";
public static final String DISK_READ_BYTES_METRIC = "cgroup/disk/read/bytes";
public static final String DISK_READ_COUNT_METRIC = "cgroup/disk/read/count";
public static final String DISK_WRITE_BYTES_METRIC = "cgroup/disk/write/bytes";
public static final String DISK_WRITE_COUNT_METRIC = "cgroup/disk/write/count";
public static long readLongValue(CgroupDiscoverer discoverer, String cgroup, String fileName, long defaultValue)
{

View File

@ -0,0 +1,163 @@
/*
* 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.java.util.metrics;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
import com.google.common.primitives.Longs;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.Cpu;
import org.apache.druid.java.util.metrics.cgroups.ProcCgroupV2Discoverer;
import org.apache.druid.java.util.metrics.cgroups.ProcSelfCgroupDiscoverer;
import java.io.BufferedReader;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.time.Instant;
import java.time.temporal.ChronoUnit;
import java.util.HashMap;
import java.util.Map;
import java.util.regex.Pattern;
/**
* Monitor that reports cpu usage stats by reading `cpu.stat` reported by cgroupv2
*/
public class CgroupV2CpuMonitor extends FeedDefiningMonitor
{
private static final Logger LOG = new Logger(CgroupV2CpuMonitor.class);
private static final String CPU_STAT_FILE = "cpu.stat";
private static final String SNAPSHOT = "snapshot";
final CgroupDiscoverer cgroupDiscoverer;
final Map<String, String[]> dimensions;
private final KeyedDiff diff = new KeyedDiff();
public CgroupV2CpuMonitor(CgroupDiscoverer cgroupDiscoverer, final Map<String, String[]> dimensions, String feed)
{
super(feed);
this.cgroupDiscoverer = cgroupDiscoverer;
this.dimensions = dimensions;
}
@VisibleForTesting
CgroupV2CpuMonitor(CgroupDiscoverer cgroupDiscoverer)
{
this(cgroupDiscoverer, ImmutableMap.of(), DEFAULT_METRICS_FEED);
}
CgroupV2CpuMonitor()
{
this(new ProcSelfCgroupDiscoverer(ProcCgroupV2Discoverer.class));
}
@Override
public boolean doMonitor(ServiceEmitter emitter)
{
final ServiceMetricEvent.Builder builder = builder();
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
Snapshot snapshot = snapshot();
final Map<String, Long> elapsed = diff.to(
"usage",
ImmutableMap.<String, Long>builder()
.put(CgroupUtil.USER, snapshot.getUserUsec())
.put(CgroupUtil.SYSTEM, snapshot.getSystemUsec())
.put(CgroupUtil.TOTAL, snapshot.getUsageUsec())
.put(SNAPSHOT, ChronoUnit.MICROS.between(Instant.EPOCH, Instant.now()))
.build()
);
if (elapsed != null) {
long elapsedUsecs = elapsed.get(SNAPSHOT);
double totalUsagePct = 100.0 * elapsed.get(CgroupUtil.TOTAL) / elapsedUsecs;
double sysUsagePct = 100.0 * elapsed.get(CgroupUtil.SYSTEM) / elapsedUsecs;
double userUsagePct = 100.0 * elapsed.get(CgroupUtil.USER) / elapsedUsecs;
emitter.emit(builder.setMetric(CgroupUtil.CPU_TOTAL_USAGE_METRIC, totalUsagePct));
emitter.emit(builder.setMetric(CgroupUtil.CPU_SYS_USAGE_METRIC, sysUsagePct));
emitter.emit(builder.setMetric(CgroupUtil.CPU_USER_USAGE_METRIC, userUsagePct));
}
return true;
}
/*
file: cpu.stat
sample content:
usage_usec 2379951538
user_usec 1802023024
system_usec 577928513
nr_periods 1581231
nr_throttled 59
throttled_usec 3095133
*/
public Snapshot snapshot()
{
Map<String, Long> entries = new HashMap<>();
try (final BufferedReader reader = Files.newBufferedReader(
Paths.get(cgroupDiscoverer.discover(Cpu.CGROUP).toString(), CPU_STAT_FILE)
)) {
for (String line = reader.readLine(); line != null; line = reader.readLine()) {
final String[] parts = line.split(Pattern.quote(" "));
if (parts.length != 2) {
// ignore
continue;
}
entries.put(parts[0], Longs.tryParse(parts[1]));
}
}
catch (IOException | RuntimeException ex) {
LOG.error(ex, "Unable to fetch cpu snapshot");
}
return new Snapshot(entries.get("usage_usec"), entries.get("user_usec"), entries.get("system_usec"));
}
public static class Snapshot
{
private final long usageUsec;
private final long userUsec;
private final long systemUsec;
public Snapshot(long usageUsec, long userUsec, long systemUsec)
{
this.usageUsec = usageUsec;
this.userUsec = userUsec;
this.systemUsec = systemUsec;
}
public long getUsageUsec()
{
return usageUsec;
}
public long getUserUsec()
{
return userUsec;
}
public long getSystemUsec()
{
return systemUsec;
}
}
}

View File

@ -0,0 +1,140 @@
/*
* 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.java.util.metrics;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.Disk;
import org.apache.druid.java.util.metrics.cgroups.ProcCgroupV2Discoverer;
import org.apache.druid.java.util.metrics.cgroups.ProcSelfCgroupDiscoverer;
import java.io.BufferedReader;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
/**
* Monitor that reports disk usage stats by reading `io.stat` reported by cgroupv2
*/
public class CgroupV2DiskMonitor extends FeedDefiningMonitor
{
private static final Logger LOG = new Logger(CgroupV2DiskMonitor.class);
private static final String IO_STAT = "io.stat";
final CgroupDiscoverer cgroupDiscoverer;
final Map<String, String[]> dimensions;
private final KeyedDiff diff = new KeyedDiff();
public CgroupV2DiskMonitor(CgroupDiscoverer cgroupDiscoverer, final Map<String, String[]> dimensions, String feed)
{
super(feed);
this.cgroupDiscoverer = cgroupDiscoverer;
this.dimensions = dimensions;
}
@VisibleForTesting
CgroupV2DiskMonitor(CgroupDiscoverer cgroupDiscoverer)
{
this(cgroupDiscoverer, ImmutableMap.of(), DEFAULT_METRICS_FEED);
}
CgroupV2DiskMonitor()
{
this(new ProcSelfCgroupDiscoverer(ProcCgroupV2Discoverer.class));
}
@Override
public boolean doMonitor(ServiceEmitter emitter)
{
for (Disk.Metrics entry : snapshot()) {
final Map<String, Long> stats = diff.to(
entry.getDiskName(),
ImmutableMap.<String, Long>builder()
.put(CgroupUtil.DISK_READ_BYTES_METRIC, entry.getReadBytes())
.put(CgroupUtil.DISK_READ_COUNT_METRIC, entry.getReadCount())
.put(CgroupUtil.DISK_WRITE_BYTES_METRIC, entry.getWriteBytes())
.put(CgroupUtil.DISK_WRITE_COUNT_METRIC, entry.getWriteCount())
.build()
);
if (stats != null) {
final ServiceMetricEvent.Builder builder = builder()
.setDimension("diskName", entry.getDiskName());
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
for (Map.Entry<String, Long> stat : stats.entrySet()) {
emitter.emit(builder.setMetric(stat.getKey(), stat.getValue()));
}
}
}
return true;
}
/*
file: io.stat
sample content:
8:0 rbytes=933898 wbytes=110870538 rios=238 wios=7132 dbytes=0 dios=0
15:0 rbytes=34566 wbytes=3466756 rios=23 wios=71 dbytes=0 dios=0
*/
public List<Disk.Metrics> snapshot()
{
List<Disk.Metrics> diskStats = new ArrayList<>();
try (final BufferedReader reader = Files.newBufferedReader(
Paths.get(cgroupDiscoverer.discover("disk").toString(), IO_STAT))) {
for (String line = reader.readLine(); line != null; line = reader.readLine()) {
Disk.Metrics disk = getDiskMetrics(line);
diskStats.add(disk);
}
}
catch (IOException | RuntimeException ex) {
LOG.error(ex, "Unable to fetch memory snapshot");
}
return diskStats;
}
private static Disk.Metrics getDiskMetrics(String line)
{
final String[] parts = line.split(Pattern.quote(" "));
Disk.Metrics disk = new Disk.Metrics(parts[0]);
Map<String, Long> stats = new HashMap<>();
for (int i = 1; i < parts.length; i++) {
String[] keyValue = parts[i].split("=");
if (keyValue.length == 2) {
stats.put(keyValue[0], Long.parseLong(keyValue[1]));
}
}
disk.setReadBytes(stats.get("rbytes"));
disk.setReadCount(stats.get("rios"));
disk.setWriteBytes(stats.get("wbytes"));
disk.setWriteCount(stats.get("wios"));
return disk;
}
}

View File

@ -0,0 +1,58 @@
/*
* 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.java.util.metrics;
import com.google.common.annotations.VisibleForTesting;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
import org.apache.druid.java.util.metrics.cgroups.ProcCgroupV2Discoverer;
import org.apache.druid.java.util.metrics.cgroups.ProcSelfCgroupDiscoverer;
/**
* Monitor that reports memory usage stats by reading `memory.*` files reported by cgroupv2
*/
public class CgroupV2MemoryMonitor extends CgroupMemoryMonitor
{
@VisibleForTesting
CgroupV2MemoryMonitor(CgroupDiscoverer cgroupDiscoverer)
{
super(cgroupDiscoverer, ImmutableMap.of(), DEFAULT_METRICS_FEED);
}
// This would be invoked when monitor is specified in config (supressing to satisy intellij-inspections)
@SuppressWarnings("unused")
CgroupV2MemoryMonitor()
{
this(new ProcSelfCgroupDiscoverer(ProcCgroupV2Discoverer.class));
}
@Override
public String memoryUsageFile()
{
return "memory.current";
}
@Override
public String memoryLimitFile()
{
return "memory.max";
}
}

View File

@ -34,8 +34,8 @@ import java.util.regex.Pattern;
*/
public class Cpu
{
public static final String CGROUP = "cpu";
private static final Logger LOG = new Logger(Cpu.class);
private static final String CGROUP = "cpu";
private static final String CPUACCT_STAT_FILE = "cpuacct.stat";
private static final String CPU_SHARES_FILE = "cpu.shares";
private static final String CPU_QUOTA_FILE = "cpu.cfs_quota_us";

View File

@ -153,6 +153,26 @@ public class Disk
return diskName;
}
public void setReadCount(long readCount)
{
this.readCount = readCount;
}
public void setWriteCount(long writeCount)
{
this.writeCount = writeCount;
}
public void setReadBytes(long readBytes)
{
this.readBytes = readBytes;
}
public void setWriteBytes(long writeBytes)
{
this.writeBytes = writeBytes;
}
@Override
public boolean equals(Object o)
{

View File

@ -37,8 +37,6 @@ public class Memory
private static final Logger LOG = new Logger(Memory.class);
private static final String CGROUP = "memory";
private static final String CGROUP_MEMORY_FILE = "memory.stat";
private static final String MEMORY_USAGE_FILE = "memory.usage_in_bytes";
private static final String MEMORY_LIMIT_FILE = "memory.limit_in_bytes";
private static final String CGROUP_MEMORY_NUMA_FILE = "memory.numa_stat";
private final CgroupDiscoverer cgroupDiscoverer;
@ -47,11 +45,11 @@ public class Memory
this.cgroupDiscoverer = cgroupDiscoverer;
}
public MemoryStat snapshot()
public MemoryStat snapshot(String memoryUsageFile, String memoryLimitFile)
{
final MemoryStat memoryStat = new MemoryStat();
memoryStat.usage = CgroupUtil.readLongValue(cgroupDiscoverer, CGROUP, MEMORY_USAGE_FILE, -1);
memoryStat.limit = CgroupUtil.readLongValue(cgroupDiscoverer, CGROUP, MEMORY_LIMIT_FILE, -1);
memoryStat.usage = CgroupUtil.readLongValue(cgroupDiscoverer, CGROUP, memoryUsageFile, -1);
memoryStat.limit = CgroupUtil.readLongValue(cgroupDiscoverer, CGROUP, memoryLimitFile, -1);
try (final BufferedReader reader = Files.newBufferedReader(
Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), CGROUP_MEMORY_FILE)

View File

@ -41,7 +41,7 @@ public class ProcCgroupDiscoverer implements CgroupDiscoverer
{
private static final String CGROUP_TYPE = "cgroup";
private final File procDir;
protected final File procDir;
/**
* Create a proc discovery mechanism based on a `/proc` directory.
@ -118,7 +118,7 @@ public class ProcCgroupDiscoverer implements CgroupDiscoverer
throw new RE("Hierarchy for [%s] not found", cgroup);
}
private ProcMountsEntry getMountEntry(final File procMounts, final String cgroup)
protected ProcMountsEntry getMountEntry(final File procMounts, final String cgroup)
{
final List<String> lines;
try {

View File

@ -0,0 +1,61 @@
/*
* 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.java.util.metrics.cgroups;
import com.google.common.io.Files;
import org.apache.druid.java.util.common.RE;
import java.io.File;
import java.io.IOException;
import java.nio.charset.StandardCharsets;
import java.nio.file.Path;
public class ProcCgroupV2Discoverer extends ProcCgroupDiscoverer
{
private static final String CGROUP_TYPE = "cgroup2";
/**
* Create a proc discovery mechanism based on a `/proc` directory.
*
* @param procDir The directory under proc. This is usually `/proc/self` or `/proc/#pid`
*/
public ProcCgroupV2Discoverer(Path procDir)
{
super(procDir);
}
@Override
public Path discover(String cgroup)
{
try {
for (final String line : Files.readLines(new File(procDir, "mounts"), StandardCharsets.UTF_8)) {
final ProcMountsEntry entry = ProcMountsEntry.parse(line);
if (CGROUP_TYPE.equals(entry.type)) {
return entry.path;
}
}
}
catch (IOException e) {
throw new RuntimeException(e);
}
throw new RE("Cgroup location not found");
}
}

View File

@ -24,11 +24,21 @@ import java.nio.file.Paths;
public class ProcSelfCgroupDiscoverer implements CgroupDiscoverer
{
private final ProcCgroupDiscoverer delegate;
private final CgroupDiscoverer delegate;
public ProcSelfCgroupDiscoverer()
{
delegate = new ProcCgroupDiscoverer(Paths.get("/proc/self"));
this(ProcCgroupDiscoverer.class);
}
public ProcSelfCgroupDiscoverer(Class<? extends CgroupDiscoverer> discoverer)
{
try {
delegate = discoverer.getDeclaredConstructor(Path.class).newInstance(Paths.get("/proc/self"));
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
@Override

View File

@ -23,6 +23,7 @@ import com.google.common.collect.ImmutableSet;
import it.unimi.dsi.fastutil.objects.Object2IntArrayMap;
import it.unimi.dsi.fastutil.objects.Object2IntMap;
import it.unimi.dsi.fastutil.objects.Object2IntOpenHashMap;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
@ -135,10 +136,28 @@ public interface ApplyFunction extends NamedFunction
{
final int length = bindings.getLength();
Object[] out = new Object[length];
final boolean computeArrayType = arrayType == null;
ExpressionType arrayElementType = arrayType != null
? (ExpressionType) arrayType.getElementType()
: null;
final ExprEval<?>[] outEval = computeArrayType ? new ExprEval[length] : null;
for (int i = 0; i < length; i++) {
ExprEval evaluated = expr.eval(bindings.withIndex(i));
arrayType = Function.ArrayConstructorFunction.setArrayOutput(arrayType, out, i, evaluated);
final ExprEval<?> eval = expr.eval(bindings.withIndex(i));
if (computeArrayType && outEval[i].value() != null) {
arrayElementType = ExpressionTypeConversion.leastRestrictiveType(arrayElementType, eval.type());
outEval[i] = eval;
} else {
out[i] = eval.castTo(arrayElementType).value();
}
}
if (arrayElementType == null) {
arrayElementType = NullHandling.sqlCompatible() ? ExpressionType.LONG : ExpressionType.STRING;
}
if (computeArrayType) {
arrayType = ExpressionTypeFactory.getInstance().ofArray(arrayElementType);
for (int i = 0; i < length; i++) {
out[i] = outEval[i].castTo(arrayElementType).value();
}
}
return ExprEval.ofArray(arrayType, out);
}
@ -237,7 +256,7 @@ public interface ApplyFunction extends NamedFunction
List<List<Object>> product = CartesianList.create(arrayInputs);
CartesianMapLambdaBinding lambdaBinding = new CartesianMapLambdaBinding(elementType, product, lambdaExpr, bindings);
ExpressionType lambdaType = lambdaExpr.getOutputType(lambdaBinding);
return applyMap(ExpressionType.asArrayType(lambdaType), lambdaExpr, lambdaBinding);
return applyMap(lambdaType == null ? null : ExpressionTypeFactory.getInstance().ofArray(lambdaType), lambdaExpr, lambdaBinding);
}
@Override

View File

@ -127,6 +127,11 @@ abstract class ConstantExpr<T> implements Expr
return new ExprEvalBasedConstantExpr<T>(realEval());
}
@Override
public <E> ExprVectorProcessor<E> asVectorProcessor(VectorInputBindingInspector inspector)
{
return VectorProcessors.constant(value, inspector.getMaxVectorSize(), outputType);
}
/**
* Constant expression based on a concreate ExprEval.
*
@ -415,7 +420,7 @@ class StringExpr extends ConstantExpr<String>
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
return VectorProcessors.constant(value, inspector.getMaxVectorSize());
return VectorProcessors.constant(value, inspector.getMaxVectorSize(), ExpressionType.STRING);
}
@Override
@ -459,12 +464,6 @@ class ArrayExpr extends ConstantExpr<Object[]>
return ExprEval.ofArray(outputType, value);
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return false;
}
@Override
public String stringify()
{
@ -547,12 +546,6 @@ class ComplexExpr extends ConstantExpr<Object>
return ExprEval.ofComplex(outputType, value);
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return false;
}
@Override
public String stringify()
{

View File

@ -2026,7 +2026,8 @@ public interface Function extends NamedFunction
{
return CastToTypeVectorProcessor.cast(
args.get(0).asVectorProcessor(inspector),
ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString()))
ExpressionType.fromString(StringUtils.toUpperCase(args.get(1).getLiteralValue().toString())),
inspector.getMaxVectorSize()
);
}
}
@ -3357,19 +3358,24 @@ public interface Function extends NamedFunction
@Override
public ExprEval apply(List<Expr> args, Expr.ObjectBinding bindings)
{
// this is copied from 'BaseMapFunction.applyMap', need to find a better way to consolidate, or construct arrays,
// or.. something...
final int length = args.size();
Object[] out = new Object[length];
ExpressionType arrayType = null;
ExpressionType arrayElementType = null;
final ExprEval[] outEval = new ExprEval[length];
for (int i = 0; i < length; i++) {
ExprEval<?> evaluated = args.get(i).eval(bindings);
arrayType = setArrayOutput(arrayType, out, i, evaluated);
outEval[i] = args.get(i).eval(bindings);
if (outEval[i].value() != null) {
arrayElementType = ExpressionTypeConversion.leastRestrictiveType(arrayElementType, outEval[i].type());
}
return ExprEval.ofArray(arrayType, out);
}
if (arrayElementType == null) {
arrayElementType = NullHandling.sqlCompatible() ? ExpressionType.LONG : ExpressionType.STRING;
}
for (int i = 0; i < length; i++) {
out[i] = outEval[i].castTo(arrayElementType).value();
}
return ExprEval.ofArray(ExpressionTypeFactory.getInstance().ofArray(arrayElementType), out);
}
@Override
@ -3394,28 +3400,6 @@ public interface Function extends NamedFunction
}
return type == null ? null : ExpressionTypeFactory.getInstance().ofArray(type);
}
/**
* Set an array element to the output array, checking for null if the array is numeric. If the type of the evaluated
* array element does not match the array element type, this method will attempt to call {@link ExprEval#castTo}
* to the array element type, else will set the element as is. If the type of the array is unknown, it will be
* detected and defined from the first element. Returns the type of the array, which will be identical to the input
* type, unless the input type was null.
*/
static ExpressionType setArrayOutput(@Nullable ExpressionType arrayType, Object[] out, int i, ExprEval evaluated)
{
if (arrayType == null) {
arrayType = ExpressionTypeFactory.getInstance().ofArray(evaluated.type());
}
if (arrayType.getElementType().isNumeric() && evaluated.isNumericNull()) {
out[i] = null;
} else if (!evaluated.asArrayType().equals(arrayType)) {
out[i] = evaluated.castTo((ExpressionType) arrayType.getElementType()).value();
} else {
out[i] = evaluated.value();
}
return arrayType;
}
}
class ArrayLengthFunction implements Function
@ -3954,6 +3938,9 @@ public interface Function extends NamedFunction
return ExprEval.ofLongBoolean(Arrays.asList(array1).containsAll(Arrays.asList(array2)));
} else {
final Object elem = rhsExpr.castTo((ExpressionType) array1Type.getElementType()).value();
if (elem == null && rhsExpr.value() != null) {
return ExprEval.ofLongBoolean(false);
}
return ExprEval.ofLongBoolean(Arrays.asList(array1).contains(elem));
}
}

View File

@ -19,7 +19,6 @@
package org.apache.druid.math.expr;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import org.apache.druid.error.DruidException;
import org.apache.druid.java.util.common.StringUtils;
@ -30,8 +29,6 @@ import org.apache.druid.segment.column.Types;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
@SuppressWarnings("unused")
final class FunctionalExpr
@ -39,127 +36,6 @@ final class FunctionalExpr
// phony class to enable maven to track the compilation of this class
}
@SuppressWarnings("ClassName")
class LambdaExpr implements Expr
{
private final ImmutableList<IdentifierExpr> args;
private final Expr expr;
LambdaExpr(List<IdentifierExpr> args, Expr expr)
{
this.args = ImmutableList.copyOf(args);
this.expr = expr;
}
@Override
public String toString()
{
return StringUtils.format("(%s -> %s)", args, expr);
}
int identifierCount()
{
return args.size();
}
@Nullable
public String getIdentifier()
{
Preconditions.checkState(args.size() < 2, "LambdaExpr has multiple arguments, use getIdentifiers");
if (args.size() == 1) {
return args.get(0).toString();
}
return null;
}
public List<String> getIdentifiers()
{
return args.stream().map(IdentifierExpr::toString).collect(Collectors.toList());
}
public List<String> stringifyIdentifiers()
{
return args.stream().map(IdentifierExpr::stringify).collect(Collectors.toList());
}
ImmutableList<IdentifierExpr> getIdentifierExprs()
{
return args;
}
public Expr getExpr()
{
return expr;
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return expr.canVectorize(inspector);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
return expr.asVectorProcessor(inspector);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return expr.eval(bindings);
}
@Override
public String stringify()
{
return StringUtils.format("(%s) -> %s", ARG_JOINER.join(stringifyIdentifiers()), expr.stringify());
}
@Override
public Expr visit(Shuttle shuttle)
{
List<IdentifierExpr> newArgs =
args.stream().map(arg -> (IdentifierExpr) shuttle.visit(arg)).collect(Collectors.toList());
Expr newBody = expr.visit(shuttle);
return shuttle.visit(new LambdaExpr(newArgs, newBody));
}
@Override
public BindingAnalysis analyzeInputs()
{
final Set<String> lambdaArgs = args.stream().map(IdentifierExpr::toString).collect(Collectors.toSet());
BindingAnalysis bodyDetails = expr.analyzeInputs();
return bodyDetails.removeLambdaArguments(lambdaArgs);
}
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return expr.getOutputType(inspector);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
LambdaExpr that = (LambdaExpr) o;
return Objects.equals(args, that.args) &&
Objects.equals(expr, that.expr);
}
@Override
public int hashCode()
{
return Objects.hash(args, expr);
}
}
/**
* {@link Expr} node for a {@link Function} call. {@link FunctionExpr} has children {@link Expr} in the form of the
* list of arguments that are passed to the {@link Function} along with the {@link Expr.ObjectBinding} when it is
@ -350,15 +226,24 @@ class ApplyFunctionExpr implements Expr
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return function.canVectorize(inspector, lambdaExpr, argsExpr) &&
lambdaExpr.canVectorize(inspector) &&
argsExpr.stream().allMatch(expr -> expr.canVectorize(inspector));
return canVectorizeNative(inspector) || (getOutputType(inspector) != null && inspector.canVectorize(argsExpr));
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
if (canVectorizeNative(inspector)) {
return function.asVectorProcessor(inspector, lambdaExpr, argsExpr);
} else {
return FallbackVectorProcessor.create(function, lambdaExpr, argsExpr, inspector);
}
}
private boolean canVectorizeNative(InputBindingInspector inspector)
{
return function.canVectorize(inspector, lambdaExpr, argsExpr) &&
lambdaExpr.canVectorize(inspector) &&
argsExpr.stream().allMatch(expr -> expr.canVectorize(inspector));
}
@Override

View File

@ -0,0 +1,151 @@
/*
* 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.math.expr;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.math.expr.vector.ExprVectorProcessor;
import javax.annotation.Nullable;
import java.util.List;
import java.util.Objects;
import java.util.Set;
import java.util.stream.Collectors;
public class LambdaExpr implements Expr
{
private final ImmutableList<IdentifierExpr> args;
private final Expr expr;
LambdaExpr(List<IdentifierExpr> args, Expr expr)
{
this.args = ImmutableList.copyOf(args);
this.expr = expr;
}
@Override
public String toString()
{
return StringUtils.format("(%s -> %s)", args, expr);
}
int identifierCount()
{
return args.size();
}
@Nullable
public String getIdentifier()
{
Preconditions.checkState(args.size() < 2, "LambdaExpr has multiple arguments, use getIdentifiers");
if (args.size() == 1) {
return args.get(0).toString();
}
return null;
}
public List<String> getIdentifiers()
{
return args.stream().map(IdentifierExpr::toString).collect(Collectors.toList());
}
public List<String> stringifyIdentifiers()
{
return args.stream().map(IdentifierExpr::stringify).collect(Collectors.toList());
}
ImmutableList<IdentifierExpr> getIdentifierExprs()
{
return args;
}
public Expr getExpr()
{
return expr;
}
@Override
public boolean canVectorize(InputBindingInspector inspector)
{
return expr.canVectorize(inspector);
}
@Override
public <T> ExprVectorProcessor<T> asVectorProcessor(VectorInputBindingInspector inspector)
{
return expr.asVectorProcessor(inspector);
}
@Override
public ExprEval eval(ObjectBinding bindings)
{
return expr.eval(bindings);
}
@Override
public String stringify()
{
return StringUtils.format("(%s) -> %s", ARG_JOINER.join(stringifyIdentifiers()), expr.stringify());
}
@Override
public Expr visit(Shuttle shuttle)
{
List<IdentifierExpr> newArgs =
args.stream().map(arg -> (IdentifierExpr) shuttle.visit(arg)).collect(Collectors.toList());
Expr newBody = expr.visit(shuttle);
return shuttle.visit(new LambdaExpr(newArgs, newBody));
}
@Override
public BindingAnalysis analyzeInputs()
{
final Set<String> lambdaArgs = args.stream().map(IdentifierExpr::toString).collect(Collectors.toSet());
BindingAnalysis bodyDetails = expr.analyzeInputs();
return bodyDetails.removeLambdaArguments(lambdaArgs);
}
@Override
public ExpressionType getOutputType(InputBindingInspector inspector)
{
return expr.getOutputType(inspector);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
LambdaExpr that = (LambdaExpr) o;
return Objects.equals(args, that.args) &&
Objects.equals(expr, that.expr);
}
@Override
public int hashCode()
{
return Objects.hash(args, expr);
}
}

View File

@ -19,32 +19,43 @@
package org.apache.druid.math.expr.vector;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExpressionType;
public final class CastToStringVectorProcessor extends CastToTypeVectorProcessor<Object[]>
public class CastToObjectVectorProcessor extends CastToTypeVectorProcessor<Object[]>
{
public CastToStringVectorProcessor(ExprVectorProcessor<?> delegate)
private final ExpressionType outputType;
private final ExpressionType delegateType;
private final Object[] output;
public CastToObjectVectorProcessor(
ExprVectorProcessor<?> delegate,
ExpressionType outputType,
int maxVectorSize
)
{
super(delegate);
this.delegateType = delegate.getOutputType();
this.outputType = outputType;
this.output = new Object[maxVectorSize];
}
@Override
public ExprEvalVector<Object[]> evalVector(Expr.VectorInputBinding bindings)
{
ExprEvalVector<?> result = delegate.evalVector(bindings);
final Object[] objects = result.getObjectVector();
final Object[] output = new String[objects.length];
for (int i = 0; i < objects.length; i++) {
output[i] = Evals.asString(objects[i]);
final ExprEvalVector<?> delegateOutput = delegate.evalVector(bindings);
final Object[] toCast = delegateOutput.getObjectVector();
for (int i = 0; i < bindings.getCurrentVectorSize(); i++) {
ExprEval<?> cast = ExprEval.ofType(delegateType, toCast[i]).castTo(outputType);
output[i] = cast.value();
}
return new ExprEvalObjectVector(output, ExpressionType.STRING);
return new ExprEvalObjectVector(output, outputType);
}
@Override
public ExpressionType getOutputType()
{
return ExpressionType.STRING;
return outputType;
}
}

View File

@ -20,7 +20,6 @@
package org.apache.druid.math.expr.vector;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.math.expr.Exprs;
public abstract class CastToTypeVectorProcessor<TOutput> implements ExprVectorProcessor<TOutput>
{
@ -31,24 +30,26 @@ public abstract class CastToTypeVectorProcessor<TOutput> implements ExprVectorPr
this.delegate = delegate;
}
public static <T> ExprVectorProcessor<T> cast(ExprVectorProcessor<?> delegate, ExpressionType type)
public static <T> ExprVectorProcessor<T> cast(
ExprVectorProcessor<?> castInput,
ExpressionType castToType,
int maxVectorSize
)
{
final ExprVectorProcessor<?> caster;
if (delegate.getOutputType().equals(type)) {
caster = delegate;
final ExpressionType inputType = castInput.getOutputType();
if (inputType.equals(castToType)) {
caster = castInput;
} else {
switch (type.getType()) {
case STRING:
caster = new CastToStringVectorProcessor(delegate);
break;
switch (castToType.getType()) {
case LONG:
caster = new CastToLongVectorProcessor(delegate);
caster = new CastToLongVectorProcessor(castInput);
break;
case DOUBLE:
caster = new CastToDoubleVectorProcessor(delegate);
caster = new CastToDoubleVectorProcessor(castInput);
break;
default:
throw Exprs.cannotVectorize();
caster = new CastToObjectVectorProcessor(castInput, castToType, maxVectorSize);
}
}
return (ExprVectorProcessor<T>) caster;

View File

@ -29,7 +29,7 @@ public abstract class DoubleOutDoubleInFunctionVectorValueProcessor
{
public DoubleOutDoubleInFunctionVectorValueProcessor(ExprVectorProcessor<double[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE, maxVectorSize), maxVectorSize);
}
public abstract double apply(double input);

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutDoubleLongInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutDoublesInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutLongDoubleInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -29,7 +29,7 @@ public abstract class DoubleOutLongInFunctionVectorValueProcessor
{
public DoubleOutLongInFunctionVectorValueProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG, maxVectorSize), maxVectorSize);
}
public abstract double apply(long input);

View File

@ -34,8 +34,8 @@ public abstract class DoubleOutLongsInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -72,7 +72,7 @@ public final class ExprEvalObjectVector extends ExprEvalVector<Object[]>
}
} else {
// ARRAY, COMPLEX
final ExprEval<?> valueEval = ExprEval.bestEffortOf(values[i]);
final ExprEval<?> valueEval = ExprEval.ofType(type, values[i]).castTo(ExpressionType.DOUBLE);
longs[i] = valueEval.asLong();
doubles[i] = valueEval.asDouble();
numericNulls[i] = valueEval.isNumericNull();

View File

@ -20,12 +20,14 @@
package org.apache.druid.math.expr.vector;
import org.apache.druid.error.DruidException;
import org.apache.druid.math.expr.ApplyFunction;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprEval;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.math.expr.ExprType;
import org.apache.druid.math.expr.ExpressionType;
import org.apache.druid.math.expr.Function;
import org.apache.druid.math.expr.LambdaExpr;
import javax.annotation.Nullable;
import java.util.ArrayList;
@ -73,6 +75,25 @@ public abstract class FallbackVectorProcessor<T> implements ExprVectorProcessor<
);
}
/**
* Create a processor for a non-vectorizable {@link ApplyFunction}.
*/
public static <T> FallbackVectorProcessor<T> create(
final ApplyFunction function,
final LambdaExpr lambdaExpr,
final List<Expr> args,
final Expr.VectorInputBindingInspector inspector
)
{
final List<Expr> adaptedArgs = makeAdaptedArgs(args, inspector);
return makeFallbackProcessor(
() -> function.apply(lambdaExpr, adaptedArgs, UnusedBinding.INSTANCE),
adaptedArgs,
function.getOutputType(inspector, lambdaExpr, args),
inspector
);
}
/**
* Create a processor for a non-vectorizable {@link ExprMacroTable.ExprMacro}.
*/

View File

@ -29,7 +29,7 @@ public abstract class LongOutDoubleInFunctionVectorValueProcessor
{
public LongOutDoubleInFunctionVectorValueProcessor(ExprVectorProcessor<double[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.DOUBLE, maxVectorSize), maxVectorSize);
}
public abstract long apply(double input);

View File

@ -34,8 +34,8 @@ public abstract class LongOutDoubleLongInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class LongOutDoublesInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.DOUBLE, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -34,8 +34,8 @@ public abstract class LongOutLongDoubleInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.DOUBLE, maxVectorSize),
maxVectorSize
);
}

View File

@ -29,7 +29,7 @@ public abstract class LongOutLongInFunctionVectorValueProcessor
{
public LongOutLongInFunctionVectorValueProcessor(ExprVectorProcessor<long[]> processor, int maxVectorSize)
{
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG), maxVectorSize);
super(CastToTypeVectorProcessor.cast(processor, ExpressionType.LONG, maxVectorSize), maxVectorSize);
}
public abstract long apply(long input);

View File

@ -34,8 +34,8 @@ public abstract class LongOutLongsInFunctionVectorValueProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG),
CastToTypeVectorProcessor.cast(left, ExpressionType.LONG, maxVectorSize),
CastToTypeVectorProcessor.cast(right, ExpressionType.LONG, maxVectorSize),
maxVectorSize
);
}

View File

@ -33,7 +33,7 @@ public abstract class LongOutObjectInFunctionVectorProcessor
ExpressionType inputType
)
{
super(CastToTypeVectorProcessor.cast(processor, inputType), maxVectorSize, new long[maxVectorSize]);
super(CastToTypeVectorProcessor.cast(processor, inputType, maxVectorSize), maxVectorSize, new long[maxVectorSize]);
}
@Override

View File

@ -37,8 +37,8 @@ public abstract class LongOutObjectsInFunctionVectorProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, inputType),
CastToTypeVectorProcessor.cast(right, inputType),
CastToTypeVectorProcessor.cast(left, inputType, maxVectorSize),
CastToTypeVectorProcessor.cast(right, inputType, maxVectorSize),
new long[maxVectorSize]
);
this.outNulls = new boolean[maxVectorSize];

View File

@ -36,8 +36,8 @@ public abstract class ObjectOutObjectsInFunctionVectorProcessor
)
{
super(
CastToTypeVectorProcessor.cast(left, expressionType),
CastToTypeVectorProcessor.cast(right, expressionType),
CastToTypeVectorProcessor.cast(left, expressionType, maxVectorSize),
CastToTypeVectorProcessor.cast(right, expressionType, maxVectorSize),
new Object[maxVectorSize]
);
this.expressionType = expressionType;

View File

@ -21,6 +21,7 @@ package org.apache.druid.math.expr.vector;
import com.google.common.base.Preconditions;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.error.DruidException;
import org.apache.druid.math.expr.Evals;
import org.apache.druid.math.expr.Expr;
import org.apache.druid.math.expr.ExprType;
@ -81,15 +82,19 @@ public class VectorProcessors
}
/**
* Creates an {@link ExprVectorProcessor} that creates a {@link ExprEvalVector} for a constant {@link String} value.
* Creates an {@link ExprVectorProcessor} that creates a {@link ExprEvalVector} for a constant any non-numeric value.
* Numeric types should use {@link #constant(Double, int)} or {@link #constant(Long, int)} instead.
*
* @see org.apache.druid.math.expr.ConstantExpr
*/
public static <T> ExprVectorProcessor<T> constant(@Nullable String constant, int maxVectorSize)
public static <T> ExprVectorProcessor<T> constant(@Nullable Object constant, int maxVectorSize, ExpressionType type)
{
final Object[] strings = new Object[maxVectorSize];
Arrays.fill(strings, constant);
final ExprEvalObjectVector eval = new ExprEvalObjectVector(strings, ExpressionType.STRING);
if (type.isNumeric()) {
throw DruidException.defensive("Type[%s] should use the numeric constant creator instead", type);
}
final Object[] objects = new Object[maxVectorSize];
Arrays.fill(objects, constant);
final ExprEvalObjectVector eval = new ExprEvalObjectVector(objects, type);
return new ExprVectorProcessor<T>()
{
@Override
@ -101,7 +106,7 @@ public class VectorProcessors
@Override
public ExpressionType getOutputType()
{
return ExpressionType.STRING;
return type;
}
};
}
@ -194,15 +199,11 @@ public class VectorProcessors
if (inputType == null) {
// nil column, we can be anything, so be a string because it's the most flexible
// (numbers will be populated with default values in default mode and non-null)
return new IdentifierVectorProcessor<Object[]>(ExpressionType.STRING)
{
@Override
public ExprEvalVector<Object[]> evalVector(Expr.VectorInputBinding bindings)
{
return new ExprEvalObjectVector(bindings.getObjectVector(binding), ExpressionType.STRING);
if (NullHandling.sqlCompatible()) {
return constant((Long) null, inspector.getMaxVectorSize());
} else {
return constant(null, inspector.getMaxVectorSize(), ExpressionType.STRING);
}
};
}
switch (inputType.getType()) {
case LONG:
@ -292,35 +293,8 @@ public class VectorProcessors
}
final long[] outputValues = new long[inspector.getMaxVectorSize()];
ExprVectorProcessor<?> processor = null;
if (Types.is(type, ExprType.STRING)) {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 1L;
} else {
outputValues[i] = 0L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
} else if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<?> processor;
if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<long[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@ -382,11 +356,35 @@ public class VectorProcessors
return ExpressionType.LONG;
}
};
} else {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 1L;
} else {
outputValues[i] = 0L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
if (processor == null) {
throw Exprs.cannotVectorize();
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
}
return (ExprVectorProcessor<T>) processor;
}
@ -406,35 +404,8 @@ public class VectorProcessors
final long[] outputValues = new long[inspector.getMaxVectorSize()];
ExprVectorProcessor<?> processor = null;
if (Types.is(type, ExprType.STRING)) {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 0L;
} else {
outputValues[i] = 1L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
} else if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<?> processor;
if (Types.is(type, ExprType.LONG)) {
final ExprVectorProcessor<long[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@ -496,11 +467,35 @@ public class VectorProcessors
return ExpressionType.LONG;
}
};
} else {
final ExprVectorProcessor<Object[]> input = expr.asVectorProcessor(inspector);
processor = new ExprVectorProcessor<long[]>()
{
@Override
public ExprEvalVector<long[]> evalVector(Expr.VectorInputBinding bindings)
{
final ExprEvalVector<Object[]> inputEval = input.evalVector(bindings);
final int currentSize = bindings.getCurrentVectorSize();
final Object[] values = inputEval.values();
for (int i = 0; i < currentSize; i++) {
if (values[i] == null) {
outputValues[i] = 0L;
} else {
outputValues[i] = 1L;
}
}
return new ExprEvalLongVector(outputValues, null);
}
if (processor == null) {
throw Exprs.cannotVectorize();
@Override
public ExpressionType getOutputType()
{
return ExpressionType.LONG;
}
};
}
return (ExprVectorProcessor<T>) processor;
}

View File

@ -74,7 +74,8 @@ public class VectorStringProcessors
for (int i = 0; i < inputs.size(); i++) {
inputProcessors[i] = CastToTypeVectorProcessor.cast(
inputs.get(i).asVectorProcessor(inspector),
ExpressionType.STRING
ExpressionType.STRING,
inspector.getMaxVectorSize()
);
}
final ExprVectorProcessor processor = new ObjectOutMultiObjectInVectorProcessor(

View File

@ -134,7 +134,7 @@ public class TimestampFloorExprMacro implements ExprMacroTable.ExprMacro
{
ExprVectorProcessor<?> processor;
processor = new LongOutLongInFunctionVectorValueProcessor(
CastToTypeVectorProcessor.cast(args.get(0).asVectorProcessor(inspector), ExpressionType.LONG),
CastToTypeVectorProcessor.cast(args.get(0).asVectorProcessor(inspector), ExpressionType.LONG, inspector.getMaxVectorSize()),
inspector.getMaxVectorSize()
)
{

View File

@ -117,7 +117,11 @@ public class TimestampShiftExprMacro implements ExprMacroTable.ExprMacro
{
ExprVectorProcessor<?> processor;
processor = new LongOutLongInFunctionVectorValueProcessor(
CastToTypeVectorProcessor.cast(args.get(0).asVectorProcessor(inspector), ExpressionType.LONG),
CastToTypeVectorProcessor.cast(
args.get(0).asVectorProcessor(inspector),
ExpressionType.LONG,
inspector.getMaxVectorSize()
),
inspector.getMaxVectorSize()
)
{

View File

@ -20,6 +20,7 @@
package org.apache.druid.query.rowsandcols;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.UOE;
import org.apache.druid.query.rowsandcols.column.Column;
import org.apache.druid.query.rowsandcols.column.ColumnAccessor;
import org.apache.druid.query.rowsandcols.column.ColumnAccessorBasedColumn;
@ -28,6 +29,7 @@ import org.apache.druid.segment.column.ColumnType;
import javax.annotation.Nullable;
import java.util.Collection;
import java.util.LinkedHashMap;
import java.util.List;
import java.util.Map;
/**
@ -128,7 +130,16 @@ public class RearrangedRowsAndColumns implements RowsAndColumns
@Override
public Object getObject(int rowNum)
{
return accessor.getObject(pointers[start + rowNum]);
Object value = accessor.getObject(pointers[start + rowNum]);
if (ColumnType.STRING.equals(getType()) && value instanceof List) {
// special handling to reject MVDs
throw new UOE(
"Encountered a multi value column [%s]. Window processing does not support MVDs. "
+ "Consider using UNNEST or MV_TO_ARRAY.",
name
);
}
return value;
}
@Override

View File

@ -182,16 +182,12 @@ public class ExpressionPlanner
traits.remove(ExpressionPlan.Trait.SINGLE_INPUT_MAPPABLE);
}
// vectorized expressions do not support incomplete, multi-valued inputs or outputs, or implicit mapping
// they also do not support unknown inputs, but they also do not currently have to deal with them, as missing
// capabilites is indicative of a non-existent column instead of an unknown schema. If this ever changes,
// vectorized expressions do not support unknown inputs, but they also do not currently have to deal with them, as
// missing capabilites is indicative of a non-existent column instead of an unknown schema. If this ever changes,
// this check should also change
boolean supportsVector = ExpressionPlan.none(
traits,
ExpressionPlan.Trait.INCOMPLETE_INPUTS,
ExpressionPlan.Trait.NEEDS_APPLIED,
ExpressionPlan.Trait.NON_SCALAR_INPUTS,
ExpressionPlan.Trait.NON_SCALAR_OUTPUT
ExpressionPlan.Trait.INCOMPLETE_INPUTS
);
if (supportsVector && expression.canVectorize(inspector)) {

View File

@ -170,7 +170,8 @@ public class ExpressionVectorSelectors
return new ExpressionVectorObjectSelector(
CastToTypeVectorProcessor.cast(
VectorProcessors.identifier(binding, columnName),
ExpressionType.fromColumnType(castTo)
ExpressionType.fromColumnType(castTo),
binding.getMaxVectorSize()
),
binding
);
@ -190,7 +191,8 @@ public class ExpressionVectorSelectors
return new ExpressionVectorValueSelector(
CastToTypeVectorProcessor.cast(
VectorProcessors.identifier(binding, columnName),
ExpressionType.fromColumnType(castTo)
ExpressionType.fromColumnType(castTo),
binding.getMaxVectorSize()
),
binding
);

View File

@ -22,13 +22,14 @@ package org.apache.druid.utils;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.metadata.DynamicConfigProvider;
import javax.annotation.Nullable;
import java.util.Collections;
import java.util.HashMap;
import java.util.Map;
public class DynamicConfigProviderUtils
{
public static Map<String, String> extraConfigAndSetStringMap(Map<String, Object> config, String dynamicConfigProviderKey, ObjectMapper mapper)
public static Map<String, String> extraConfigAndSetStringMap(@Nullable Map<String, Object> config, String dynamicConfigProviderKey, ObjectMapper mapper)
{
HashMap<String, String> newConfig = new HashMap<>();
if (config != null) {
@ -43,7 +44,7 @@ public class DynamicConfigProviderUtils
return newConfig;
}
public static Map<String, Object> extraConfigAndSetObjectMap(Map<String, Object> config, String dynamicConfigProviderKey, ObjectMapper mapper)
public static Map<String, Object> extraConfigAndSetObjectMap(@Nullable Map<String, Object> config, String dynamicConfigProviderKey, ObjectMapper mapper)
{
HashMap<String, Object> newConfig = new HashMap<>();
if (config != null) {
@ -58,7 +59,7 @@ public class DynamicConfigProviderUtils
return newConfig;
}
private static Map<String, String> extraConfigFromProvider(Object dynamicConfigProviderJson, ObjectMapper mapper)
private static Map<String, String> extraConfigFromProvider(@Nullable Object dynamicConfigProviderJson, ObjectMapper mapper)
{
if (dynamicConfigProviderJson != null) {
DynamicConfigProvider dynamicConfigProvider = mapper.convertValue(dynamicConfigProviderJson, DynamicConfigProvider.class);

Some files were not shown because too many files have changed in this diff Show More