mirror of
https://github.com/apache/druid.git
synced 2025-02-28 14:29:17 +00:00
Merge remote-tracking branch 'apache/master' into quidem-kttm
This commit is contained in:
commit
528765c9b2
@ -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
|
||||
);
|
||||
|
@ -933,7 +933,6 @@ Host: http://ROUTER_IP:ROUTER_PORT
|
||||
"context": {
|
||||
"__resultFormat": "array",
|
||||
"__user": "allowAll",
|
||||
"enableWindowing": true,
|
||||
"executionMode": "async",
|
||||
"finalize": true,
|
||||
"maxNumTasks": 2,
|
||||
|
@ -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.|
|
||||
|
@ -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
|
||||
|
@ -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:
|
||||
|
||||
|
@ -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.
|
||||
|
@ -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>
|
||||
|
||||
|
@ -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)
|
||||
|
||||
)
|
||||
|
@ -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();
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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();
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
||||
}
|
@ -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.
|
||||
|
@ -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;
|
||||
|
@ -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(
|
||||
|
@ -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"
|
||||
|
@ -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);
|
||||
}
|
||||
|
||||
|
@ -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>
|
||||
|
@ -0,0 +1,3 @@
|
||||
{
|
||||
"dependsOnDruidExtensions": ["druid-lookups-cached-global"]
|
||||
}
|
@ -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);
|
||||
|
||||
|
@ -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)
|
||||
);
|
||||
}
|
||||
|
||||
|
@ -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
|
||||
);
|
||||
|
||||
/**
|
||||
|
@ -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(
|
||||
|
@ -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++;
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -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
|
||||
}
|
@ -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
|
||||
}
|
||||
}
|
||||
|
@ -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(
|
||||
|
@ -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.
|
||||
|
@ -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
|
||||
|
@ -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)
|
||||
|
File diff suppressed because it is too large
Load Diff
@ -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());
|
||||
}
|
||||
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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(
|
||||
|
@ -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()
|
||||
);
|
||||
|
@ -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;
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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());
|
||||
|
@ -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()
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
@ -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();
|
||||
}
|
||||
}
|
||||
|
@ -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) {
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
);
|
||||
|
@ -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());
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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());
|
||||
|
||||
|
@ -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;
|
||||
|
||||
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
|
@ -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"));
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
@ -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()
|
||||
);
|
||||
|
||||
|
@ -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";
|
||||
}
|
||||
}
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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";
|
||||
}
|
||||
}
|
@ -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";
|
||||
|
@ -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)
|
||||
{
|
||||
|
@ -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)
|
||||
|
@ -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 {
|
||||
|
@ -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");
|
||||
}
|
||||
}
|
@ -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
|
||||
|
@ -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
|
||||
|
@ -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()
|
||||
{
|
||||
|
@ -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));
|
||||
}
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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);
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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();
|
||||
|
@ -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}.
|
||||
*/
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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);
|
||||
|
@ -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
|
||||
);
|
||||
}
|
||||
|
@ -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
|
||||
|
@ -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];
|
||||
|
@ -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;
|
||||
|
@ -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;
|
||||
}
|
||||
|
||||
|
@ -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(
|
||||
|
@ -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()
|
||||
)
|
||||
{
|
||||
|
@ -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()
|
||||
)
|
||||
{
|
||||
|
@ -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
|
||||
|
@ -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)) {
|
||||
|
@ -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
|
||||
);
|
||||
|
@ -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
Loading…
x
Reference in New Issue
Block a user