diff --git a/docs/configuration/extensions.md b/docs/configuration/extensions.md
index 575ad25d6551f..29356ebc05cc2 100644
--- a/docs/configuration/extensions.md
+++ b/docs/configuration/extensions.md
@@ -44,11 +44,11 @@ Core extensions are maintained by Druid committers.
|druid-google-extensions|Google Cloud Storage deep storage.|[link](../development/extensions-core/google.md)|
|druid-hdfs-storage|HDFS deep storage.|[link](../development/extensions-core/hdfs.md)|
|druid-histogram|Approximate histograms and quantiles aggregator. Deprecated, please use the [DataSketches quantiles aggregator](../development/extensions-core/datasketches-quantiles.md) from the `druid-datasketches` extension instead.|[link](../development/extensions-core/approximate-histograms.md)|
-|druid-kafka-extraction-namespace|Apache Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../development/extensions-core/kafka-extraction-namespace.md)|
+|druid-kafka-extraction-namespace|Apache Kafka-based namespaced lookup. Requires namespace lookup extension.|[link](../querying/kafka-extraction-namespace.md)|
|druid-kafka-indexing-service|Supervised exactly-once Apache Kafka ingestion for the indexing service.|[link](../ingestion/kafka-ingestion.md)|
|druid-kinesis-indexing-service|Supervised exactly-once Kinesis ingestion for the indexing service.|[link](../ingestion/kinesis-ingestion.md)|
|druid-kerberos|Kerberos authentication for druid processes.|[link](../development/extensions-core/druid-kerberos.md)|
-|druid-lookups-cached-global|A module for [lookups](../querying/lookups.md) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../development/extensions-core/lookups-cached-global.md)|
+|druid-lookups-cached-global|A module for [lookups](../querying/lookups.md) providing a jvm-global eager caching for lookups. It provides JDBC and URI implementations for fetching lookup data.|[link](../querying/lookups-cached-global.md)|
|druid-lookups-cached-single| Per lookup caching module to support the use cases where a lookup need to be isolated from the global pool of lookups |[link](../development/extensions-core/druid-lookups.md)|
|druid-multi-stage-query| Support for the multi-stage query architecture for Apache Druid and the multi-stage query task engine.|[link](../multi-stage-query/index.md)|
|druid-orc-extensions|Support for data in Apache ORC data format.|[link](../development/extensions-core/orc.md)|
diff --git a/docs/configuration/index.md b/docs/configuration/index.md
index c618e196dbcad..f8583b958411e 100644
--- a/docs/configuration/index.md
+++ b/docs/configuration/index.md
@@ -394,6 +394,7 @@ Metric monitoring is an essential part of Druid operations. The following monito
|`org.apache.druid.java.util.metrics.JvmThreadsMonitor`|Reports Thread statistics in the JVM, like numbers of total, daemon, started, died threads.|
|`org.apache.druid.java.util.metrics.CgroupCpuMonitor`|Reports CPU shares and quotas as per the `cpu` cgroup.|
|`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.server.metrics.EventReceiverFirehoseMonitor`|Reports how many events have been queued in the EventReceiverFirehose.|
|`org.apache.druid.server.metrics.HistoricalMetricsMonitor`|Reports statistics on Historical services. Available only on Historical services.|
@@ -626,7 +627,7 @@ the [HTTP input source](../ingestion/input-sources.md#http-input-source).
You can use the following properties to specify permissible JDBC options for:
- [SQL input source](../ingestion/input-sources.md#sql-input-source)
-- [globally cached JDBC lookups](../development/extensions-core/lookups-cached-global.md#jdbc-lookup)
+- [globally cached JDBC lookups](../querying/lookups-cached-global.md#jdbc-lookup)
- [JDBC Data Fetcher for per-lookup caching](../development/extensions-core/druid-lookups.md#data-fetcher-layer).
These properties do not apply to metadata storage connections.
@@ -1125,6 +1126,7 @@ These Overlord static configurations can be defined in the `overlord/runtime.pro
|`druid.indexer.queue.startDelay`|Sleep this long before starting Overlord queue management. This can be useful to give a cluster time to re-orient itself (for example, after a widespread network issue).|`PT1M`|
|`druid.indexer.queue.restartDelay`|Sleep this long when Overlord queue management throws an exception before trying again.|`PT30S`|
|`druid.indexer.queue.storageSyncRate`|Sync Overlord state this often with an underlying task persistence mechanism.|`PT1M`|
+|`druid.indexer.queue.maxTaskPayloadSize`|Maximum allowed size in bytes of a single task payload accepted by the Overlord.|none (allow all task payload sizes)|
The following configs only apply if the Overlord is running in remote mode. For a description of local vs. remote mode, see [Overlord service](../design/overlord.md).
diff --git a/docs/ingestion/flatten-json.md b/docs/ingestion/flatten-json.md
new file mode 100644
index 0000000000000..2ac2f0230714d
--- /dev/null
+++ b/docs/ingestion/flatten-json.md
@@ -0,0 +1,28 @@
+---
+id: flatten-json
+title: "Redirecting"
+---
+
+
+
+
+
+
+Click here if you are not redirected.
\ No newline at end of file
diff --git a/docs/ingestion/ingestion-spec.md b/docs/ingestion/ingestion-spec.md
index 6d1f9609d856d..0d9a82d6b3c42 100644
--- a/docs/ingestion/ingestion-spec.md
+++ b/docs/ingestion/ingestion-spec.md
@@ -471,14 +471,27 @@ An example `ioConfig` to read JSON data is:
...
}
```
-For more details, see the documentation provided by each [ingestion method](./index.md#ingestion-methods).
+
+For details, see the documentation provided by each [ingestion method](./index.md#ingestion-methods).
## `tuningConfig`
-Tuning properties are specified in a `tuningConfig`, which goes at the top level of an ingestion spec. Some
-properties apply to all [ingestion methods](./index.md#ingestion-methods), but most are specific to each individual
-ingestion method. An example `tuningConfig` that sets all of the shared, common properties to their defaults
-is:
+You specify tuning properties in a `tuningConfig` object, which goes at the top level of an ingestion spec.
+Some properties apply to all [ingestion methods](./index.md#ingestion-methods), but most are specific to each individual ingestion method.
+
+The following table lists the common tuning properties shared among ingestion methods:
+
+|Field|Description|Default|
+|-----|-----------|-------|
+|type|Each ingestion method has its own tuning type code. You must specify the type code that matches your ingestion method. Common options are `index`, `hadoop`, `kafka`, and `kinesis`.||
+|maxRowsInMemory|The maximum number of records to store in memory before persisting to disk. Note that this is the number of rows post-rollup, and so it may not be equal to the number of input records. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).|`1000000`|
+|maxBytesInMemory|The maximum aggregate size of records, in bytes, to store in the JVM heap before persisting. This is based on a rough estimate of memory usage. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first). `maxBytesInMemory` also includes heap usage of artifacts created from intermediary persists. This means that after every persist, the amount of `maxBytesInMemory` until the next persist will decrease. If the sum of bytes of all intermediary persisted artifacts exceeds `maxBytesInMemory` the task fails. Setting `maxBytesInMemory` to -1 disables this check, meaning Druid will rely entirely on `maxRowsInMemory` to control memory usage. Setting it to zero means the default value will be used (one-sixth of JVM heap size). Note that the estimate of memory usage is designed to be an overestimate, and can be especially high when using complex ingest-time aggregators, including sketches. If this causes your indexing workloads to persist to disk too often, you can set `maxBytesInMemory` to -1 and rely on `maxRowsInMemory` instead.|One-sixth of max JVM heap size|
+|skipBytesInMemoryOverheadCheck|The calculation of maxBytesInMemory takes into account overhead objects created during ingestion and each intermediate persist. Setting this to true can exclude the bytes of these overhead objects from maxBytesInMemory check.|false|
+|indexSpec|Defines segment storage format options to use at indexing time.|See [`indexSpec`](#indexspec) for more information.|
+|indexSpecForIntermediatePersists|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments.|See [`indexSpec`](#indexspec) for more information.|
+|Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../ingestion/kafka-ingestion.md#tuning-configuration), [Kinesis indexing service](../ingestion/kinesis-ingestion.md#tuning-configuration), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).||
+
+The following example shows a `tuningConfig` object that sets all of the shared common properties to their defaults:
```plaintext
"tuningConfig": {
@@ -495,38 +508,57 @@ is:
}
```
-|Field|Description|Default|
-|-----|-----------|-------|
-|type|Each ingestion method has its own tuning type code. You must specify the type code that matches your ingestion method. Common options are `index`, `hadoop`, `kafka`, and `kinesis`.||
-|maxRowsInMemory|The maximum number of records to store in memory before persisting to disk. Note that this is the number of rows post-rollup, and so it may not be equal to the number of input records. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first).|`1000000`|
-|maxBytesInMemory|The maximum aggregate size of records, in bytes, to store in the JVM heap before persisting. This is based on a rough estimate of memory usage. Ingested records will be persisted to disk when either `maxRowsInMemory` or `maxBytesInMemory` are reached (whichever happens first). `maxBytesInMemory` also includes heap usage of artifacts created from intermediary persists. This means that after every persist, the amount of `maxBytesInMemory` until the next persist will decrease. If the sum of bytes of all intermediary persisted artifacts exceeds `maxBytesInMemory` the task fails. Setting `maxBytesInMemory` to -1 disables this check, meaning Druid will rely entirely on `maxRowsInMemory` to control memory usage. Setting it to zero means the default value will be used (one-sixth of JVM heap size). Note that the estimate of memory usage is designed to be an overestimate, and can be especially high when using complex ingest-time aggregators, including sketches. If this causes your indexing workloads to persist to disk too often, you can set `maxBytesInMemory` to -1 and rely on `maxRowsInMemory` instead.|One-sixth of max JVM heap size|
-|skipBytesInMemoryOverheadCheck|The calculation of maxBytesInMemory takes into account overhead objects created during ingestion and each intermediate persist. Setting this to true can exclude the bytes of these overhead objects from maxBytesInMemory check.|false|
-|indexSpec|Defines segment storage format options to use at indexing time.|See [`indexSpec`](#indexspec) for more information.|
-|indexSpecForIntermediatePersists|Defines segment storage format options to use at indexing time for intermediate persisted temporary segments.|See [`indexSpec`](#indexspec) for more information.|
-|Other properties|Each ingestion method has its own list of additional tuning properties. See the documentation for each method for a full list: [Kafka indexing service](../ingestion/kafka-ingestion.md#tuning-configuration), [Kinesis indexing service](../ingestion/kinesis-ingestion.md#tuning-configuration), [Native batch](native-batch.md#tuningconfig), and [Hadoop-based](hadoop.md#tuningconfig).||
-
### `indexSpec`
-The `indexSpec` object can include the following properties:
+The `indexSpec` object can include the following properties.
+For information on defining an `indexSpec` in a query context, see [SQL-based ingestion reference](../multi-stage-query/reference.md#context-parameters).
|Field|Description|Default|
|-----|-----------|-------|
|bitmap|Compression format for bitmap indexes. Should be a JSON object with `type` set to `roaring` or `concise`.|`{"type": "roaring"}`|
-|dimensionCompression|Compression format for dimension columns. Options are `lz4`, `lzf`, `zstd`, or `uncompressed`.|`lz4`|
-|stringDictionaryEncoding|Encoding format for STRING value dictionaries used by STRING and COMPLEX<json> columns. Example to enable front coding: `{"type":"frontCoded", "bucketSize": 4}` `bucketSize` is the number of values to place in a bucket to perform delta encoding. Must be a power of 2, maximum is 128. Defaults to 4. `formatVersion` can specify older versions for backwards compatibility during rolling upgrades, valid options are `0` and `1`. Defaults to `0` for backwards compatibility. See [Front coding](#front-coding) for more information.|`{"type":"utf8"}`|
+|dimensionCompression|Compression format for dimension columns. One of `lz4`, `lzf`, `zstd`, or `uncompressed`.|`lz4`|
+|stringDictionaryEncoding|Encoding format for string value dictionaries used by STRING and [COMPLEX<json>](../querying/nested-columns.md) columns. To enable front coding, set `stringDictionaryEncoding.type` to `frontCoded`. Optionally, you can specify the `bucketSize` and `formatVersion` properties. See [Front coding](#front-coding) for more information.|`{"type":"utf8"}`|
|metricCompression|Compression format for primitive type metric columns. Options are `lz4`, `lzf`, `zstd`, `uncompressed`, or `none` (which is more efficient than `uncompressed`, but not supported by older versions of Druid).|`lz4`|
|longEncoding|Encoding format for long-typed columns. Applies regardless of whether they are dimensions or metrics. Options are `auto` or `longs`. `auto` encodes the values using offset or lookup table depending on column cardinality, and store them with variable size. `longs` stores the value as-is with 8 bytes each.|`longs`|
|jsonCompression|Compression format to use for nested column raw data. Options are `lz4`, `lzf`, `zstd`, or `uncompressed`.|`lz4`|
-##### Front coding
-
-Front coding is an experimental feature starting in version 25.0. Front coding is an incremental encoding strategy that Druid can use to store STRING and [COMPLEX<json>](../querying/nested-columns.md) columns. It allows Druid to create smaller UTF-8 encoded segments with very little performance cost.
-
-You can enable front coding with all types of ingestion. For information on defining an `indexSpec` in a query context, see [SQL-based ingestion reference](../multi-stage-query/reference.md#context-parameters).
+#### Front coding
:::info
- Front coding was originally introduced in Druid 25.0, and an improved 'version 1' was introduced in Druid 26.0, with typically faster read speed and smaller storage size. The current recommendation is to enable it in a staging environment and fully test your use case before using in production. By default, segments created with front coding enabled in Druid 26.0 are backwards compatible with Druid 25.0, but those created with Druid 26.0 or 25.0 are not compatible with Druid versions older than 25.0. If using front coding in Druid 25.0 and upgrading to Druid 26.0, the `formatVersion` defaults to `0` to keep writing out the older format to enable seamless downgrades to Druid 25.0, and then later is recommended to be changed to `1` once determined that rollback is not necessary.
+Front coding is an [experimental feature](../development/experimental.md).
:::
-Beyond these properties, each ingestion method has its own specific tuning properties. See the documentation for each
-[ingestion method](./index.md#ingestion-methods) for details.
+Druid encodes string columns into dictionaries for better compression.
+Front coding is an incremental encoding strategy that lets you store STRING and [COMPLEX<json>](../querying/nested-columns.md) columns in Druid with minimal performance impact.
+Front-coded dictionaries reduce storage and improve performance by optimizing for strings where the front part looks similar.
+For example, if you are tracking website visits, most URLs start with `https://domain.xyz/`, and front coding is able to exploit this pattern for more optimal compression when storing such datasets.
+Druid performs the optimization automatically, which means that the performance of string columns is generally not affected when they don't match the front-coded pattern.
+Consequently, you can enable this feature universally without having to know the underlying data shapes of the columns.
+
+You can use front coding with all types of ingestion.
+
+##### Enable front coding
+
+Before you enable front coding for your cluster, review the [Migration guide for front-coded dictionaries](../release-info/migr-front-coded-dict.md).
+It contains important information about compatibility with Druid versions preceding 25.0.0.
+
+To enable front coding, set `indexSpec.stringDictionaryEncoding.type` to `frontCoded` in the `tuningConfig` object of your [ingestion spec](../ingestion/ingestion-spec.md).
+
+You can specify the following optional properties:
+
+* `bucketSize`: Number of values to place in a bucket to perform delta encoding. Setting this property instructs indexing tasks to write segments using compressed dictionaries of the specified bucket size. You can set it to any power of 2 less than or equal to 128. `bucketSize` defaults to 4.
+* `formatVersion`: Specifies which front coding version to use. Options are 0 and 1 (supported for Druid versions 26.0.0 and higher). `formatVersion` defaults to 0. For faster speeds and smaller storage sizes, set `formatVersion` to 1. After setting `formatVersion` to 1, you can no longer downgrade to Druid 25.0.0 seamlessly. To downgrade to Druid 25.0.0, you must re-ingest your data with the `formatVersion` property set to 0.
+
+For example:
+
+```
+"tuningConfig": {
+ "indexSpec": {
+ "stringDictionaryEncoding": {
+ "type":"frontCoded",
+ "bucketSize": 4,
+ "formatVersion": 0
+ }
+ }
+}
+```
\ No newline at end of file
diff --git a/docs/ingestion/input-sources.md b/docs/ingestion/input-sources.md
index f01bb26096cd2..f89693740c85b 100644
--- a/docs/ingestion/input-sources.md
+++ b/docs/ingestion/input-sources.md
@@ -300,19 +300,16 @@ Google Cloud Storage object:
|path|The path where data is located.|None|yes|
|systemFields|JSON array of system fields to return as part of input rows. Possible values: `__file_uri` (Google Cloud Storage URI starting with `gs://`), `__file_bucket` (GCS bucket), and `__file_path` (GCS key).|None|no|
-## Azure input source
+## Azure input source
:::info
You need to include the [`druid-azure-extensions`](../development/extensions-core/azure.md) as an extension to use the Azure input source.
:::
-The Azure input source reads objects directly from Azure Blob store or Azure Data Lake sources. You can
+The Azure input source (that uses the type `azureStorage`) reads objects directly from Azure Blob store or Azure Data Lake sources. You can
specify objects as a list of file URI strings or prefixes. You can split the Azure input source for use with [Parallel task](./native-batch.md) indexing and each worker task reads one chunk of the split data.
-
-:::info
-The old `azure` schema is deprecated. Update your specs to use the `azureStorage` schema described below instead.
-:::
+The `azureStorage` input source is a new schema for Azure input sources that allows you to specify which storage account files should be ingested from. We recommend that you update any specs that use the old `azure` schema to use the new `azureStorage` schema. The new schema provides more functionality than the older `azure` schema.
Sample specs:
@@ -410,10 +407,10 @@ The `properties` property can be one of the following:
|appRegistrationClientSecret|The client secret of the Azure App registration to authenticate as|None|Yes if `appRegistrationClientId` is provided|
|tenantId|The tenant ID of the Azure App registration to authenticate as|None|Yes if `appRegistrationClientId` is provided|
-
- Show the deprecated 'azure' input source
-Note that the deprecated `azure` input source doesn't support specifying which storage account to ingest from. We recommend using the `azureStorage` instead.
+#### `azure` input source
+
+The Azure input source that uses the type `azure` is an older version of the Azure input type and is not recommended. It doesn't support specifying which storage account to ingest from. We recommend using the [`azureStorage` input source schema](#azure-input-source) instead since it provides more functionality.
Sample specs:
@@ -490,7 +487,6 @@ The `objects` property is:
|bucket|Name of the Azure Blob Storage or Azure Data Lake container|None|yes|
|path|The path where data is located.|None|yes|
-
## HDFS input source
@@ -1065,20 +1061,22 @@ The catalog object supports `local` and `hive` catalog types.
The following table lists the properties of a `local` catalog:
-|Property|Description|Required|
-|--------|-----------|---------|
-|type|Set this value to `local`.|yes|
-|warehousePath|The location of the warehouse associated with the catalog|yes|
-|catalogProperties|Map of any additional properties that needs to be attached to the catalog|no|
+|Property|Description|Default|Required|
+|--------|-----------|-------|---------|
+|type|Set this value to `local`.|None|yes|
+|warehousePath|The location of the warehouse associated with the catalog.|None|yes|
+|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 `hive` catalog:
-|Property|Description|Required|
-|--------|-----------|---------|
-|type|Set this value to `hive`.|yes|
-|warehousePath|The location of the warehouse associated with the catalog|yes|
-|catalogUri|The URI associated with the hive catalog|yes|
-|catalogProperties|Map of any additional properties that needs to be attached to the catalog|no|
+|Property|Description|Default|Required|
+|--------|-----------|-------|---------|
+|type|Set this value to `hive`.|None|yes|
+|warehousePath|The location of the warehouse associated with the catalog.|None|yes|
+|catalogUri|The URI associated with the hive catalog.|None|yes|
+|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|
### Iceberg filter object
diff --git a/docs/operations/metrics.md b/docs/operations/metrics.md
index 51041952cfae1..bf241ac5708a8 100644
--- a/docs/operations/metrics.md
+++ b/docs/operations/metrics.md
@@ -516,8 +516,17 @@ These metrics are available on operating systems with the cgroup kernel feature.
|------|-----------|----------|------------|
|`cgroup/cpu/shares`|Relative value of CPU time available to this process. Read from `cpu.shares`.||Varies|
|`cgroup/cpu/cores_quota`|Number of cores available to this process. Derived from `cpu.cfs_quota_us`/`cpu.cfs_period_us`.||Varies. A value of -1 indicates there is no explicit quota set.|
+|`cgroup/cpu/usage/total/percentage`|Total cpu percentage used by cgroup of process that is running||0-100|
+|`cgroup/cpu/usage/user/percentage`|User cpu percentage used by cgroup of process that is running||0-100|
+|`cgroup/cpu/usage/sys/percentage`|Sys cpu percentage used by cgroup of process that is running||0-100|
+|`cgroup/disk/read/size`|Reports the number of bytes transferred to specific devices by a cgroup of process that is running.|`diskName`|Varies|
+|`cgroup/disk/write/size`|Reports the number of bytes transferred from specific devices by a cgroup of process that is running.|`diskName`|Varies|
+|`cgroup/disk/read/count`|Reports the number of read operations performed on specific devices by a cgroup of process that is running.|`diskName`|Varies|
+|`cgroup/disk/write/count`|Reports the number of write operations performed on specific devices by a cgroup of process that is running.|`diskName`|Varies|
|`cgroup/memory/*`|Memory stats for this process, such as `cache` and `total_swap`. Each stat produces a separate metric. Read from `memory.stat`.||Varies|
|`cgroup/memory_numa/*/pages`|Memory stats, per NUMA node, for this process, such as `total` and `unevictable`. Each stat produces a separate metric. Read from `memory.num_stat`.|`numaZone`|Varies|
+|`cgroup/memory/limit/bytes`|Reports the maximum memory that can be used by processes in the cgroup (in bytes)||Varies|
+|`cgroup/memory/usage/bytes`|Reports the maximum amount of user memory (including file cache)||Varies|
|`cgroup/cpuset/cpu_count`|Total number of CPUs available to the process. Derived from `cpuset.cpus`.||Varies|
|`cgroup/cpuset/effective_cpu_count`|Total number of active CPUs available to the process. Derived from `cpuset.effective_cpus`.||Varies|
|`cgroup/cpuset/mems_count`|Total number of memory nodes available to the process. Derived from `cpuset.mems`.||Varies|
diff --git a/docs/development/extensions-core/kafka-extraction-namespace.md b/docs/querying/kafka-extraction-namespace.md
similarity index 92%
rename from docs/development/extensions-core/kafka-extraction-namespace.md
rename to docs/querying/kafka-extraction-namespace.md
index 2d841dfc94367..1cfa91aac5546 100644
--- a/docs/development/extensions-core/kafka-extraction-namespace.md
+++ b/docs/querying/kafka-extraction-namespace.md
@@ -22,7 +22,7 @@ title: "Apache Kafka Lookups"
~ under the License.
-->
-To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` and `druid-kafka-extraction-namespace` in the extensions load list.
+To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` and `druid-kafka-extraction-namespace` in the extensions load list.
If you need updates to populate as promptly as possible, it is possible to plug into a Kafka topic whose key is the old value and message is the desired new value (both in UTF-8) as a LookupExtractorFactory.
@@ -41,13 +41,13 @@ If you need updates to populate as promptly as possible, it is possible to plug
| `kafkaTopic` | The Kafka topic to read the data from | Yes ||
| `kafkaProperties` | Kafka consumer properties (`bootstrap.servers` must be specified) | Yes ||
| `connectTimeout` | How long to wait for an initial connection | No | `0` (do not wait) |
-| `isOneToOne` | The map is a one-to-one (see [Lookup DimensionSpecs](../../querying/dimensionspecs.md)) | No | `false` |
+| `isOneToOne` | The map is a one-to-one (see [Lookup DimensionSpecs](./dimensionspecs.md)) | No | `false` |
The extension `kafka-extraction-namespace` enables reading from an [Apache Kafka](https://kafka.apache.org/) topic which has name/key pairs to allow renaming of dimension values. An example use case would be to rename an ID to a human-readable format.
## How it Works
-The extractor works by consuming the configured Kafka topic from the beginning, and appending every record to an internal map. The key of the Kafka record is used as they key of the map, and the payload of the record is used as the value. At query time, a lookup can be used to transform the key into the associated value. See [lookups](../../querying/lookups.md) for how to configure and use lookups in a query. Keys and values are both stored as strings by the lookup extractor.
+The extractor works by consuming the configured Kafka topic from the beginning, and appending every record to an internal map. The key of the Kafka record is used as they key of the map, and the payload of the record is used as the value. At query time, a lookup can be used to transform the key into the associated value. See [lookups](./lookups.md) for how to configure and use lookups in a query. Keys and values are both stored as strings by the lookup extractor.
The extractor remains subscribed to the topic, so new records are added to the lookup map as they appear. This allows for lookup values to be updated in near-realtime. If two records are added to the topic with the same key, the record with the larger offset will replace the previous record in the lookup map. A record with a `null` payload will be treated as a tombstone record, and the associated key will be removed from the lookup map.
diff --git a/docs/development/extensions-core/lookups-cached-global.md b/docs/querying/lookups-cached-global.md
similarity index 93%
rename from docs/development/extensions-core/lookups-cached-global.md
rename to docs/querying/lookups-cached-global.md
index 5cfcbea01c244..72c4189c2dad0 100644
--- a/docs/development/extensions-core/lookups-cached-global.md
+++ b/docs/querying/lookups-cached-global.md
@@ -22,12 +22,12 @@ title: "Globally Cached Lookups"
~ under the License.
-->
-To use this Apache Druid extension, [include](../../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` in the extensions load list.
+To use this Apache Druid extension, [include](../configuration/extensions.md#loading-extensions) `druid-lookups-cached-global` in the extensions load list.
## Configuration
:::info
Static configuration is no longer supported. Lookups can be configured through
- [dynamic configuration](../../querying/lookups.md#configuration).
+ [dynamic configuration](./lookups.md#configuration).
:::
Globally cached lookups are appropriate for lookups which are not possible to pass at query time due to their size,
@@ -36,7 +36,7 @@ and are small enough to reasonably populate in-memory. This usually means tens t
Globally cached lookups all draw from the same cache pool, allowing each process to have a fixed cache pool that can be used by cached lookups.
-Globally cached lookups can be specified as part of the [cluster wide config for lookups](../../querying/lookups.md) as a type of `cachedNamespace`
+Globally cached lookups can be specified as part of the [cluster wide config for lookups](./lookups.md) as a type of `cachedNamespace`
```json
{
@@ -84,7 +84,7 @@ The parameters are as follows
|--------|-----------|--------|-------|
|`extractionNamespace`|Specifies how to populate the local cache. See below|Yes|-|
|`firstCacheTimeout`|How long to wait (in ms) for the first run of the cache to populate. 0 indicates to not wait|No|`0` (do not wait)|
-|`injective`|If the underlying map is [injective](../../querying/lookups.md#query-rewrites) (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`|
+|`injective`|If the underlying map is [injective](./lookups.md#query-rewrites) (keys and values are unique) then optimizations can occur internally by setting this to `true`|No|`false`|
If `firstCacheTimeout` is set to a non-zero value, it should be less than `druid.manager.lookups.hostUpdateTimeout`. If `firstCacheTimeout` is NOT set, then management is essentially asynchronous and does not know if a lookup succeeded or failed in starting. In such a case logs from the processes using lookups should be monitored for repeated failures.
@@ -93,7 +93,7 @@ Proper functionality of globally cached lookups requires the following extension
## Example configuration
-In a simple case where only one [tier](../../querying/lookups.md#dynamic-configuration) exists (`realtime_customer2`) with one `cachedNamespace` lookup called `country_code`, the resulting configuration JSON looks similar to the following:
+In a simple case where only one [tier](./lookups.md#dynamic-configuration) exists (`realtime_customer2`) with one `cachedNamespace` lookup called `country_code`, the resulting configuration JSON looks similar to the following:
```json
{
@@ -170,7 +170,7 @@ It's highly recommended that `druid.lookup.namespace.numBufferedEntries` is set
## Supported lookups
-For additional lookups, please see our [extensions list](../../configuration/extensions.md).
+For additional lookups, please see our [extensions list](../configuration/extensions.md).
### URI lookup
@@ -345,7 +345,7 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol
|Parameter|Description|Required|Default|
|---------|-----------|--------|-------|
-|`connectorConfig`|The connector config to use. You can set `connectURI`, `user` and `password`. You can selectively allow JDBC properties in `connectURI`. See [JDBC connections security config](../../configuration/index.md#jdbc-connections-to-external-databases) for more details.|Yes||
+|`connectorConfig`|The connector config to use. You can set `connectURI`, `user` and `password`. You can selectively allow JDBC properties in `connectURI`. See [JDBC connections security config](../configuration/index.md#jdbc-connections-to-external-databases) for more details.|Yes||
|`table`|The table which contains the key value pairs|Yes||
|`keyColumn`|The column in `table` which contains the keys|Yes||
|`valueColumn`|The column in `table` which contains the values|Yes||
@@ -377,7 +377,7 @@ The JDBC lookups will poll a database to populate its local cache. If the `tsCol
:::info
If using JDBC, you will need to add your database's client JAR files to the extension's directory.
For Postgres, the connector JAR is already included.
- See the MySQL extension documentation for instructions to obtain [MySQL](./mysql.md#installing-the-mysql-connector-library) or [MariaDB](./mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries.
+ See the MySQL extension documentation for instructions to obtain [MySQL](../development/extensions-core/mysql.md#installing-the-mysql-connector-library) or [MariaDB](../development/extensions-core/mysql.md#alternative-installing-the-mariadb-connector-library) connector libraries.
The connector JAR should reside in the classpath of Druid's main class loader.
To add the connector JAR to the classpath, you can copy the downloaded file to `lib/` under the distribution root directory. Alternatively, create a symbolic link to the connector in the `lib` directory.
:::
diff --git a/docs/querying/lookups.md b/docs/querying/lookups.md
index bbc1b03faca80..a22fbf03928c2 100644
--- a/docs/querying/lookups.md
+++ b/docs/querying/lookups.md
@@ -24,7 +24,7 @@ title: "Lookups"
Lookups are a concept in Apache Druid where dimension values are (optionally) replaced with new values, allowing join-like
functionality. Applying lookups in Druid is similar to joining a dimension table in a data warehouse. See
-[dimension specs](../querying/dimensionspecs.md) for more information. For the purpose of these documents, a "key"
+[dimension specs](./dimensionspecs.md) for more information. For the purpose of these documents, a "key"
refers to a dimension value to match, and a "value" refers to its replacement. So if you wanted to map
`appid-12345` to `Super Mega Awesome App` then the key would be `appid-12345` and the value would be
`Super Mega Awesome App`.
@@ -43,12 +43,12 @@ and such data belongs in the raw denormalized data for use in Druid.
Lookups are generally preloaded in-memory on all servers. But very small lookups (on the order of a few dozen to a few
hundred entries) can also be passed inline in native queries time using the "map" lookup type. Refer to the
-[dimension specs](dimensionspecs.md) documentation for details.
+[dimension specs](./dimensionspecs.md) documentation for details.
Other lookup types are available as extensions, including:
-- Globally cached lookups from local files, remote URIs, or JDBC through [lookups-cached-global](../development/extensions-core/lookups-cached-global.md).
-- Globally cached lookups from a Kafka topic through [kafka-extraction-namespace](../development/extensions-core/kafka-extraction-namespace.md).
+- Globally cached lookups from local files, remote URIs, or JDBC through [lookups-cached-global](./lookups-cached-global.md).
+- Globally cached lookups from a Kafka topic through [kafka-extraction-namespace](./kafka-extraction-namespace.md).
Query Syntax
------------
@@ -213,7 +213,7 @@ Injective lookups are eligible for the largest set of query rewrites. Injective
function may encounter null input values.
To determine whether a lookup is injective, Druid relies on an `injective` property that you can set in the
-[lookup definition](../development/extensions-core/lookups-cached-global.md). In general, you should set
+[lookup definition](./lookups-cached-global.md). In general, you should set
`injective: true` for any lookup that satisfies the required properties, to allow Druid to run your queries as fast as
possible.
diff --git a/docs/release-info/migr-front-coded-dict.md b/docs/release-info/migr-front-coded-dict.md
new file mode 100644
index 0000000000000..1c825d4f6e342
--- /dev/null
+++ b/docs/release-info/migr-front-coded-dict.md
@@ -0,0 +1,89 @@
+---
+id: migr-front-coded-dict
+title: "Migration guide: front-coded dictionaries"
+sidebar_label: Front-coded dictionaries
+---
+
+
+
+:::info
+Front coding is an [experimental feature](../development/experimental.md) introduced in Druid 25.0.0.
+:::
+
+Apache Druid encodes string columns into dictionaries for better compression.
+Front coding is an incremental encoding strategy that lets you store STRING and [COMPLEX<json>](../querying/nested-columns.md) columns in Druid with minimal performance impact.
+Front-coded dictionaries reduce storage and improve performance by optimizing for strings where the front part looks similar.
+For example, if you are tracking website visits, most URLs start with `https://domain.xyz/`, and front coding is able to exploit this pattern for more optimal compression when storing such datasets.
+Druid performs the optimization automatically, which means that the performance of string columns is generally not affected when they don't match the front-coded pattern.
+Consequently, you can enable this feature universally without having to know the underlying data shapes of the columns.
+
+You can use front coding with all types of ingestion.
+
+## Enable front coding
+
+To enable front coding, set `indexSpec.stringDictionaryEncoding.type` to `frontCoded` in the `tuningConfig` object of your [ingestion spec](../ingestion/ingestion-spec.md).
+
+You can specify the following optional properties:
+
+* `bucketSize`: Number of values to place in a bucket to perform delta encoding. Setting this property instructs indexing tasks to write segments using compressed dictionaries of the specified bucket size. You can set it to any power of 2 less than or equal to 128. `bucketSize` defaults to 4.
+* `formatVersion`: Specifies which front coding version to use. Options are 0 and 1 (supported for Druid versions 26.0.0 and higher). `formatVersion` defaults to 0.
+
+For example:
+
+```
+"tuningConfig": {
+ "indexSpec": {
+ "stringDictionaryEncoding": {
+ "type":"frontCoded",
+ "bucketSize": 4,
+ "formatVersion": 0
+ }
+ }
+}
+```
+
+## Upgrade from Druid 25.0.0
+
+Druid 26.0.0 introduced a new version of the front-coded dictionary, version 1, offering typically faster read speeds and smaller storage sizes.
+When upgrading to versions Druid 26.0.0 and higher, Druid continues to default front coding settings to version 0.
+This default enables seamless downgrades to Druid 25.0.0.
+
+To use the newer version, set the `formatVersion` property to 1:
+
+```
+"tuningConfig": {
+ "indexSpec": {
+ "stringDictionaryEncoding": {
+ "type":"frontCoded",
+ "bucketSize": 4,
+ "formatVersion": 1
+ }
+ }
+}
+```
+
+## Downgrade to Druid 25.0.0
+
+After upgrading to version 1, you can no longer downgrade to Druid 25.0.0 seamlessly.
+To downgrade to Druid 25.0.0, re-ingest your data with the `stringDictionaryEncoding.formatVersion` property set to 0.
+
+## Downgrade to a version preceding Druid 25.0.0
+
+Druid versions preceding 25.0.0 can't read segments with front-coded dictionaries. To downgrade to an older version, you must either delete the segments containing front-coded dictionaries or re-ingest them with `stringDictionaryEncoding.type` set to `utf8`.
diff --git a/docs/release-info/migr-subquery-limit.md b/docs/release-info/migr-subquery-limit.md
new file mode 100644
index 0000000000000..853be50294dcd
--- /dev/null
+++ b/docs/release-info/migr-subquery-limit.md
@@ -0,0 +1,60 @@
+---
+id: migr-subquery-limit
+title: "Migration guide: Subquery limit"
+sidebar_label: Subquery limit
+---
+
+
+
+Druid now allows you to set a byte-based limit on subquery size, to prevent brokers from running out of memory when handling large subqueries.
+Druid uses subqueries as joins as well as in common table expressions, such as WITH.
+
+The byte-based subquery limit overrides Druid's row-based subquery limit.
+
+:::info
+We recommend that you move towards using byte-based limits starting in Druid 30.0.
+:::
+
+For queries that generate a large number of rows (5 million or more), we recommend that you don't use `maxSubqueryBytes` from the outset.
+You can increase `maxSubqueryRows` and then configure the byte-based limit if you find that Druid needs it to process the query.
+
+## Row-based subquery limit
+
+Druid uses the `maxSubqueryRows` property to limit the number of rows Druid returns in a subquery.
+Because this is a row-based limit, it doesn't restrict the overall size of the returned data.
+
+The `maxSubqueryRows` property is set to 100,000 by default.
+
+## Enable a byte-based subquery limit
+
+Set the optional property `maxSubqueryBytes` to set a maximum number of returned bytes.
+This property takes precedence over `maxSubqueryRows`.
+
+## Usage considerations
+
+You can set both `maxSubqueryRows` and `maxSubqueryBytes` at cluster level and override them in individual queries.
+See [Overriding default query context values](../configuration#overriding-default-query-context-values) for more information.
+
+## Learn more
+
+See the following topics for more information:
+
+- [Query context](../querying/query-context.md) for information on setting query context parameters.
+- [Broker configuration reference](../configuration#guardrails-for-materialization-of-subqueries) for more information on `maxSubqueryRows` and `maxSubqueryBytes`.
diff --git a/docs/release-info/migration-guide.md b/docs/release-info/migration-guide.md
new file mode 100644
index 0000000000000..711f6b3da2575
--- /dev/null
+++ b/docs/release-info/migration-guide.md
@@ -0,0 +1,43 @@
+---
+id: migration-guide
+title: "Migration guides"
+description: How to migrate from legacy features to get the most from Druid updates
+---
+
+
+
+In general, when we introduce new features and behaviors into Apache Druid, we make every effort to avoid breaking existing features when introducing new behaviors. However, sometimes there are either bugs or performance limitations with the old behaviors that are not possible to fix in a backward-compatible way. In these cases, we must introduce breaking changes for the future maintainability of Druid.
+
+The guides in this section outline breaking changes introduced in Druid 25 and later. Each guide provides instructions to migrate to new features.
+
+
+
+## Migrate to front-coded dictionary encoding
+
+Druid encodes string columns into dictionaries for better compression. Front-coded dictionary encoding reduces storage and improves performance by optimizing for strings that share similar beginning substrings. See [Migration guide: front-coded dictionaries](migr-front-coded-dict.md) for more information.
\ No newline at end of file
diff --git a/docs/release-info/upgrade-notes.md b/docs/release-info/upgrade-notes.md
index 3146d88540080..fd622f21ea348 100644
--- a/docs/release-info/upgrade-notes.md
+++ b/docs/release-info/upgrade-notes.md
@@ -26,6 +26,84 @@ The upgrade notes assume that you are upgrading from the Druid version that imme
For the full release notes for a specific version, see the [releases page](https://github.com/apache/druid/releases).
+## 30.0.0
+
+### Upgrade notes
+
+#### Append JsonPath function
+
+The `append` function for JsonPath for ORC format now fails with an exception. Previously, it would run but not append anything.
+
+[#15772](https://github.com/apache/druid/pull/15772)
+
+#### Kinesis ingestion tuning
+
+The following properties have been deprecated as part of simplifying the memory tuning for Kinesis ingestion:
+
+- `recordBufferSize`, use `recordBufferSizeBytes` instead
+- `maxRecordsPerPoll`, use `maxBytesPerPoll` instead
+
+[#15360](https://github.com/apache/druid/pull/15360)
+
+#### Improved Supervisor rolling restarts
+
+The `stopTaskCount` config now prioritizes stopping older tasks first. As part of this change, you must also explicitly set a value for `stopTaskCount`. It no longer defaults to the same value as `taskCount`.
+
+[#15859](https://github.com/apache/druid/pull/15859)
+
+#### Changes to Coordinator default values
+
+The following are the changes to the default values for the Coordinator service:
+
+* The default value for `druid.coordinator.kill.period` (if unspecified) has changed from `P1D` to the value of `druid.coordinator.period.indexingPeriod`. Operators can choose to override `druid.coordinator.kill.period` and that takes precedence over the default behavior.
+* The default value for the dynamic configuration property `killTaskSlotRatio` has been updated from `1.0` to `0.1`. This ensures that kill tasks take up only one task slot by default instead of consuming all available task slots.
+
+[#16247](https://github.com/apache/druid/pull/16247)
+
+#### `GoogleTaskLogs` upload buffer size
+
+Changed the upload buffer size in `GoogleTaskLogs` to 1 MB instead of 15 MB to allow more uploads in parallel and prevent the MiddleManager service from running out of memory.
+
+[#16236](https://github.com/apache/druid/pull/16236)
+
+### Incompatible changes
+
+#### Changes to `targetDataSource` in EXPLAIN queries
+
+Druid 30.0.0 includes a breaking change that restores the behavior for `targetDataSource` to its 28.0.0 and earlier state, different from Druid 29.0.0 and only 29.0.0. In 29.0.0, `targetDataSource` returns a JSON object that includes the datasource name. In all other versions, `targetDataSource` returns a string containing the name of the datasource.
+
+If you're upgrading from any version other than 29.0.0, there is no change in behavior.
+
+If you are upgrading from 29.0.0, this is an incompatible change.
+
+[#16004](https://github.com/apache/druid/pull/16004)
+
+#### Removed ZooKeeper-based segment loading
+
+ZooKeeper-based segment loading is being removed due to known issues.
+It has been deprecated for several releases.
+Recent improvements to the Druid Coordinator have significantly enhanced performance with HTTP-based segment loading.
+
+[#15705](https://github.com/apache/druid/pull/15705)
+
+#### Removed Coordinator configs
+
+Removed the following Coordinator configs:
+
+* `druid.coordinator.load.timeout`: Not needed as the default value of this parameter (15 minutes) is known to work well for all clusters.
+* `druid.coordinator.loadqueuepeon.type`: Not needed as this value is always `http`.
+* `druid.coordinator.curator.loadqueuepeon.numCallbackThreads`: Not needed as ZooKeeper(curator)-based segment loading isn't an option anymore.
+
+Auto-cleanup of compaction configs of inactive datasources is now enabled by default.
+
+[#15705](https://github.com/apache/druid/pull/15705)
+
+#### Changed `useMaxMemoryEstimates` for Hadoop jobs
+
+The default value of the `useMaxMemoryEstimates` parameter for Hadoop jobs is now `false`.
+
+[#16280](https://github.com/apache/druid/pull/16280)
+
## 29.0.1
### Incompatible changes
diff --git a/extensions-contrib/druid-deltalake-extensions/pom.xml b/extensions-contrib/druid-deltalake-extensions/pom.xml
index fb5ca5912ba67..053f6556573c3 100644
--- a/extensions-contrib/druid-deltalake-extensions/pom.xml
+++ b/extensions-contrib/druid-deltalake-extensions/pom.xml
@@ -35,7 +35,7 @@
4.0.0
- 3.1.0
+ 3.2.0
diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java
index 56ccd2a41ae9f..7d126caef349d 100644
--- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java
+++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/DeltaInputSource.java
@@ -28,12 +28,12 @@
import io.delta.kernel.ScanBuilder;
import io.delta.kernel.Snapshot;
import io.delta.kernel.Table;
-import io.delta.kernel.TableNotFoundException;
-import io.delta.kernel.client.TableClient;
import io.delta.kernel.data.ColumnarBatch;
import io.delta.kernel.data.FilteredColumnarBatch;
import io.delta.kernel.data.Row;
-import io.delta.kernel.defaults.client.DefaultTableClient;
+import io.delta.kernel.defaults.engine.DefaultEngine;
+import io.delta.kernel.engine.Engine;
+import io.delta.kernel.exceptions.TableNotFoundException;
import io.delta.kernel.expressions.Predicate;
import io.delta.kernel.internal.InternalScanFileUtils;
import io.delta.kernel.internal.data.ScanStateRow;
@@ -120,7 +120,7 @@ public boolean needsFormat()
/**
* Instantiates a {@link DeltaInputSourceReader} to read the Delta table rows. If a {@link DeltaSplit} is supplied,
- * the Delta files and schema are obtained from it to instantiate the reader. Otherwise, a Delta table client is
+ * the Delta files and schema are obtained from it to instantiate the reader. Otherwise, the Delta engine is
* instantiated with the supplied configuration to read the table.
*
* @param inputRowSchema schema for {@link org.apache.druid.data.input.InputRow}
@@ -134,40 +134,40 @@ public InputSourceReader reader(
File temporaryDirectory
)
{
- final TableClient tableClient = createTableClient();
+ final Engine engine = createDeltaEngine();
try {
final List> scanFileDataIters = new ArrayList<>();
if (deltaSplit != null) {
- final Row scanState = deserialize(tableClient, deltaSplit.getStateRow());
+ final Row scanState = deserialize(engine, deltaSplit.getStateRow());
final StructType physicalReadSchema =
- ScanStateRow.getPhysicalDataReadSchema(tableClient, scanState);
+ ScanStateRow.getPhysicalDataReadSchema(engine, scanState);
for (String file : deltaSplit.getFiles()) {
- final Row scanFile = deserialize(tableClient, file);
+ final Row scanFile = deserialize(engine, file);
scanFileDataIters.add(
- getTransformedDataIterator(tableClient, scanState, scanFile, physicalReadSchema, Optional.empty())
+ getTransformedDataIterator(engine, scanState, scanFile, physicalReadSchema, Optional.empty())
);
}
} else {
- final Table table = Table.forPath(tableClient, tablePath);
- final Snapshot latestSnapshot = table.getLatestSnapshot(tableClient);
- final StructType fullSnapshotSchema = latestSnapshot.getSchema(tableClient);
+ final Table table = Table.forPath(engine, tablePath);
+ final Snapshot latestSnapshot = table.getLatestSnapshot(engine);
+ final StructType fullSnapshotSchema = latestSnapshot.getSchema(engine);
final StructType prunedSchema = pruneSchema(
fullSnapshotSchema,
inputRowSchema.getColumnsFilter()
);
- final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(tableClient);
+ final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(engine);
if (filter != null) {
- scanBuilder.withFilter(tableClient, filter.getFilterPredicate(fullSnapshotSchema));
+ scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema));
}
- final Scan scan = scanBuilder.withReadSchema(tableClient, prunedSchema).build();
- final CloseableIterator scanFilesIter = scan.getScanFiles(tableClient);
- final Row scanState = scan.getScanState(tableClient);
+ final Scan scan = scanBuilder.withReadSchema(engine, prunedSchema).build();
+ final CloseableIterator scanFilesIter = scan.getScanFiles(engine);
+ final Row scanState = scan.getScanState(engine);
final StructType physicalReadSchema =
- ScanStateRow.getPhysicalDataReadSchema(tableClient, scanState);
+ ScanStateRow.getPhysicalDataReadSchema(engine, scanState);
while (scanFilesIter.hasNext()) {
final FilteredColumnarBatch scanFileBatch = scanFilesIter.next();
@@ -176,7 +176,7 @@ public InputSourceReader reader(
while (scanFileRows.hasNext()) {
final Row scanFile = scanFileRows.next();
scanFileDataIters.add(
- getTransformedDataIterator(tableClient, scanState, scanFile, physicalReadSchema, scan.getRemainingFilter())
+ getTransformedDataIterator(engine, scanState, scanFile, physicalReadSchema, scan.getRemainingFilter())
);
}
}
@@ -203,26 +203,26 @@ public Stream> createSplits(InputFormat inputFormat, @Nul
return Stream.of(new InputSplit<>(deltaSplit));
}
- final TableClient tableClient = createTableClient();
+ final Engine engine = createDeltaEngine();
final Snapshot latestSnapshot;
+ final Table table = Table.forPath(engine, tablePath);
try {
- final Table table = Table.forPath(tableClient, tablePath);
- latestSnapshot = table.getLatestSnapshot(tableClient);
+ latestSnapshot = table.getLatestSnapshot(engine);
}
catch (TableNotFoundException e) {
throw InvalidInput.exception(e, "tablePath[%s] not found.", tablePath);
}
- final StructType fullSnapshotSchema = latestSnapshot.getSchema(tableClient);
+ final StructType fullSnapshotSchema = latestSnapshot.getSchema(engine);
- final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(tableClient);
+ final ScanBuilder scanBuilder = latestSnapshot.getScanBuilder(engine);
if (filter != null) {
- scanBuilder.withFilter(tableClient, filter.getFilterPredicate(fullSnapshotSchema));
+ scanBuilder.withFilter(engine, filter.getFilterPredicate(fullSnapshotSchema));
}
- final Scan scan = scanBuilder.withReadSchema(tableClient, fullSnapshotSchema).build();
+ final Scan scan = scanBuilder.withReadSchema(engine, fullSnapshotSchema).build();
// scan files iterator for the current snapshot
- final CloseableIterator scanFilesIterator = scan.getScanFiles(tableClient);
+ final CloseableIterator scanFilesIterator = scan.getScanFiles(engine);
- final Row scanState = scan.getScanState(tableClient);
+ final Row scanState = scan.getScanState(engine);
final String scanStateStr = RowSerde.serializeRowToJson(scanState);
Iterator deltaSplitIterator = Iterators.transform(
@@ -256,9 +256,9 @@ public InputSource withSplit(InputSplit split)
);
}
- private Row deserialize(TableClient tableClient, String row)
+ private Row deserialize(Engine engine, String row)
{
- return RowSerde.deserializeRowFromJson(tableClient, row);
+ return RowSerde.deserializeRowFromJson(engine, row);
}
/**
@@ -285,17 +285,17 @@ private StructType pruneSchema(final StructType baseSchema, final ColumnsFilter
}
/**
- * @return a table client where the client is initialized with {@link Configuration} class that uses the class's
+ * @return a Delta engine initialized with {@link Configuration} class that uses the class's
* class loader instead of the context classloader. The latter by default doesn't know about the extension classes,
- * so the table client cannot load runtime classes resulting in {@link ClassNotFoundException}.
+ * so the Delta engine cannot load runtime classes resulting in {@link ClassNotFoundException}.
*/
- private TableClient createTableClient()
+ private Engine createDeltaEngine()
{
final ClassLoader currCtxClassloader = Thread.currentThread().getContextClassLoader();
try {
Thread.currentThread().setContextClassLoader(getClass().getClassLoader());
final Configuration conf = new Configuration();
- return DefaultTableClient.create(conf);
+ return DefaultEngine.create(conf);
}
finally {
Thread.currentThread().setContextClassLoader(currCtxClassloader);
@@ -308,7 +308,7 @@ private TableClient createTableClient()
* SingleThreadedTableReader.java.
*/
private CloseableIterator getTransformedDataIterator(
- final TableClient tableClient,
+ final Engine engine,
final Row scanState,
final Row scanFile,
final StructType physicalReadSchema,
@@ -317,14 +317,14 @@ private CloseableIterator getTransformedDataIterator(
{
final FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFile);
- final CloseableIterator physicalDataIter = tableClient.getParquetHandler().readParquetFiles(
+ final CloseableIterator physicalDataIter = engine.getParquetHandler().readParquetFiles(
Utils.singletonCloseableIterator(fileStatus),
physicalReadSchema,
optionalPredicate
);
return Scan.transformPhysicalData(
- tableClient,
+ engine,
scanState,
scanFile,
physicalDataIter
diff --git a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java
index e37c5d503314d..bad6191496d79 100644
--- a/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java
+++ b/extensions-contrib/druid-deltalake-extensions/src/main/java/org/apache/druid/delta/input/RowSerde.java
@@ -23,9 +23,9 @@
import com.fasterxml.jackson.databind.JsonNode;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.node.ObjectNode;
-import io.delta.kernel.client.TableClient;
import io.delta.kernel.data.Row;
import io.delta.kernel.defaults.internal.data.DefaultJsonRow;
+import io.delta.kernel.engine.Engine;
import io.delta.kernel.internal.util.VectorUtils;
import io.delta.kernel.types.ArrayType;
import io.delta.kernel.types.BooleanType;
@@ -84,13 +84,12 @@ public static String serializeRowToJson(Row row)
/**
* Utility method to deserialize a {@link Row} object from the JSON form.
*/
- public static Row deserializeRowFromJson(TableClient tableClient, String jsonRowWithSchema)
+ public static Row deserializeRowFromJson(Engine engine, String jsonRowWithSchema)
{
try {
JsonNode jsonNode = OBJECT_MAPPER.readTree(jsonRowWithSchema);
JsonNode schemaNode = jsonNode.get("schema");
- StructType schema =
- tableClient.getJsonHandler().deserializeStructType(schemaNode.asText());
+ StructType schema = engine.getJsonHandler().deserializeStructType(schemaNode.asText());
return parseRowFromJsonWithSchema((ObjectNode) jsonNode.get("row"), schema);
}
catch (JsonProcessingException e) {
diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java
index 9e597894d05c9..4e1c2566f02ec 100644
--- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java
+++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaInputRowTest.java
@@ -20,12 +20,12 @@
package org.apache.druid.delta.input;
import io.delta.kernel.Scan;
-import io.delta.kernel.TableNotFoundException;
-import io.delta.kernel.client.TableClient;
import io.delta.kernel.data.ColumnarBatch;
import io.delta.kernel.data.FilteredColumnarBatch;
import io.delta.kernel.data.Row;
-import io.delta.kernel.defaults.client.DefaultTableClient;
+import io.delta.kernel.defaults.engine.DefaultEngine;
+import io.delta.kernel.engine.Engine;
+import io.delta.kernel.exceptions.TableNotFoundException;
import io.delta.kernel.internal.InternalScanFileUtils;
import io.delta.kernel.internal.data.ScanStateRow;
import io.delta.kernel.internal.util.Utils;
@@ -68,13 +68,13 @@ public void testDeltaInputRow(
final List> expectedRows
) throws TableNotFoundException, IOException
{
- final TableClient tableClient = DefaultTableClient.create(new Configuration());
- final Scan scan = DeltaTestUtils.getScan(tableClient, deltaTablePath);
+ final Engine engine = DefaultEngine.create(new Configuration());
+ final Scan scan = DeltaTestUtils.getScan(engine, deltaTablePath);
- final Row scanState = scan.getScanState(tableClient);
- final StructType physicalReadSchema = ScanStateRow.getPhysicalDataReadSchema(tableClient, scanState);
+ final Row scanState = scan.getScanState(engine);
+ final StructType physicalReadSchema = ScanStateRow.getPhysicalDataReadSchema(engine, scanState);
- final CloseableIterator scanFileIter = scan.getScanFiles(tableClient);
+ final CloseableIterator scanFileIter = scan.getScanFiles(engine);
int totalRecordCount = 0;
while (scanFileIter.hasNext()) {
final FilteredColumnarBatch scanFileBatch = scanFileIter.next();
@@ -84,13 +84,13 @@ public void testDeltaInputRow(
final Row scanFile = scanFileRows.next();
final FileStatus fileStatus = InternalScanFileUtils.getAddFileStatus(scanFile);
- final CloseableIterator physicalDataIter = tableClient.getParquetHandler().readParquetFiles(
+ final CloseableIterator physicalDataIter = engine.getParquetHandler().readParquetFiles(
Utils.singletonCloseableIterator(fileStatus),
physicalReadSchema,
Optional.empty()
);
final CloseableIterator dataIter = Scan.transformPhysicalData(
- tableClient,
+ engine,
scanState,
scanFile,
physicalDataIter
diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java
index 96696e7b6a438..6d49428ce0280 100644
--- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java
+++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/DeltaTestUtils.java
@@ -23,19 +23,19 @@
import io.delta.kernel.ScanBuilder;
import io.delta.kernel.Snapshot;
import io.delta.kernel.Table;
-import io.delta.kernel.TableNotFoundException;
-import io.delta.kernel.client.TableClient;
+import io.delta.kernel.engine.Engine;
+import io.delta.kernel.exceptions.TableNotFoundException;
import io.delta.kernel.types.StructType;
public class DeltaTestUtils
{
- public static Scan getScan(final TableClient tableClient, final String deltaTablePath) throws TableNotFoundException
+ public static Scan getScan(final Engine engine, final String deltaTablePath) throws TableNotFoundException
{
- final Table table = Table.forPath(tableClient, deltaTablePath);
- final Snapshot snapshot = table.getLatestSnapshot(tableClient);
- final StructType readSchema = snapshot.getSchema(tableClient);
- final ScanBuilder scanBuilder = snapshot.getScanBuilder(tableClient)
- .withReadSchema(tableClient, readSchema);
+ final Table table = Table.forPath(engine, deltaTablePath);
+ final Snapshot snapshot = table.getLatestSnapshot(engine);
+ final StructType readSchema = snapshot.getSchema(engine);
+ final ScanBuilder scanBuilder = snapshot.getScanBuilder(engine)
+ .withReadSchema(engine, readSchema);
return scanBuilder.build();
}
}
diff --git a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java
index fe2b85d2f3f75..7ac3eec09ef67 100644
--- a/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java
+++ b/extensions-contrib/druid-deltalake-extensions/src/test/java/org/apache/druid/delta/input/RowSerdeTest.java
@@ -20,11 +20,11 @@
package org.apache.druid.delta.input;
import io.delta.kernel.Scan;
-import io.delta.kernel.TableNotFoundException;
import io.delta.kernel.data.Row;
-import io.delta.kernel.defaults.client.DefaultTableClient;
+import io.delta.kernel.defaults.engine.DefaultEngine;
+import io.delta.kernel.exceptions.TableNotFoundException;
import org.apache.hadoop.conf.Configuration;
-import org.junit.Assert;
+import org.junit.jupiter.api.Assertions;
import org.junit.jupiter.params.ParameterizedTest;
import org.junit.jupiter.params.provider.MethodSource;
@@ -46,13 +46,13 @@ public static Collection data()
@ParameterizedTest(name = "{index}:with context {0}")
public void testSerializeDeserializeRoundtrip(final String tablePath) throws TableNotFoundException
{
- final DefaultTableClient tableClient = DefaultTableClient.create(new Configuration());
- final Scan scan = DeltaTestUtils.getScan(tableClient, tablePath);
- final Row scanState = scan.getScanState(tableClient);
+ final DefaultEngine engine = DefaultEngine.create(new Configuration());
+ final Scan scan = DeltaTestUtils.getScan(engine, tablePath);
+ final Row scanState = scan.getScanState(engine);
final String rowJson = RowSerde.serializeRowToJson(scanState);
- final Row row = RowSerde.deserializeRowFromJson(tableClient, rowJson);
+ final Row row = RowSerde.deserializeRowFromJson(engine, rowJson);
- Assert.assertEquals(scanState.getSchema(), row.getSchema());
+ Assertions.assertEquals(scanState.getSchema(), row.getSchema());
}
}
diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java
index a22f886cb31b0..7b659d031296c 100644
--- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java
+++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/HiveIcebergCatalog.java
@@ -57,6 +57,9 @@ public class HiveIcebergCatalog extends IcebergCatalog
@JsonProperty
private Map catalogProperties;
+ @JsonProperty
+ private final Boolean caseSensitive;
+
private final Configuration configuration;
private BaseMetastoreCatalog hiveCatalog;
@@ -69,6 +72,7 @@ public HiveIcebergCatalog(
@JsonProperty("catalogUri") String catalogUri,
@JsonProperty("catalogProperties") @Nullable
Map catalogProperties,
+ @JsonProperty("caseSensitive") Boolean caseSensitive,
@JacksonInject @Json ObjectMapper mapper,
@JacksonInject @HiveConf Configuration configuration
)
@@ -76,6 +80,7 @@ public HiveIcebergCatalog(
this.warehousePath = Preconditions.checkNotNull(warehousePath, "warehousePath cannot be null");
this.catalogUri = Preconditions.checkNotNull(catalogUri, "catalogUri cannot be null");
this.catalogProperties = DynamicConfigProviderUtils.extraConfigAndSetStringMap(catalogProperties, DRUID_DYNAMIC_CONFIG_PROVIDER_KEY, mapper);
+ this.caseSensitive = caseSensitive == null ? true : caseSensitive;
this.configuration = configuration;
this.catalogProperties
.forEach(this.configuration::set);
@@ -137,4 +142,10 @@ public Map getCatalogProperties()
{
return catalogProperties;
}
+
+ @Override
+ public boolean isCaseSensitive()
+ {
+ return caseSensitive;
+ }
}
diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java
index 07b41b6e10b14..fe08dedef4d50 100644
--- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java
+++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/IcebergCatalog.java
@@ -50,6 +50,11 @@ public abstract class IcebergCatalog
public abstract BaseMetastoreCatalog retrieveCatalog();
+ public boolean isCaseSensitive()
+ {
+ return true;
+ }
+
/**
* Extract the iceberg data files upto the latest snapshot associated with the table
*
@@ -92,6 +97,8 @@ public List extractSnapshotDataFiles(
if (snapshotTime != null) {
tableScan = tableScan.asOfTime(snapshotTime.getMillis());
}
+
+ tableScan = tableScan.caseSensitive(isCaseSensitive());
CloseableIterable tasks = tableScan.planFiles();
CloseableIterable.transform(tasks, FileScanTask::file)
.forEach(dataFile -> dataFilePaths.add(dataFile.path().toString()));
diff --git a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java
index d4961bb09678e..4539a582670f6 100644
--- a/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java
+++ b/extensions-contrib/druid-iceberg-extensions/src/main/java/org/apache/druid/iceberg/input/LocalCatalog.java
@@ -43,18 +43,23 @@ public class LocalCatalog extends IcebergCatalog
@JsonProperty
private final Map catalogProperties;
+ @JsonProperty
+ private final Boolean caseSensitive;
+
private BaseMetastoreCatalog catalog;
@JsonCreator
public LocalCatalog(
@JsonProperty("warehousePath") String warehousePath,
@JsonProperty("catalogProperties") @Nullable
- Map catalogProperties
+ Map catalogProperties,
+ @JsonProperty("caseSensitive") Boolean caseSensitive
)
{
Preconditions.checkNotNull(warehousePath, "warehousePath is null");
this.warehousePath = warehousePath;
this.catalogProperties = catalogProperties;
+ this.caseSensitive = caseSensitive == null ? true : caseSensitive;
this.catalog = retrieveCatalog();
}
@@ -71,6 +76,12 @@ public Map getCatalogProperties()
return catalogProperties;
}
+ @Override
+ public boolean isCaseSensitive()
+ {
+ return caseSensitive;
+ }
+
@Override
public BaseMetastoreCatalog retrieveCatalog()
{
@@ -100,12 +111,13 @@ public boolean equals(Object o)
}
LocalCatalog that = (LocalCatalog) o;
return warehousePath.equals(that.warehousePath)
- && Objects.equals(catalogProperties, that.catalogProperties);
+ && Objects.equals(catalogProperties, that.catalogProperties)
+ && Objects.equals(caseSensitive, that.caseSensitive);
}
@Override
public int hashCode()
{
- return Objects.hash(warehousePath, catalogProperties);
+ return Objects.hash(warehousePath, catalogProperties, caseSensitive);
}
}
diff --git a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/HiveIcebergCatalogTest.java b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/HiveIcebergCatalogTest.java
index def778ee9063a..d7e181e2c1cb9 100644
--- a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/HiveIcebergCatalogTest.java
+++ b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/HiveIcebergCatalogTest.java
@@ -42,6 +42,7 @@ public void testCatalogCreate()
warehouseDir.getPath(),
"hdfs://testuri",
new HashMap<>(),
+ true,
mapper,
new Configuration()
);
@@ -49,6 +50,7 @@ public void testCatalogCreate()
warehouseDir.getPath(),
"hdfs://testuri",
null,
+ null,
mapper,
new Configuration()
);
@@ -68,6 +70,7 @@ public void testAuthenticate()
warehouseDir.getPath(),
"hdfs://testuri",
catalogMap,
+ null,
mapper,
new Configuration()
);
diff --git a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/IcebergInputSourceTest.java b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/IcebergInputSourceTest.java
index 668a34b352b19..5a2429d6c7cfb 100644
--- a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/IcebergInputSourceTest.java
+++ b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/IcebergInputSourceTest.java
@@ -67,6 +67,7 @@ public class IcebergInputSourceTest
private IcebergCatalog testCatalog;
private TableIdentifier tableIdentifier;
+ private File warehouseDir;
private Schema tableSchema = new Schema(
Types.NestedField.required(1, "id", Types.StringType.get()),
@@ -80,8 +81,8 @@ public class IcebergInputSourceTest
@Before
public void setup() throws IOException
{
- final File warehouseDir = FileUtils.createTempDir();
- testCatalog = new LocalCatalog(warehouseDir.getPath(), new HashMap<>());
+ warehouseDir = FileUtils.createTempDir();
+ testCatalog = new LocalCatalog(warehouseDir.getPath(), new HashMap<>(), true);
tableIdentifier = TableIdentifier.of(Namespace.of(NAMESPACE), TABLENAME);
createAndLoadTable(tableIdentifier);
@@ -187,6 +188,33 @@ public void testInputSourceReadFromLatestSnapshot() throws IOException
Assert.assertEquals(1, splits.count());
}
+ @Test
+ public void testCaseInsensitiveFiltering() throws IOException
+ {
+ LocalCatalog caseInsensitiveCatalog = new LocalCatalog(warehouseDir.getPath(), new HashMap<>(), false);
+ Table icebergTableFromSchema = testCatalog.retrieveCatalog().loadTable(tableIdentifier);
+
+ icebergTableFromSchema.updateSchema().renameColumn("name", "Name").commit();
+ IcebergInputSource inputSource = new IcebergInputSource(
+ TABLENAME,
+ NAMESPACE,
+ new IcebergEqualsFilter("name", "Foo"),
+ caseInsensitiveCatalog,
+ new LocalInputSourceFactory(),
+ null
+ );
+
+ Stream>> splits = inputSource.createSplits(null, new MaxSizeSplitHintSpec(null, null));
+ List localInputSourceList = splits.map(inputSource::withSplit)
+ .map(inpSource -> (LocalInputSource) inpSource)
+ .map(LocalInputSource::getFiles)
+ .flatMap(List::stream)
+ .collect(Collectors.toList());
+
+ Assert.assertEquals(1, inputSource.estimateNumSplits(null, new MaxSizeSplitHintSpec(1L, null)));
+ Assert.assertEquals(1, localInputSourceList.size());
+ }
+
@After
public void tearDown()
{
@@ -197,7 +225,6 @@ private void createAndLoadTable(TableIdentifier tableIdentifier) throws IOExcept
{
//Setup iceberg table and schema
Table icebergTableFromSchema = testCatalog.retrieveCatalog().createTable(tableIdentifier, tableSchema);
-
//Generate an iceberg record and write it to a file
GenericRecord record = GenericRecord.create(tableSchema);
ImmutableList.Builder builder = ImmutableList.builder();
diff --git a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/LocalCatalogTest.java b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/LocalCatalogTest.java
index b0a7b5528a175..69b1df5951587 100644
--- a/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/LocalCatalogTest.java
+++ b/extensions-contrib/druid-iceberg-extensions/src/test/java/org/apache/druid/iceberg/input/LocalCatalogTest.java
@@ -36,7 +36,7 @@ public void testCatalogSerDe() throws JsonProcessingException
{
final File warehouseDir = FileUtils.createTempDir();
DefaultObjectMapper mapper = new DefaultObjectMapper();
- LocalCatalog before = new LocalCatalog(warehouseDir.getPath(), new HashMap<>());
+ LocalCatalog before = new LocalCatalog(warehouseDir.getPath(), new HashMap<>(), true);
LocalCatalog after = mapper.readValue(
mapper.writeValueAsString(before), LocalCatalog.class);
Assert.assertEquals(before, after);
diff --git a/extensions-core/azure-extensions/pom.xml b/extensions-core/azure-extensions/pom.xml
index 494b246094d13..2955d88c406cc 100644
--- a/extensions-core/azure-extensions/pom.xml
+++ b/extensions-core/azure-extensions/pom.xml
@@ -143,8 +143,23 @@
- junit
- junit
+ org.junit.jupiter
+ junit-jupiter
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-api
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-engine
+ test
+
+
+ org.junit.jupiter
+ junit-jupiter-params
test
diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureClientFactory.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureClientFactory.java
index 7afde0466d512..a6e1ef2f49ea6 100644
--- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureClientFactory.java
+++ b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/AzureClientFactory.java
@@ -32,8 +32,8 @@
import javax.annotation.Nullable;
import java.time.Duration;
-import java.util.HashMap;
import java.util.Map;
+import java.util.concurrent.ConcurrentHashMap;
/**
* Factory class for generating BlobServiceClient objects used for deep storage.
@@ -47,7 +47,7 @@ public class AzureClientFactory
public AzureClientFactory(AzureAccountConfig config)
{
this.config = config;
- this.cachedBlobServiceClients = new HashMap<>();
+ this.cachedBlobServiceClients = new ConcurrentHashMap<>();
}
// It's okay to store clients in a map here because all the configs for specifying azure retries are static, and there are only 2 of them.
diff --git a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/ICloudSpecificObjectToCloudObjectLocationConverter.java b/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/ICloudSpecificObjectToCloudObjectLocationConverter.java
deleted file mode 100644
index cc5ce4f2fb75f..0000000000000
--- a/extensions-core/azure-extensions/src/main/java/org/apache/druid/storage/azure/ICloudSpecificObjectToCloudObjectLocationConverter.java
+++ /dev/null
@@ -1,31 +0,0 @@
-/*
- * 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.storage.azure;
-
-import org.apache.druid.data.input.impl.CloudObjectLocation;
-
-/**
- * Interface for converting between from some object, T, and a {@link CloudObjectLocation} object
- * @param The object to convert to a {@link CloudObjectLocation} object
- */
-public interface ICloudSpecificObjectToCloudObjectLocationConverter
-{
- CloudObjectLocation createCloudObjectLocation(T cloudSpecificImpl);
-}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureEntityTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureEntityTest.java
index 8453b71b4b70e..540f7e8f5bac0 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureEntityTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureEntityTest.java
@@ -28,14 +28,16 @@
import org.apache.druid.storage.azure.AzureUtils;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
+
public class AzureEntityTest extends EasyMockSupport
{
private static final String STORAGE_ACCOUNT_NAME = "storageAccount";
@@ -49,8 +51,6 @@ public class AzureEntityTest extends EasyMockSupport
private CloudObjectLocation location;
private AzureByteSourceFactory byteSourceFactory;
private AzureByteSource byteSource;
-
- private AzureEntity azureEntity;
private AzureStorage azureStorage;
static {
@@ -62,7 +62,7 @@ public class AzureEntityTest extends EasyMockSupport
}
}
- @Before
+ @BeforeEach
public void setup()
{
location = createMock(CloudObjectLocation.class);
@@ -80,13 +80,12 @@ public void test_getUri_returnsLocationUri()
EasyMock.expect(location.toUri(AzureInputSource.SCHEME)).andReturn(ENTITY_URI);
replayAll();
- azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
+ final AzureEntity azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
URI actualUri = azureEntity.getUri();
- Assert.assertEquals(ENTITY_URI, actualUri);
+ assertEquals(ENTITY_URI, actualUri);
verifyAll();
-
}
@Test
@@ -95,20 +94,19 @@ public void test_getUri_returnsLocationUri_azureStorageScheme()
EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_NAME, azureStorage)).andReturn(byteSource);
replayAll();
- azureEntity = new AzureEntity(
+ final AzureEntity azureEntity = new AzureEntity(
new CloudObjectLocation(STORAGE_ACCOUNT_NAME, CONTAINER_NAME + "/" + BLOB_NAME),
azureStorage,
AzureStorageAccountInputSource.SCHEME,
byteSourceFactory
);
- Assert.assertEquals(
+ assertEquals(
URI.create(AzureStorageAccountInputSource.SCHEME + "://" + STORAGE_ACCOUNT_NAME + "/" + CONTAINER_NAME + "/" + BLOB_NAME),
azureEntity.getUri()
);
verifyAll();
-
}
@Test
@@ -120,10 +118,10 @@ public void test_readFromStart_returnsExpectedStream() throws Exception
EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_NAME, azureStorage)).andReturn(byteSource);
replayAll();
- azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
+ final AzureEntity azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
InputStream actualInputStream = azureEntity.readFrom(0);
- Assert.assertSame(INPUT_STREAM, actualInputStream);
+ assertSame(INPUT_STREAM, actualInputStream);
}
@Test
@@ -135,10 +133,10 @@ public void test_readFrom_returnsExpectedStream() throws Exception
EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_NAME, azureStorage)).andReturn(byteSource);
replayAll();
- azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
+ final AzureEntity azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
InputStream actualInputStream = azureEntity.readFrom(OFFSET);
- Assert.assertSame(INPUT_STREAM, actualInputStream);
+ assertSame(INPUT_STREAM, actualInputStream);
}
@Test
@@ -151,7 +149,7 @@ public void test_readFrom_throwsIOException_propogatesError()
EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_NAME, azureStorage)).andReturn(byteSource);
replayAll();
- azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
+ final AzureEntity azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
azureEntity.readFrom(OFFSET);
}
catch (IOException e) {
@@ -167,10 +165,10 @@ public void test_getPath_returnsLocationPath()
EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_NAME, azureStorage)).andReturn(byteSource);
replayAll();
- azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
+ final AzureEntity azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
String actualPath = azureEntity.getPath();
- Assert.assertEquals(BLOB_NAME, actualPath);
+ assertEquals(BLOB_NAME, actualPath);
verifyAll();
}
@@ -180,20 +178,18 @@ public void test_getPath_azureStorageScheme()
EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_NAME, azureStorage)).andReturn(byteSource);
replayAll();
- azureEntity = new AzureEntity(
+ final AzureEntity azureEntity = new AzureEntity(
new CloudObjectLocation(STORAGE_ACCOUNT_NAME, CONTAINER_NAME + "/" + BLOB_NAME),
azureStorage,
AzureStorageAccountInputSource.SCHEME,
byteSourceFactory
);
- Assert.assertEquals(
- CONTAINER_NAME + "/" + BLOB_NAME,
- azureEntity.getPath()
- );
+ assertEquals(CONTAINER_NAME + "/" + BLOB_NAME, azureEntity.getPath());
verifyAll();
}
+
@Test
public void test_getRetryCondition_returnsExpectedRetryCondition()
{
@@ -202,8 +198,8 @@ public void test_getRetryCondition_returnsExpectedRetryCondition()
EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_NAME, azureStorage)).andReturn(byteSource);
replayAll();
- azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
+ final AzureEntity azureEntity = new AzureEntity(location, azureStorage, AzureInputSource.SCHEME, byteSourceFactory);
Predicate actualRetryCondition = azureEntity.getRetryCondition();
- Assert.assertSame(AzureUtils.AZURE_RETRY, actualRetryCondition);
+ assertSame(AzureUtils.AZURE_RETRY, actualRetryCondition);
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceSerdeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceSerdeTest.java
index e11ce8ba5480a..1fc572c3fd7d7 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceSerdeTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceSerdeTest.java
@@ -32,39 +32,44 @@
import org.apache.druid.storage.azure.AzureStorage;
import org.apache.druid.storage.azure.AzureStorageDruidModule;
import org.easymock.EasyMockSupport;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
import java.net.URI;
import java.util.Collections;
import java.util.List;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNull;
+
public class AzureInputSourceSerdeTest extends EasyMockSupport
{
- private static final String JSON_WITH_URIS = "{\n"
- + " \"type\": \"azure\",\n"
- + " \"uris\": [\"azure://datacontainer2/wikipedia.json\"]\n"
- + "}";
-
- private static final String JSON_WITH_PREFIXES = "{\n"
- + " \"type\": \"azure\",\n"
- + " \"prefixes\": [\"azure://datacontainer2\"]\n"
- + "}";
-
- private static final String JSON_WITH_OBJECTS = "{\n"
- + " \"type\": \"azure\",\n"
- + " \"objects\": [\n"
- + " { \"bucket\": \"container1\", \"path\": \"bar/file1.json\"},\n"
- + " { \"bucket\": \"conatiner2\", \"path\": \"foo/file2.json\"}\n"
- + " ]\n"
- + " }";
+ private static final String JSON_WITH_URIS =
+ "{\n"
+ + " \"type\": \"azure\",\n"
+ + " \"uris\": [\"azure://datacontainer2/wikipedia.json\"]\n"
+ + "}";
+
+ private static final String JSON_WITH_PREFIXES =
+ "{\n"
+ + " \"type\": \"azure\",\n"
+ + " \"prefixes\": [\"azure://datacontainer2\"]\n"
+ + "}";
+
+ private static final String JSON_WITH_OBJECTS =
+ "{\n"
+ + " \"type\": \"azure\",\n"
+ + " \"objects\": [\n"
+ + " { \"bucket\": \"container1\", \"path\": \"bar/file1.json\"},\n"
+ + " { \"bucket\": \"conatiner2\", \"path\": \"foo/file2.json\"}\n"
+ + " ]\n"
+ + "}";
private static final String JSON_WITH_URIS_AND_SYSFIELDS =
"{\n"
- + " \"type\": \"azure\",\n"
- + " \"uris\": [\"azure://datacontainer2/wikipedia.json\"],\n"
- + " \"systemFields\": [\"__file_uri\"]\n"
+ + " \"type\": \"azure\",\n"
+ + " \"uris\": [\"azure://datacontainer2/wikipedia.json\"],\n"
+ + " \"systemFields\": [\"__file_uri\"]\n"
+ "}";
private static final List EXPECTED_URIS;
@@ -77,7 +82,6 @@ public class AzureInputSourceSerdeTest extends EasyMockSupport
private AzureInputDataConfig inputDataConfig;
private AzureAccountConfig accountConfig;
-
static {
try {
EXPECTED_URIS = ImmutableList.of(new URI("azure://datacontainer2/wikipedia.json"));
@@ -92,7 +96,7 @@ public class AzureInputSourceSerdeTest extends EasyMockSupport
}
}
- @Before
+ @BeforeEach
public void setup()
{
azureStorage = createMock(AzureStorage.class);
@@ -117,7 +121,6 @@ public void test_uriSerde_constructsProperAzureInputSource() throws Exception
objectMapper.writeValueAsBytes(inputSource),
AzureInputSource.class);
verifyInputSourceWithUris(roundTripInputSource);
-
}
@Test
@@ -129,12 +132,12 @@ public void test_uriAndSystemFieldsSerde_constructsProperAzureInputSource() thro
objectMapper.setInjectableValues(injectableValues);
final AzureInputSource inputSource = objectMapper.readValue(JSON_WITH_URIS_AND_SYSFIELDS, AzureInputSource.class);
- Assert.assertEquals(Collections.singleton(SystemField.URI), inputSource.getConfiguredSystemFields());
+ assertEquals(Collections.singleton(SystemField.URI), inputSource.getConfiguredSystemFields());
final AzureInputSource roundTripInputSource = objectMapper.readValue(
objectMapper.writeValueAsBytes(inputSource),
AzureInputSource.class);
- Assert.assertEquals(Collections.singleton(SystemField.URI), roundTripInputSource.getConfiguredSystemFields());
+ assertEquals(Collections.singleton(SystemField.URI), roundTripInputSource.getConfiguredSystemFields());
}
@Test
@@ -153,7 +156,6 @@ public void test_prefixSerde_constructsProperAzureInputSource() throws Exception
objectMapper.writeValueAsBytes(inputSource),
AzureInputSource.class);
verifyInputSourceWithPrefixes(roundTripInputSource);
-
}
@Test
@@ -186,24 +188,22 @@ private InjectableValues.Std initInjectableValues()
private static void verifyInputSourceWithUris(final AzureInputSource inputSource)
{
-
- Assert.assertEquals(EXPECTED_URIS, inputSource.getUris());
- Assert.assertNull(inputSource.getPrefixes());
- Assert.assertNull(inputSource.getObjects());
+ assertEquals(EXPECTED_URIS, inputSource.getUris());
+ assertNull(inputSource.getPrefixes());
+ assertNull(inputSource.getObjects());
}
private static void verifyInputSourceWithPrefixes(final AzureInputSource inputSource)
{
-
- Assert.assertNull(inputSource.getUris());
- Assert.assertEquals(EXPECTED_PREFIXES, inputSource.getPrefixes());
- Assert.assertNull(inputSource.getObjects());
+ assertNull(inputSource.getUris());
+ assertEquals(EXPECTED_PREFIXES, inputSource.getPrefixes());
+ assertNull(inputSource.getObjects());
}
private static void verifyInputSourceWithObjects(final AzureInputSource inputSource)
{
- Assert.assertNull(inputSource.getUris());
- Assert.assertNull(inputSource.getPrefixes());
- Assert.assertEquals(EXPECTED_CLOUD_OBJECTS, inputSource.getObjects());
+ assertNull(inputSource.getUris());
+ assertNull(inputSource.getPrefixes());
+ assertEquals(EXPECTED_CLOUD_OBJECTS, inputSource.getObjects());
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceTest.java
index a7cb7c708c6ba..531fdf1441093 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureInputSourceTest.java
@@ -42,10 +42,9 @@
import org.apache.druid.storage.azure.blob.CloudBlobHolder;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
import java.net.URI;
import java.nio.file.FileSystems;
@@ -57,6 +56,11 @@
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
public class AzureInputSourceTest extends EasyMockSupport
{
private static final String CONTAINER_NAME = "container";
@@ -84,12 +88,10 @@ public class AzureInputSourceTest extends EasyMockSupport
private AzureInputDataConfig inputDataConfig;
private InputSplit> inputSplit;
- private AzureEntity azureEntity1;
- private CloudBlobHolder cloudBlobDruid1;
+ private AzureEntity azureEntity;
+ private CloudBlobHolder cloudBlobDruid;
private AzureCloudBlobIterable azureCloudBlobIterable;
- private AzureInputSource azureInputSource;
-
static {
try {
PREFIX_URI = new URI(AzureInputSource.SCHEME + "://" + CONTAINER_NAME + "/" + BLOB_NAME);
@@ -99,45 +101,51 @@ public class AzureInputSourceTest extends EasyMockSupport
}
}
- @Before
+ @BeforeEach
public void setup()
{
storage = createMock(AzureStorage.class);
entityFactory = createMock(AzureEntityFactory.class);
inputSplit = createMock(InputSplit.class);
- azureEntity1 = createMock(AzureEntity.class);
+ azureEntity = createMock(AzureEntity.class);
azureCloudBlobIterableFactory = createMock(AzureCloudBlobIterableFactory.class);
inputDataConfig = createMock(AzureInputDataConfig.class);
- cloudBlobDruid1 = createMock(CloudBlobHolder.class);
+ cloudBlobDruid = createMock(CloudBlobHolder.class);
azureCloudBlobIterable = createMock(AzureCloudBlobIterable.class);
}
- @Test(expected = IllegalArgumentException.class)
+ @Test
public void test_constructor_emptyUrisEmptyPrefixesEmptyObjects_throwsIllegalArgumentException()
{
replayAll();
- azureInputSource = new AzureInputSource(
- storage,
- entityFactory,
- azureCloudBlobIterableFactory,
- inputDataConfig,
- EMPTY_URIS,
- EMPTY_PREFIXES,
- EMPTY_OBJECTS,
- null,
- null
+
+ //noinspection ResultOfObjectAllocationIgnored
+ assertThrows(
+ IllegalArgumentException.class,
+ () -> new AzureInputSource(
+ storage,
+ entityFactory,
+ azureCloudBlobIterableFactory,
+ inputDataConfig,
+ EMPTY_URIS,
+ EMPTY_PREFIXES,
+ EMPTY_OBJECTS,
+ null,
+ null
+ )
);
}
@Test
public void test_createEntity_returnsExpectedEntity()
{
- EasyMock.expect(entityFactory.create(CLOUD_OBJECT_LOCATION_1, storage, AzureInputSource.SCHEME)).andReturn(azureEntity1);
+ EasyMock.expect(entityFactory.create(CLOUD_OBJECT_LOCATION_1, storage, AzureInputSource.SCHEME))
+ .andReturn(azureEntity);
EasyMock.expect(inputSplit.get()).andReturn(ImmutableList.of(CLOUD_OBJECT_LOCATION_1)).times(2);
replayAll();
List objects = ImmutableList.of(CLOUD_OBJECT_LOCATION_1);
- azureInputSource = new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -149,9 +157,9 @@ public void test_createEntity_returnsExpectedEntity()
null
);
- Assert.assertEquals(1, inputSplit.get().size());
+ assertEquals(1, inputSplit.get().size());
AzureEntity actualAzureEntity = azureInputSource.createEntity(inputSplit.get().get(0));
- Assert.assertSame(azureEntity1, actualAzureEntity);
+ assertSame(azureEntity, actualAzureEntity);
verifyAll();
}
@@ -160,18 +168,18 @@ public void test_createSplits_successfullyCreatesCloudLocation_returnsExpectedLo
{
List prefixes = ImmutableList.of(PREFIX_URI);
List> expectedCloudLocations = ImmutableList.of(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
- List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid1);
+ List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid);
Iterator expectedCloudBlobsIterator = expectedCloudBlobs.iterator();
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_LISTING_LENGTH);
- EasyMock.expect(azureCloudBlobIterableFactory.create(prefixes, MAX_LISTING_LENGTH, storage)).andReturn(
- azureCloudBlobIterable);
+ EasyMock.expect(azureCloudBlobIterableFactory.create(prefixes, MAX_LISTING_LENGTH, storage))
+ .andReturn(azureCloudBlobIterable);
EasyMock.expect(azureCloudBlobIterable.iterator()).andReturn(expectedCloudBlobsIterator);
- EasyMock.expect(cloudBlobDruid1.getContainerName()).andReturn(CONTAINER).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getName()).andReturn(BLOB_PATH).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getBlobLength()).andReturn(100L).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getContainerName()).andReturn(CONTAINER).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getName()).andReturn(BLOB_PATH).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getBlobLength()).andReturn(100L).anyTimes();
replayAll();
- azureInputSource = new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -191,7 +199,7 @@ public void test_createSplits_successfullyCreatesCloudLocation_returnsExpectedLo
List> actualCloudLocationList = cloudObjectStream.map(InputSplit::get)
.collect(Collectors.toList());
verifyAll();
- Assert.assertEquals(expectedCloudLocations, actualCloudLocationList);
+ assertEquals(expectedCloudLocations, actualCloudLocationList);
}
@Test
@@ -199,7 +207,7 @@ public void test_getPrefixesSplitStream_withObjectGlob_successfullyCreatesCloudL
{
List prefixes = ImmutableList.of(PREFIX_URI);
List> expectedCloudLocations = ImmutableList.of(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
- List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid1);
+ List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid);
Iterator expectedCloudBlobsIterator = expectedCloudBlobs.iterator();
String objectGlob = "**.csv";
@@ -211,16 +219,16 @@ public void test_getPrefixesSplitStream_withObjectGlob_successfullyCreatesCloudL
);
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_LISTING_LENGTH);
- EasyMock.expect(azureCloudBlobIterableFactory.create(prefixes, MAX_LISTING_LENGTH, storage)).andReturn(
- azureCloudBlobIterable);
+ EasyMock.expect(azureCloudBlobIterableFactory.create(prefixes, MAX_LISTING_LENGTH, storage))
+ .andReturn(azureCloudBlobIterable);
EasyMock.expect(azureCloudBlobIterable.iterator()).andReturn(expectedCloudBlobsIterator);
- EasyMock.expect(cloudBlobDruid1.getBlobLength()).andReturn(100L).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getContainerName()).andReturn(CONTAINER).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getName()).andReturn(BLOB_PATH).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getBlobLength()).andReturn(100L).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getContainerName()).andReturn(CONTAINER).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getName()).andReturn(BLOB_PATH).anyTimes();
replayAll();
- azureInputSource = new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -240,7 +248,7 @@ public void test_getPrefixesSplitStream_withObjectGlob_successfullyCreatesCloudL
List> actualCloudLocationList = cloudObjectStream.map(InputSplit::get)
.collect(Collectors.toList());
verifyAll();
- Assert.assertEquals(expectedCloudLocations, actualCloudLocationList);
+ assertEquals(expectedCloudLocations, actualCloudLocationList);
}
@Test
@@ -250,7 +258,7 @@ public void test_withSplit_constructsExpectedInputSource()
EasyMock.expect(inputSplit.get()).andReturn(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
replayAll();
- azureInputSource = new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -263,7 +271,7 @@ public void test_withSplit_constructsExpectedInputSource()
);
SplittableInputSource> newInputSource = azureInputSource.withSplit(inputSplit);
- Assert.assertTrue(newInputSource.isSplittable());
+ assertTrue(newInputSource.isSplittable());
verifyAll();
}
@@ -271,7 +279,7 @@ public void test_withSplit_constructsExpectedInputSource()
public void test_toString_returnsExpectedString()
{
List prefixes = ImmutableList.of(PREFIX_URI);
- azureInputSource = new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -284,7 +292,7 @@ public void test_toString_returnsExpectedString()
);
String actualToString = azureInputSource.toString();
- Assert.assertEquals(
+ assertEquals(
"AzureInputSource{uris=[], prefixes=[azure://container/blob], objects=[], objectGlob=null}",
actualToString
);
@@ -294,7 +302,7 @@ public void test_toString_returnsExpectedString()
public void test_toString_withAllSystemFields_returnsExpectedString()
{
List prefixes = ImmutableList.of(PREFIX_URI);
- azureInputSource = new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -307,7 +315,7 @@ public void test_toString_withAllSystemFields_returnsExpectedString()
);
String actualToString = azureInputSource.toString();
- Assert.assertEquals(
+ assertEquals(
"AzureInputSource{"
+ "uris=[], "
+ "prefixes=[azure://container/blob], "
@@ -323,7 +331,7 @@ public void test_toString_withAllSystemFields_returnsExpectedString()
public void test_getTypes_returnsExpectedTypes()
{
List prefixes = ImmutableList.of(PREFIX_URI);
- azureInputSource = new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -334,13 +342,13 @@ public void test_getTypes_returnsExpectedTypes()
null,
null
);
- Assert.assertEquals(ImmutableSet.of(AzureInputSource.SCHEME), azureInputSource.getTypes());
+ assertEquals(ImmutableSet.of(AzureInputSource.SCHEME), azureInputSource.getTypes());
}
@Test
public void test_systemFields()
{
- azureInputSource = (AzureInputSource) new AzureInputSource(
+ final AzureInputSource azureInputSource = new AzureInputSource(
storage,
entityFactory,
azureCloudBlobIterableFactory,
@@ -352,7 +360,7 @@ public void test_systemFields()
new SystemFields(EnumSet.of(SystemField.URI, SystemField.BUCKET, SystemField.PATH))
);
- Assert.assertEquals(
+ assertEquals(
EnumSet.of(SystemField.URI, SystemField.BUCKET, SystemField.PATH),
azureInputSource.getConfiguredSystemFields()
);
@@ -364,9 +372,9 @@ public void test_systemFields()
(containerName, blobPath, storage) -> null
);
- Assert.assertEquals("azure://foo/bar", azureInputSource.getSystemFieldValue(entity, SystemField.URI));
- Assert.assertEquals("foo", azureInputSource.getSystemFieldValue(entity, SystemField.BUCKET));
- Assert.assertEquals("bar", azureInputSource.getSystemFieldValue(entity, SystemField.PATH));
+ assertEquals("azure://foo/bar", azureInputSource.getSystemFieldValue(entity, SystemField.URI));
+ assertEquals("foo", azureInputSource.getSystemFieldValue(entity, SystemField.BUCKET));
+ assertEquals("bar", azureInputSource.getSystemFieldValue(entity, SystemField.PATH));
}
@Test
@@ -375,7 +383,11 @@ public void abidesEqualsContract()
EqualsVerifier.forClass(AzureInputSource.class)
.usingGetClass()
.withPrefabValues(Logger.class, new Logger(AzureStorage.class), new Logger(AzureStorage.class))
- .withPrefabValues(BlobContainerClient.class, new BlobContainerClientBuilder().buildClient(), new BlobContainerClientBuilder().buildClient())
+ .withPrefabValues(
+ BlobContainerClient.class,
+ new BlobContainerClientBuilder().buildClient(),
+ new BlobContainerClientBuilder().buildClient()
+ )
.withPrefabValues(AzureStorage.class, new AzureStorage(null, null), new AzureStorage(null, null))
.withNonnullFields("storage")
.withNonnullFields("entityFactory")
@@ -386,7 +398,7 @@ public void abidesEqualsContract()
.verify();
}
- @After
+ @AfterEach
public void cleanup()
{
resetAll();
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureStorageAccountInputSourceTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureStorageAccountInputSourceTest.java
index 8d17d9ba01e60..d50472ed91948 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureStorageAccountInputSourceTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/data/input/azure/AzureStorageAccountInputSourceTest.java
@@ -45,10 +45,9 @@
import org.apache.druid.storage.azure.blob.CloudBlobHolder;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
import java.net.URI;
import java.nio.file.FileSystems;
@@ -57,23 +56,28 @@
import java.util.EnumSet;
import java.util.Iterator;
import java.util.List;
+import java.util.Locale;
import java.util.stream.Collectors;
import java.util.stream.Stream;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
public class AzureStorageAccountInputSourceTest extends EasyMockSupport
{
private static final String BLOB_NAME = "blob";
private static final URI PREFIX_URI;
- private final List EMPTY_URIS = ImmutableList.of();
- private final List EMPTY_PREFIXES = ImmutableList.of();
- private final List EMPTY_OBJECTS = ImmutableList.of();
private static final String STORAGE_ACCOUNT = "STORAGE_ACCOUNT";
private static final String DEFAULT_STORAGE_ACCOUNT = "DEFAULT_STORAGE_ACCOUNT";
private static final String CONTAINER = "CONTAINER";
private static final String BLOB_PATH = "BLOB_PATH.csv";
- private static final CloudObjectLocation CLOUD_OBJECT_LOCATION_1 = new CloudObjectLocation(STORAGE_ACCOUNT, CONTAINER + "/" + BLOB_PATH);
+ private static final CloudObjectLocation CLOUD_OBJECT_LOCATION_1 = new CloudObjectLocation(
+ STORAGE_ACCOUNT,
+ CONTAINER + "/" + BLOB_PATH
+ );
private static final int MAX_LISTING_LENGTH = 10;
-
private static final InputFormat INPUT_FORMAT = new JsonInputFormat(
new JSONPathSpec(true, null),
null,
@@ -82,72 +86,85 @@ public class AzureStorageAccountInputSourceTest extends EasyMockSupport
null
);
+ private final List EMPTY_URIS = ImmutableList.of();
+ private final List EMPTY_PREFIXES = ImmutableList.of();
+ private final List EMPTY_OBJECTS = ImmutableList.of();
private AzureStorage storage;
private AzureEntityFactory entityFactory;
private AzureCloudBlobIterableFactory azureCloudBlobIterableFactory;
private AzureInputDataConfig inputDataConfig;
private AzureStorageAccountInputSourceConfig azureStorageAccountInputSourceConfig;
private AzureAccountConfig azureAccountConfig;
-
private InputSplit> inputSplit;
- private AzureEntity azureEntity1;
- private CloudBlobHolder cloudBlobDruid1;
+ private AzureEntity azureEntity;
+ private CloudBlobHolder cloudBlobDruid;
private AzureCloudBlobIterable azureCloudBlobIterable;
- private AzureStorageAccountInputSource azureInputSource;
-
static {
try {
- PREFIX_URI = new URI(AzureStorageAccountInputSource.SCHEME + "://" + STORAGE_ACCOUNT + "/" + CONTAINER + "/" + BLOB_NAME);
+ PREFIX_URI = new URI(
+ AzureStorageAccountInputSource.SCHEME + "://" + STORAGE_ACCOUNT + "/" + CONTAINER + "/" + BLOB_NAME
+ );
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
- @Before
+ @BeforeEach
public void setup()
{
storage = createMock(AzureStorage.class);
entityFactory = createMock(AzureEntityFactory.class);
inputSplit = createMock(InputSplit.class);
- azureEntity1 = createMock(AzureEntity.class);
+ azureEntity = createMock(AzureEntity.class);
azureCloudBlobIterableFactory = createMock(AzureCloudBlobIterableFactory.class);
inputDataConfig = createMock(AzureInputDataConfig.class);
- cloudBlobDruid1 = createMock(CloudBlobHolder.class);
+ cloudBlobDruid = createMock(CloudBlobHolder.class);
azureCloudBlobIterable = createMock(AzureCloudBlobIterable.class);
azureStorageAccountInputSourceConfig = createMock(AzureStorageAccountInputSourceConfig.class);
azureAccountConfig = createMock(AzureAccountConfig.class);
EasyMock.expect(azureAccountConfig.getAccount()).andReturn(DEFAULT_STORAGE_ACCOUNT).anyTimes();
}
- @Test(expected = IllegalArgumentException.class)
+ @Test
public void test_constructor_emptyUrisEmptyPrefixesEmptyObjects_throwsIllegalArgumentException()
{
replayAll();
- azureInputSource = new AzureStorageAccountInputSource(
- entityFactory,
- azureCloudBlobIterableFactory,
- inputDataConfig,
- azureAccountConfig,
- EMPTY_URIS,
- EMPTY_PREFIXES,
- EMPTY_OBJECTS,
- null,
- azureStorageAccountInputSourceConfig,
- null
+
+ //noinspection ResultOfObjectAllocationIgnored
+ assertThrows(
+ IllegalArgumentException.class,
+ () -> new AzureStorageAccountInputSource(
+ entityFactory,
+ azureCloudBlobIterableFactory,
+ inputDataConfig,
+ azureAccountConfig,
+ EMPTY_URIS,
+ EMPTY_PREFIXES,
+ EMPTY_OBJECTS,
+ null,
+ azureStorageAccountInputSourceConfig,
+ null
+ )
);
}
@Test
public void test_createEntity_returnsExpectedEntity()
{
- EasyMock.expect(entityFactory.create(EasyMock.eq(CLOUD_OBJECT_LOCATION_1), EasyMock.anyObject(AzureStorage.class), EasyMock.eq(AzureStorageAccountInputSource.SCHEME))).andReturn(azureEntity1);
+ EasyMock.expect(
+ entityFactory.create(
+ EasyMock.eq(CLOUD_OBJECT_LOCATION_1),
+ EasyMock.anyObject(AzureStorage.class),
+ EasyMock.eq(AzureStorageAccountInputSource.SCHEME)
+ )
+ ).andReturn(azureEntity);
EasyMock.expect(inputSplit.get()).andReturn(ImmutableList.of(CLOUD_OBJECT_LOCATION_1)).times(2);
replayAll();
List objects = ImmutableList.of(CLOUD_OBJECT_LOCATION_1);
- azureInputSource = new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
@@ -160,9 +177,9 @@ public void test_createEntity_returnsExpectedEntity()
null
);
- Assert.assertEquals(1, inputSplit.get().size());
+ assertEquals(1, inputSplit.get().size());
AzureEntity actualAzureEntity = azureInputSource.createEntity(inputSplit.get().get(0));
- Assert.assertSame(azureEntity1, actualAzureEntity);
+ assertSame(azureEntity, actualAzureEntity);
verifyAll();
}
@@ -171,19 +188,24 @@ public void test_createSplits_successfullyCreatesCloudLocation_returnsExpectedLo
{
List prefixes = ImmutableList.of(PREFIX_URI);
List> expectedCloudLocations = ImmutableList.of(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
- List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid1);
+ List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid);
Iterator expectedCloudBlobsIterator = expectedCloudBlobs.iterator();
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_LISTING_LENGTH);
- EasyMock.expect(azureCloudBlobIterableFactory.create(EasyMock.eq(prefixes), EasyMock.eq(MAX_LISTING_LENGTH), EasyMock.anyObject(AzureStorage.class))).andReturn(
- azureCloudBlobIterable);
+ EasyMock.expect(
+ azureCloudBlobIterableFactory.create(
+ EasyMock.eq(prefixes),
+ EasyMock.eq(MAX_LISTING_LENGTH),
+ EasyMock.anyObject(AzureStorage.class)
+ )
+ ).andReturn(azureCloudBlobIterable);
EasyMock.expect(azureCloudBlobIterable.iterator()).andReturn(expectedCloudBlobsIterator);
- EasyMock.expect(cloudBlobDruid1.getStorageAccount()).andReturn(STORAGE_ACCOUNT).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getContainerName()).andReturn(CONTAINER).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getName()).andReturn(BLOB_PATH).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getBlobLength()).andReturn(100L).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getStorageAccount()).andReturn(STORAGE_ACCOUNT).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getContainerName()).andReturn(CONTAINER).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getName()).andReturn(BLOB_PATH).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getBlobLength()).andReturn(100L).anyTimes();
replayAll();
- azureInputSource = new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
@@ -201,10 +223,11 @@ public void test_createSplits_successfullyCreatesCloudLocation_returnsExpectedLo
new MaxSizeSplitHintSpec(null, 1)
);
- List> actualCloudLocationList = cloudObjectStream.map(InputSplit::get)
+ List> actualCloudLocationList = cloudObjectStream
+ .map(InputSplit::get)
.collect(Collectors.toList());
verifyAll();
- Assert.assertEquals(expectedCloudLocations, actualCloudLocationList);
+ assertEquals(expectedCloudLocations, actualCloudLocationList);
}
@Test
@@ -212,7 +235,7 @@ public void test_getPrefixesSplitStream_withObjectGlob_successfullyCreatesCloudL
{
List prefixes = ImmutableList.of(PREFIX_URI);
List> expectedCloudLocations = ImmutableList.of(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
- List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid1);
+ List expectedCloudBlobs = ImmutableList.of(cloudBlobDruid);
Iterator expectedCloudBlobsIterator = expectedCloudBlobs.iterator();
String objectGlob = "**.csv";
@@ -224,17 +247,22 @@ public void test_getPrefixesSplitStream_withObjectGlob_successfullyCreatesCloudL
);
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_LISTING_LENGTH);
- EasyMock.expect(azureCloudBlobIterableFactory.create(EasyMock.eq(prefixes), EasyMock.eq(MAX_LISTING_LENGTH), EasyMock.anyObject(AzureStorage.class))).andReturn(
- azureCloudBlobIterable);
+ EasyMock.expect(
+ azureCloudBlobIterableFactory.create(
+ EasyMock.eq(prefixes),
+ EasyMock.eq(MAX_LISTING_LENGTH),
+ EasyMock.anyObject(AzureStorage.class)
+ )
+ ).andReturn(azureCloudBlobIterable);
EasyMock.expect(azureCloudBlobIterable.iterator()).andReturn(expectedCloudBlobsIterator);
- EasyMock.expect(cloudBlobDruid1.getStorageAccount()).andReturn(STORAGE_ACCOUNT).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getBlobLength()).andReturn(100L).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getContainerName()).andReturn(CONTAINER).anyTimes();
- EasyMock.expect(cloudBlobDruid1.getName()).andReturn(BLOB_PATH).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getStorageAccount()).andReturn(STORAGE_ACCOUNT).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getBlobLength()).andReturn(100L).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getContainerName()).andReturn(CONTAINER).anyTimes();
+ EasyMock.expect(cloudBlobDruid.getName()).andReturn(BLOB_PATH).anyTimes();
replayAll();
- azureInputSource = new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
@@ -252,10 +280,11 @@ public void test_getPrefixesSplitStream_withObjectGlob_successfullyCreatesCloudL
new MaxSizeSplitHintSpec(null, 1)
);
- List> actualCloudLocationList = cloudObjectStream.map(InputSplit::get)
+ List> actualCloudLocationList = cloudObjectStream
+ .map(InputSplit::get)
.collect(Collectors.toList());
verifyAll();
- Assert.assertEquals(expectedCloudLocations, actualCloudLocationList);
+ assertEquals(expectedCloudLocations, actualCloudLocationList);
}
@Test
@@ -265,7 +294,7 @@ public void test_withSplit_constructsExpectedInputSource()
EasyMock.expect(inputSplit.get()).andReturn(ImmutableList.of(CLOUD_OBJECT_LOCATION_1));
replayAll();
- azureInputSource = new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
@@ -279,30 +308,33 @@ public void test_withSplit_constructsExpectedInputSource()
);
SplittableInputSource> newInputSource = azureInputSource.withSplit(inputSplit);
- Assert.assertTrue(newInputSource.isSplittable());
+ assertTrue(newInputSource.isSplittable());
verifyAll();
}
@Test
public void test_toString_returnsExpectedString()
{
- List prefixes = ImmutableList.of(PREFIX_URI);
- azureInputSource = new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
azureAccountConfig,
EMPTY_URIS,
- prefixes,
+ ImmutableList.of(PREFIX_URI),
EMPTY_OBJECTS,
null,
azureStorageAccountInputSourceConfig,
null
);
- String azureStorageAccountInputSourceString = azureInputSource.toString();
- Assert.assertEquals(
- "AzureStorageAccountInputSource{uris=[], prefixes=[azureStorage://STORAGE_ACCOUNT/CONTAINER/blob], objects=[], objectGlob=null, azureStorageAccountInputSourceConfig=" + azureStorageAccountInputSourceConfig + "}",
- azureStorageAccountInputSourceString
+
+ assertEquals(
+ String.format(
+ Locale.ENGLISH,
+ "AzureStorageAccountInputSource{uris=[], prefixes=[azureStorage://STORAGE_ACCOUNT/CONTAINER/blob], objects=[], objectGlob=null, azureStorageAccountInputSourceConfig=%s}",
+ azureStorageAccountInputSourceConfig
+ ),
+ azureInputSource.toString()
);
}
@@ -310,7 +342,7 @@ public void test_toString_returnsExpectedString()
public void test_toString_withAllSystemFields_returnsExpectedString()
{
List prefixes = ImmutableList.of(PREFIX_URI);
- azureInputSource = new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
@@ -325,15 +357,15 @@ public void test_toString_withAllSystemFields_returnsExpectedString()
String azureStorageAccountInputSourceString = azureInputSource.toString();
- Assert.assertEquals(
+ assertEquals(
"AzureStorageAccountInputSource{"
- + "uris=[], "
- + "prefixes=[azureStorage://STORAGE_ACCOUNT/CONTAINER/blob], "
- + "objects=[], "
- + "objectGlob=null, "
- + "azureStorageAccountInputSourceConfig=" + azureStorageAccountInputSourceConfig + ", "
- + "systemFields=[__file_uri, __file_bucket, __file_path]"
- + "}",
+ + "uris=[], "
+ + "prefixes=[azureStorage://STORAGE_ACCOUNT/CONTAINER/blob], "
+ + "objects=[], "
+ + "objectGlob=null, "
+ + "azureStorageAccountInputSourceConfig=" + azureStorageAccountInputSourceConfig + ", "
+ + "systemFields=[__file_uri, __file_bucket, __file_path]"
+ + "}",
azureStorageAccountInputSourceString
);
}
@@ -342,7 +374,7 @@ public void test_toString_withAllSystemFields_returnsExpectedString()
public void test_getTypes_returnsExpectedTypes()
{
List prefixes = ImmutableList.of(PREFIX_URI);
- azureInputSource = new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
@@ -354,13 +386,13 @@ public void test_getTypes_returnsExpectedTypes()
azureStorageAccountInputSourceConfig,
null
);
- Assert.assertEquals(ImmutableSet.of(AzureStorageAccountInputSource.SCHEME), azureInputSource.getTypes());
+ assertEquals(ImmutableSet.of(AzureStorageAccountInputSource.SCHEME), azureInputSource.getTypes());
}
@Test
public void test_systemFields()
{
- azureInputSource = (AzureStorageAccountInputSource) new AzureStorageAccountInputSource(
+ final AzureStorageAccountInputSource azureInputSource = new AzureStorageAccountInputSource(
entityFactory,
azureCloudBlobIterableFactory,
inputDataConfig,
@@ -373,7 +405,7 @@ public void test_systemFields()
new SystemFields(EnumSet.of(SystemField.URI, SystemField.BUCKET, SystemField.PATH))
);
- Assert.assertEquals(
+ assertEquals(
EnumSet.of(SystemField.URI, SystemField.BUCKET, SystemField.PATH),
azureInputSource.getConfiguredSystemFields()
);
@@ -385,19 +417,31 @@ public void test_systemFields()
(containerName, blobPath, storage) -> null
);
- Assert.assertEquals("azureStorage://foo/container/bar", azureInputSource.getSystemFieldValue(entity, SystemField.URI));
- Assert.assertEquals("foo", azureInputSource.getSystemFieldValue(entity, SystemField.BUCKET));
- Assert.assertEquals("container/bar", azureInputSource.getSystemFieldValue(entity, SystemField.PATH));
+ assertEquals(
+ "azureStorage://foo/container/bar",
+ azureInputSource.getSystemFieldValue(entity, SystemField.URI)
+ );
+ assertEquals("foo", azureInputSource.getSystemFieldValue(entity, SystemField.BUCKET));
+ assertEquals("container/bar", azureInputSource.getSystemFieldValue(entity, SystemField.PATH));
}
@Test
public void abidesEqualsContract()
{
- EqualsVerifier.forClass(AzureStorageAccountInputSource.class)
+ EqualsVerifier
+ .forClass(AzureStorageAccountInputSource.class)
.usingGetClass()
.withPrefabValues(Logger.class, new Logger(AzureStorage.class), new Logger(AzureStorage.class))
- .withPrefabValues(BlobContainerClient.class, new BlobContainerClientBuilder().buildClient(), new BlobContainerClientBuilder().buildClient())
- .withPrefabValues(AzureIngestClientFactory.class, new AzureIngestClientFactory(null, null), new AzureIngestClientFactory(null, null))
+ .withPrefabValues(
+ BlobContainerClient.class,
+ new BlobContainerClientBuilder().buildClient(),
+ new BlobContainerClientBuilder().buildClient()
+ )
+ .withPrefabValues(
+ AzureIngestClientFactory.class,
+ new AzureIngestClientFactory(null, null),
+ new AzureIngestClientFactory(null, null)
+ )
.withIgnoredFields("entityFactory")
.withIgnoredFields("azureCloudBlobIterableFactory")
.withNonnullFields("inputDataConfig")
@@ -415,9 +459,8 @@ public void test_getContainerAndPathFromObjectLocation()
Pair storageLocation = AzureStorageAccountInputSource.getContainerAndPathFromObjectLocation(
CLOUD_OBJECT_LOCATION_1
);
- Assert.assertEquals(CONTAINER, storageLocation.lhs);
- Assert.assertEquals(BLOB_PATH, storageLocation.rhs);
-
+ assertEquals(CONTAINER, storageLocation.lhs);
+ assertEquals(BLOB_PATH, storageLocation.rhs);
}
@Test
@@ -426,12 +469,11 @@ public void test_getContainerAndPathFromObjectLocatio_nullpath()
Pair storageLocation = AzureStorageAccountInputSource.getContainerAndPathFromObjectLocation(
new CloudObjectLocation(STORAGE_ACCOUNT, CONTAINER)
);
- Assert.assertEquals(CONTAINER, storageLocation.lhs);
- Assert.assertEquals("", storageLocation.rhs);
-
+ assertEquals(CONTAINER, storageLocation.lhs);
+ assertEquals("", storageLocation.rhs);
}
- @After
+ @AfterEach
public void cleanup()
{
resetAll();
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureAccountConfigTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureAccountConfigTest.java
index d22f112198ee5..27328e2a97bc8 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureAccountConfigTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureAccountConfigTest.java
@@ -22,8 +22,9 @@
import com.fasterxml.jackson.core.JsonProcessingException;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.jackson.DefaultObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
public class AzureAccountConfigTest
{
@@ -35,8 +36,8 @@ public void test_getBlobStorageEndpoint_endpointSuffixNullAndStorageAccountEndpo
{
AzureAccountConfig config = new AzureAccountConfig();
AzureAccountConfig configSerde = MAPPER.readValue("{}", AzureAccountConfig.class);
- Assert.assertEquals(configSerde, config);
- Assert.assertEquals(AzureUtils.AZURE_STORAGE_HOST_ADDRESS, config.getBlobStorageEndpoint());
+ assertEquals(configSerde, config);
+ assertEquals(AzureUtils.AZURE_STORAGE_HOST_ADDRESS, config.getBlobStorageEndpoint());
}
@Test
@@ -51,8 +52,8 @@ public void test_getBlobStorageEndpoint_endpointSuffixNotNullAndStorageAccountEn
+ "\"endpointSuffix\": \"" + endpointSuffix + "\""
+ "}",
AzureAccountConfig.class);
- Assert.assertEquals(configSerde, config);
- Assert.assertEquals(AzureUtils.BLOB + "." + endpointSuffix, config.getBlobStorageEndpoint());
+ assertEquals(configSerde, config);
+ assertEquals(AzureUtils.BLOB + "." + endpointSuffix, config.getBlobStorageEndpoint());
}
@Test
@@ -70,8 +71,8 @@ public void test_getBlobStorageEndpoint_endpointSuffixNotNullAndStorageAccountEn
+ " \"storageAccountEndpointSuffix\": \"" + storageAccountEndpointSuffix + "\""
+ "}",
AzureAccountConfig.class);
- Assert.assertEquals(configSerde, config);
- Assert.assertEquals(AzureUtils.BLOB + "." + endpointSuffix, config.getBlobStorageEndpoint());
+ assertEquals(configSerde, config);
+ assertEquals(AzureUtils.BLOB + "." + endpointSuffix, config.getBlobStorageEndpoint());
}
@Test
@@ -86,8 +87,8 @@ public void test_getBlobStorageEndpoint_endpointSuffixNullAndStorageAccountEndpo
+ "\"storageAccountEndpointSuffix\": \"" + storageAccountEndpointSuffix + "\""
+ "}",
AzureAccountConfig.class);
- Assert.assertEquals(configSerde, config);
- Assert.assertEquals(storageAccountEndpointSuffix, config.getBlobStorageEndpoint());
+ assertEquals(configSerde, config);
+ assertEquals(storageAccountEndpointSuffix, config.getBlobStorageEndpoint());
}
@Test
@@ -102,7 +103,7 @@ public void test_getManagedIdentityClientId_withValueForManagedIdentityClientId_
+ "\"managedIdentityClientId\": \"" + managedIdentityClientId + "\""
+ "}",
AzureAccountConfig.class);
- Assert.assertEquals(configSerde, config);
- Assert.assertEquals("blah", config.getManagedIdentityClientId());
+ assertEquals(configSerde, config);
+ assertEquals("blah", config.getManagedIdentityClientId());
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java
index f8c7af2470a04..53983e9b9f585 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureByteSourceTest.java
@@ -23,11 +23,13 @@
import com.azure.storage.blob.models.BlobStorageException;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.io.InputStream;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
public class AzureByteSourceTest extends EasyMockSupport
{
private static final long NO_OFFSET = 0L;
@@ -71,8 +73,8 @@ public void test_openStream_withOffset_succeeds() throws IOException, BlobStorag
verifyAll();
}
- @Test(expected = IOException.class)
- public void openStreamWithRecoverableErrorTest() throws BlobStorageException, IOException
+ @Test
+ public void openStreamWithRecoverableErrorTest() throws BlobStorageException
{
final String containerName = "container";
final String blobPath = "/path/to/file";
@@ -80,19 +82,14 @@ public void openStreamWithRecoverableErrorTest() throws BlobStorageException, IO
HttpResponse httpResponse = createMock(HttpResponse.class);
EasyMock.expect(httpResponse.getStatusCode()).andReturn(500).anyTimes();
EasyMock.replay(httpResponse);
- EasyMock.expect(azureStorage.getBlockBlobInputStream(NO_OFFSET, containerName, blobPath)).andThrow(
- new BlobStorageException(
- "",
- httpResponse,
- null
- )
- );
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(NO_OFFSET, containerName, blobPath))
+ .andThrow(new BlobStorageException("", httpResponse, null));
EasyMock.replay(azureStorage);
AzureByteSource byteSource = new AzureByteSource(azureStorage, containerName, blobPath);
- byteSource.openStream();
+ assertThrows(IOException.class, byteSource::openStream);
verifyAll();
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureClientFactoryTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureClientFactoryTest.java
index 1361a9351c0b0..5c4d1c433c843 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureClientFactoryTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureClientFactoryTest.java
@@ -24,11 +24,19 @@
import com.azure.storage.blob.BlobServiceClient;
import com.azure.storage.common.StorageSharedKeyCredential;
import com.google.common.collect.ImmutableMap;
-import org.junit.Assert;
-import org.junit.Test;
+import org.apache.druid.java.util.common.concurrent.Execs;
+import org.junit.jupiter.api.Test;
import java.net.MalformedURLException;
import java.net.URL;
+import java.util.concurrent.CountDownLatch;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
public class AzureClientFactoryTest
{
@@ -41,7 +49,7 @@ public void test_blobServiceClient_accountName()
AzureAccountConfig config = new AzureAccountConfig();
azureClientFactory = new AzureClientFactory(config);
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
- Assert.assertEquals(ACCOUNT, blobServiceClient.getAccountName());
+ assertEquals(ACCOUNT, blobServiceClient.getAccountName());
}
@Test
@@ -54,12 +62,14 @@ public void test_blobServiceClientBuilder_key() throws MalformedURLException
StorageSharedKeyCredential storageSharedKeyCredential = StorageSharedKeyCredential.getSharedKeyCredentialFromPipeline(
blobServiceClient.getHttpPipeline()
);
- Assert.assertNotNull(storageSharedKeyCredential);
+ assertNotNull(storageSharedKeyCredential);
// Azure doesn't let us look at the key in the StorageSharedKeyCredential so make sure the authorization header generated is what we expect.
- Assert.assertEquals(
- new StorageSharedKeyCredential(ACCOUNT, "key").generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of()),
- storageSharedKeyCredential.generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of())
+ assertEquals(
+ new StorageSharedKeyCredential(ACCOUNT, "key")
+ .generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of()),
+ storageSharedKeyCredential
+ .generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of())
);
}
@@ -77,7 +87,7 @@ public void test_blobServiceClientBuilder_sasToken()
}
}
- Assert.assertNotNull(azureSasCredentialPolicy);
+ assertNotNull(azureSasCredentialPolicy);
}
@Test
@@ -94,7 +104,7 @@ public void test_blobServiceClientBuilder_useDefaultCredentialChain()
}
}
- Assert.assertNotNull(bearerTokenAuthenticationPolicy);
+ assertNotNull(bearerTokenAuthenticationPolicy);
}
@Test
@@ -105,7 +115,7 @@ public void test_blobServiceClientBuilder_useCachedClient()
azureClientFactory = new AzureClientFactory(config);
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
BlobServiceClient blobServiceClient2 = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
- Assert.assertEquals(blobServiceClient, blobServiceClient2);
+ assertEquals(blobServiceClient, blobServiceClient2);
}
@Test
@@ -116,7 +126,7 @@ public void test_blobServiceClientBuilder_useNewClientForDifferentRetryCount()
azureClientFactory = new AzureClientFactory(config);
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
BlobServiceClient blobServiceClient2 = azureClientFactory.getBlobServiceClient(1, ACCOUNT);
- Assert.assertNotEquals(blobServiceClient, blobServiceClient2);
+ assertNotEquals(blobServiceClient, blobServiceClient2);
}
@Test
@@ -127,7 +137,7 @@ public void test_blobServiceClientBuilder_useAzureAccountConfig_asDefaultMaxTrie
azureClientFactory = new AzureClientFactory(config);
BlobServiceClient expectedBlobServiceClient = azureClientFactory.getBlobServiceClient(AzureAccountConfig.DEFAULT_MAX_TRIES, ACCOUNT);
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
- Assert.assertEquals(expectedBlobServiceClient, blobServiceClient);
+ assertEquals(expectedBlobServiceClient, blobServiceClient);
}
@Test
@@ -141,7 +151,7 @@ public void test_blobServiceClientBuilder_useAzureAccountConfigWithNonDefaultEnd
URL expectedAccountUrl = new URL(AzureAccountConfig.DEFAULT_PROTOCOL, ACCOUNT + "." + AzureUtils.BLOB + "." + endpointSuffix, "");
azureClientFactory = new AzureClientFactory(config);
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
- Assert.assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
+ assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
}
@Test
@@ -157,7 +167,7 @@ public void test_blobServiceClientBuilder_useAzureAccountConfigWithStorageAccoun
URL expectedAccountUrl = new URL(AzureAccountConfig.DEFAULT_PROTOCOL, ACCOUNT + "." + AzureUtils.BLOB + "." + endpointSuffix, "");
azureClientFactory = new AzureClientFactory(config);
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
- Assert.assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
+ assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
}
@Test
@@ -171,6 +181,56 @@ public void test_blobServiceClientBuilder_useAzureAccountConfigWithStorageAccoun
URL expectedAccountUrl = new URL(AzureAccountConfig.DEFAULT_PROTOCOL, ACCOUNT + "." + storageAccountEndpointSuffix, "");
azureClientFactory = new AzureClientFactory(config);
BlobServiceClient blobServiceClient = azureClientFactory.getBlobServiceClient(null, ACCOUNT);
- Assert.assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
+ assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
+ }
+
+ @Test
+ public void test_concurrent_azureClientFactory_gets() throws Exception
+ {
+ for (int i = 0; i < 10; i++) {
+ concurrentAzureClientFactoryGets();
+ }
+ }
+
+ private void concurrentAzureClientFactoryGets() throws Exception
+ {
+ final int threads = 100;
+ String endpointSuffix = "core.nonDefault.windows.net";
+ String storageAccountEndpointSuffix = "ABC123.blob.storage.azure.net";
+ AzureAccountConfig config = new AzureAccountConfig();
+ config.setKey("key");
+ config.setEndpointSuffix(endpointSuffix);
+ config.setStorageAccountEndpointSuffix(storageAccountEndpointSuffix);
+ final AzureClientFactory localAzureClientFactory = new AzureClientFactory(config);
+ final URL expectedAccountUrl = new URL(
+ AzureAccountConfig.DEFAULT_PROTOCOL,
+ ACCOUNT + "." + storageAccountEndpointSuffix,
+ ""
+ );
+
+ final CountDownLatch latch = new CountDownLatch(threads);
+ ExecutorService executorService = Execs.multiThreaded(threads, "azure-client-fetcher-%d");
+ final AtomicReference failureException = new AtomicReference<>();
+ for (int i = 0; i < threads; i++) {
+ final int retry = i % 2;
+ executorService.submit(() -> {
+ try {
+ latch.countDown();
+ latch.await();
+ BlobServiceClient blobServiceClient = localAzureClientFactory.getBlobServiceClient(retry, ACCOUNT);
+ assertEquals(expectedAccountUrl.toString(), blobServiceClient.getAccountUrl());
+ }
+ catch (Exception e) {
+ failureException.compareAndSet(null, e);
+ }
+ });
+ }
+
+ //noinspection ResultOfMethodCallIgnored
+ executorService.awaitTermination(1000, TimeUnit.MICROSECONDS);
+
+ if (failureException.get() != null) {
+ throw failureException.get();
+ }
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureCloudBlobIteratorTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureCloudBlobIteratorTest.java
index 996028377ed5b..a77c6d104138e 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureCloudBlobIteratorTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureCloudBlobIteratorTest.java
@@ -30,53 +30,54 @@
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.storage.azure.blob.CloudBlobHolder;
import org.easymock.EasyMock;
-import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockExtension;
import org.easymock.EasyMockSupport;
import org.easymock.Mock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
+
import java.net.URI;
+import java.net.URISyntaxException;
import java.util.ArrayList;
import java.util.List;
import java.util.NoSuchElementException;
import java.util.stream.Collectors;
-@RunWith(EasyMockRunner.class)
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
+@ExtendWith(EasyMockExtension.class)
public class AzureCloudBlobIteratorTest extends EasyMockSupport
{
@Mock
private AzureStorage storage;
- private AzureCloudBlobIterator azureCloudBlobIterator;
private final AzureAccountConfig config = new AzureAccountConfig();
private final Integer MAX_TRIES = 3;
private final Integer MAX_LISTING_LENGTH = 10;
private final String CONTAINER = "container";
- private final String STORAGE_ACCOUNT = "storageAccount";
private final String DEFAULT_STORAGE_ACCOUNT = "defaultStorageAccount";
-
- @Before
+ @BeforeEach
public void setup()
{
config.setMaxTries(MAX_TRIES);
config.setAccount(DEFAULT_STORAGE_ACCOUNT);
-
}
@Test
public void test_hasNext_noBlobs_returnsFalse()
{
- azureCloudBlobIterator = new AzureCloudBlobIterator(
+ final AzureCloudBlobIterator azureCloudBlobIterator = new AzureCloudBlobIterator(
storage,
config,
ImmutableList.of(),
1
);
boolean hasNext = azureCloudBlobIterator.hasNext();
- Assert.assertFalse(hasNext);
+ assertFalse(hasNext);
}
@Test
@@ -103,7 +104,7 @@ public void test_next_prefixesWithMultipleBlobsAndSomeDirectories_returnsExpecte
.andReturn(pagedIterable2);
replayAll();
- azureCloudBlobIterator = new AzureCloudBlobIterator(
+ final AzureCloudBlobIterator azureCloudBlobIterator = new AzureCloudBlobIterator(
storage,
config,
prefixes,
@@ -118,16 +119,16 @@ public void test_next_prefixesWithMultipleBlobsAndSomeDirectories_returnsExpecte
new CloudBlobHolder(blobItem, CONTAINER, DEFAULT_STORAGE_ACCOUNT),
new CloudBlobHolder(blobItem2, CONTAINER, DEFAULT_STORAGE_ACCOUNT)
);
- Assert.assertEquals(expectedBlobItems.size(), actualBlobItems.size());
- Assert.assertEquals(
+ assertEquals(expectedBlobItems.size(), actualBlobItems.size());
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet())
);
@@ -150,7 +151,7 @@ public void test_next_prefixesWithMultipleBlobsAndOneDirectory_returnsExpectedBl
replayAll();
- azureCloudBlobIterator = new AzureCloudBlobIterator(
+ final AzureCloudBlobIterator azureCloudBlobIterator = new AzureCloudBlobIterator(
storage,
config,
prefixes,
@@ -165,34 +166,36 @@ public void test_next_prefixesWithMultipleBlobsAndOneDirectory_returnsExpectedBl
new CloudBlobHolder(blobItem, CONTAINER, DEFAULT_STORAGE_ACCOUNT),
new CloudBlobHolder(blobItem2, CONTAINER, DEFAULT_STORAGE_ACCOUNT)
);
- Assert.assertEquals(expectedBlobItems.size(), actualBlobItems.size());
- Assert.assertEquals(
+ assertEquals(expectedBlobItems.size(), actualBlobItems.size());
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet())
);
}
@Test
- public void test_next_prefixesWithMultipleBlobsAndSomeDirectories_returnsExpectedBlobs_azureStorage() throws Exception
+ public void test_next_prefixesWithMultipleBlobsAndSomeDirectories_returnsExpectedBlobs_azureStorage()
+ throws URISyntaxException
{
+ final String storageAccount = "storageAccount";
List prefixes = ImmutableList.of(
- new URI(StringUtils.format("azureStorage://%s/%s/dir1", STORAGE_ACCOUNT, CONTAINER)),
- new URI(StringUtils.format("azureStorage://%s/%s/dir2", STORAGE_ACCOUNT, CONTAINER))
+ new URI(StringUtils.format("azureStorage://%s/%s/dir1", storageAccount, CONTAINER)),
+ new URI(StringUtils.format("azureStorage://%s/%s/dir2", storageAccount, CONTAINER))
);
BlobItem blobItem = new BlobItem().setName("blobName").setProperties(new BlobItemProperties().setContentLength(10L));
SettableSupplier> supplier = new SettableSupplier<>();
supplier.set(new TestPagedResponse<>(ImmutableList.of(blobItem)));
PagedIterable pagedIterable = new PagedIterable<>(supplier);
- EasyMock.expect(storage.listBlobsWithPrefixInContainerSegmented(STORAGE_ACCOUNT, CONTAINER, "dir1", MAX_LISTING_LENGTH, MAX_TRIES))
+ EasyMock.expect(storage.listBlobsWithPrefixInContainerSegmented(storageAccount, CONTAINER, "dir1", MAX_LISTING_LENGTH, MAX_TRIES))
.andReturn(pagedIterable);
BlobItem blobPrefixItem = new BlobItem().setIsPrefix(true).setName("subdir").setProperties(new BlobItemProperties());
@@ -200,11 +203,11 @@ public void test_next_prefixesWithMultipleBlobsAndSomeDirectories_returnsExpecte
SettableSupplier> supplier2 = new SettableSupplier<>();
supplier2.set(new TestPagedResponse<>(ImmutableList.of(blobPrefixItem, blobItem2)));
PagedIterable pagedIterable2 = new PagedIterable<>(supplier2);
- EasyMock.expect(storage.listBlobsWithPrefixInContainerSegmented(STORAGE_ACCOUNT, CONTAINER, "dir2", MAX_LISTING_LENGTH, MAX_TRIES))
+ EasyMock.expect(storage.listBlobsWithPrefixInContainerSegmented(storageAccount, CONTAINER, "dir2", MAX_LISTING_LENGTH, MAX_TRIES))
.andReturn(pagedIterable2);
replayAll();
- azureCloudBlobIterator = new AzureCloudBlobIterator(
+ final AzureCloudBlobIterator azureCloudBlobIterator = new AzureCloudBlobIterator(
storage,
config,
prefixes,
@@ -215,20 +218,21 @@ public void test_next_prefixesWithMultipleBlobsAndSomeDirectories_returnsExpecte
actualBlobItems.add(azureCloudBlobIterator.next());
}
verifyAll();
+
List expectedBlobItems = ImmutableList.of(
- new CloudBlobHolder(blobItem, CONTAINER, STORAGE_ACCOUNT),
- new CloudBlobHolder(blobItem2, CONTAINER, STORAGE_ACCOUNT)
+ new CloudBlobHolder(blobItem, CONTAINER, storageAccount),
+ new CloudBlobHolder(blobItem2, CONTAINER, storageAccount)
);
- Assert.assertEquals(expectedBlobItems.size(), actualBlobItems.size());
- Assert.assertEquals(
+ assertEquals(expectedBlobItems.size(), actualBlobItems.size());
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet())
);
@@ -251,7 +255,7 @@ public void test_next_emptyObjects_skipEmptyObjects() throws Exception
.andReturn(pagedIterable);
replayAll();
- azureCloudBlobIterator = new AzureCloudBlobIterator(
+ final AzureCloudBlobIterator azureCloudBlobIterator = new AzureCloudBlobIterator(
storage,
config,
prefixes,
@@ -265,34 +269,35 @@ public void test_next_emptyObjects_skipEmptyObjects() throws Exception
List expectedBlobItems = ImmutableList.of(
new CloudBlobHolder(blobItem, CONTAINER, DEFAULT_STORAGE_ACCOUNT)
);
- Assert.assertEquals(expectedBlobItems.size(), actualBlobItems.size());
- Assert.assertEquals(
+ assertEquals(expectedBlobItems.size(), actualBlobItems.size());
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getName).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getStorageAccount).collect(Collectors.toSet())
);
- Assert.assertEquals(
+ assertEquals(
expectedBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet()),
actualBlobItems.stream().map(CloudBlobHolder::getContainerName).collect(Collectors.toSet())
);
}
- @Test(expected = NoSuchElementException.class)
+ @Test
public void test_next_emptyPrefixes_throwsNoSuchElementException()
{
- azureCloudBlobIterator = new AzureCloudBlobIterator(
+ final AzureCloudBlobIterator azureCloudBlobIterator = new AzureCloudBlobIterator(
storage,
config,
ImmutableList.of(),
MAX_LISTING_LENGTH
);
- azureCloudBlobIterator.next();
+
+ assertThrows(NoSuchElementException.class, azureCloudBlobIterator::next);
}
- @Test(expected = RE.class)
+ @Test
public void test_fetchNextBatch_moreThanMaxTriesRetryableExceptionsThrownInStorage_throwsREException() throws Exception
{
List prefixes = ImmutableList.of(
@@ -305,19 +310,25 @@ public void test_fetchNextBatch_moreThanMaxTriesRetryableExceptionsThrownInStora
EasyMock.anyString(),
EasyMock.anyInt(),
EasyMock.anyInt()
- )).andThrow(new BlobStorageException("", null, null)).times(3);
+ )).andThrow(new BlobStorageException("", null, null));
replayAll();
- azureCloudBlobIterator = new AzureCloudBlobIterator(
- storage,
- config,
- prefixes,
- MAX_LISTING_LENGTH
+
+ //noinspection ResultOfObjectAllocationIgnored
+ assertThrows(
+ RE.class,
+ () -> new AzureCloudBlobIterator(
+ storage,
+ config,
+ prefixes,
+ MAX_LISTING_LENGTH
+ )
);
+
verifyAll();
}
- @Test(expected = RE.class)
+ @Test
public void test_fetchNextBatch_nonRetryableExceptionThrownInStorage_throwsREException() throws Exception
{
List prefixes = ImmutableList.of(
@@ -330,13 +341,20 @@ public void test_fetchNextBatch_nonRetryableExceptionThrownInStorage_throwsREExc
EasyMock.anyInt(),
EasyMock.anyInt()
)).andThrow(new RuntimeException(""));
+
replayAll();
- azureCloudBlobIterator = new AzureCloudBlobIterator(
- storage,
- config,
- prefixes,
- MAX_LISTING_LENGTH
+
+ //noinspection ResultOfObjectAllocationIgnored
+ assertThrows(
+ RE.class,
+ () -> new AzureCloudBlobIterator(
+ storage,
+ config,
+ prefixes,
+ MAX_LISTING_LENGTH
+ )
);
+
verifyAll();
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentKillerTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentKillerTest.java
index 43844c0c7cb08..40be9737d8d14 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentKillerTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentKillerTest.java
@@ -29,13 +29,12 @@
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.storage.azure.blob.CloudBlobHolder;
import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.apache.druid.timeline.partition.LinearShardSpec;
import org.easymock.Capture;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
import java.io.IOException;
import java.net.URI;
@@ -44,6 +43,10 @@
import java.util.HashSet;
import java.util.List;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
public class AzureDataSegmentKillerTest extends EasyMockSupport
{
private static final String CONTAINER_NAME = "container";
@@ -70,7 +73,7 @@ public class AzureDataSegmentKillerTest extends EasyMockSupport
ImmutableMap.of("containerName", CONTAINER_NAME, "blobPath", BLOB_PATH),
null,
null,
- NoneShardSpec.instance(),
+ new LinearShardSpec(0),
0,
1
);
@@ -82,7 +85,7 @@ public class AzureDataSegmentKillerTest extends EasyMockSupport
ImmutableMap.of("containerName", CONTAINER_NAME, "blobPath", BLOB_PATH_2),
null,
null,
- NoneShardSpec.instance(),
+ new LinearShardSpec(0),
0,
1
);
@@ -93,7 +96,7 @@ public class AzureDataSegmentKillerTest extends EasyMockSupport
private AzureStorage azureStorage;
private AzureCloudBlobIterableFactory azureCloudBlobIterableFactory;
- @Before
+ @BeforeEach
public void before()
{
segmentConfig = createMock(AzureDataSegmentConfig.class);
@@ -106,7 +109,6 @@ public void before()
@Test
public void killTest() throws SegmentLoadingException, BlobStorageException
{
-
List deletedFiles = new ArrayList<>();
final String dirPath = Paths.get(BLOB_PATH).getParent().toString();
@@ -114,47 +116,67 @@ public void killTest() throws SegmentLoadingException, BlobStorageException
replayAll();
- AzureDataSegmentKiller killer = new AzureDataSegmentKiller(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
+ final AzureDataSegmentKiller killer = new AzureDataSegmentKiller(
+ segmentConfig,
+ inputDataConfig,
+ accountConfig,
+ azureStorage,
+ azureCloudBlobIterableFactory
+ );
killer.kill(DATA_SEGMENT);
verifyAll();
}
- @Test(expected = SegmentLoadingException.class)
+ @Test
public void test_kill_StorageExceptionExtendedErrorInformationNull_throwsException()
- throws SegmentLoadingException, BlobStorageException
{
+ String dirPath = Paths.get(BLOB_PATH).getParent().toString();
- common_test_kill_StorageExceptionExtendedError_throwsException();
- }
+ EasyMock.expect(azureStorage.emptyCloudBlobDirectory(CONTAINER_NAME, dirPath))
+ .andThrow(new BlobStorageException("", null, null));
- @Test(expected = SegmentLoadingException.class)
- public void test_kill_StorageExceptionExtendedErrorInformationNotNull_throwsException()
- throws SegmentLoadingException, BlobStorageException
- {
+ replayAll();
+
+ final AzureDataSegmentKiller killer = new AzureDataSegmentKiller(
+ segmentConfig,
+ inputDataConfig,
+ accountConfig,
+ azureStorage,
+ azureCloudBlobIterableFactory
+ );
- common_test_kill_StorageExceptionExtendedError_throwsException();
+ assertThrows(
+ SegmentLoadingException.class,
+ () -> killer.kill(DATA_SEGMENT)
+ );
+
+ verifyAll();
}
- @Test(expected = RuntimeException.class)
+ @Test
public void test_kill_runtimeException_throwsException()
- throws SegmentLoadingException, BlobStorageException
{
+ final String dirPath = Paths.get(BLOB_PATH).getParent().toString();
- String dirPath = Paths.get(BLOB_PATH).getParent().toString();
-
- EasyMock.expect(azureStorage.emptyCloudBlobDirectory(CONTAINER_NAME, dirPath)).andThrow(
- new RuntimeException(
- ""
- )
- );
+ EasyMock.expect(azureStorage.emptyCloudBlobDirectory(CONTAINER_NAME, dirPath))
+ .andThrow(new RuntimeException(""));
replayAll();
- AzureDataSegmentKiller killer = new AzureDataSegmentKiller(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
+ final AzureDataSegmentKiller killer = new AzureDataSegmentKiller(
+ segmentConfig,
+ inputDataConfig,
+ accountConfig,
+ azureStorage,
+ azureCloudBlobIterableFactory
+ );
- killer.kill(DATA_SEGMENT);
+ assertThrows(
+ RuntimeException.class,
+ () -> killer.kill(DATA_SEGMENT)
+ );
verifyAll();
}
@@ -182,7 +204,7 @@ public void test_killAll_segmentConfigWithNullContainerAndPrefix_throwsISEExcept
thrownISEException = true;
}
- Assert.assertTrue(thrownISEException);
+ assertTrue(thrownISEException);
EasyMock.verify(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
}
@@ -194,35 +216,35 @@ public void test_killAll_noException_deletesAllSegments() throws Exception
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_KEYS);
EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- CloudBlobHolder object1 = AzureTestUtils.newCloudBlobHolder(CONTAINER, KEY_1, TIME_0);
- CloudBlobHolder object2 = AzureTestUtils.newCloudBlobHolder(CONTAINER, KEY_2, TIME_1);
+ CloudBlobHolder blob1 = AzureTestUtils.newCloudBlobHolder(CONTAINER, KEY_1, TIME_0);
+ CloudBlobHolder blob2 = AzureTestUtils.newCloudBlobHolder(CONTAINER, KEY_2, TIME_1);
AzureCloudBlobIterable azureCloudBlobIterable = AzureTestUtils.expectListObjects(
azureCloudBlobIterableFactory,
MAX_KEYS,
PREFIX_URI,
- ImmutableList.of(object1, object2),
+ ImmutableList.of(blob1, blob2),
azureStorage
);
- EasyMock.replay(object1, object2);
+ EasyMock.replay(blob1, blob2);
AzureTestUtils.expectDeleteObjects(
azureStorage,
- ImmutableList.of(object1, object2),
+ ImmutableList.of(blob1, blob2),
ImmutableMap.of(),
MAX_TRIES
);
EasyMock.replay(segmentConfig, inputDataConfig, accountConfig, azureCloudBlobIterable, azureCloudBlobIterableFactory, azureStorage);
AzureDataSegmentKiller killer = new AzureDataSegmentKiller(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
killer.killAll();
- EasyMock.verify(segmentConfig, inputDataConfig, accountConfig, object1, object2, azureCloudBlobIterable, azureCloudBlobIterableFactory, azureStorage);
+ EasyMock.verify(segmentConfig, inputDataConfig, accountConfig, blob1, blob2, azureCloudBlobIterable, azureCloudBlobIterableFactory, azureStorage);
}
@Test
public void test_killAll_nonrecoverableExceptionWhenListingObjects_deletesAllSegments()
{
boolean ioExceptionThrown = false;
- CloudBlobHolder object1 = null;
+ CloudBlobHolder cloudBlob = null;
AzureCloudBlobIterable azureCloudBlobIterable = null;
try {
EasyMock.expect(segmentConfig.getContainer()).andReturn(CONTAINER).atLeastOnce();
@@ -230,21 +252,21 @@ public void test_killAll_nonrecoverableExceptionWhenListingObjects_deletesAllSeg
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_KEYS);
EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- object1 = AzureTestUtils.newCloudBlobHolder(CONTAINER, KEY_1, TIME_0);
+ cloudBlob = AzureTestUtils.newCloudBlobHolder(CONTAINER, KEY_1, TIME_0);
azureCloudBlobIterable = AzureTestUtils.expectListObjects(
azureCloudBlobIterableFactory,
MAX_KEYS,
PREFIX_URI,
- ImmutableList.of(object1),
+ ImmutableList.of(cloudBlob),
azureStorage
);
- EasyMock.replay(object1);
+ EasyMock.replay(cloudBlob);
AzureTestUtils.expectDeleteObjects(
azureStorage,
ImmutableList.of(),
- ImmutableMap.of(object1, NON_RECOVERABLE_EXCEPTION),
+ ImmutableMap.of(cloudBlob, NON_RECOVERABLE_EXCEPTION),
MAX_TRIES
);
EasyMock.replay(
@@ -268,41 +290,19 @@ public void test_killAll_nonrecoverableExceptionWhenListingObjects_deletesAllSeg
ioExceptionThrown = true;
}
- Assert.assertTrue(ioExceptionThrown);
+ assertTrue(ioExceptionThrown);
EasyMock.verify(
segmentConfig,
inputDataConfig,
accountConfig,
- object1,
+ cloudBlob,
azureCloudBlobIterable,
azureCloudBlobIterableFactory,
azureStorage
);
}
- private void common_test_kill_StorageExceptionExtendedError_throwsException()
- throws SegmentLoadingException, BlobStorageException
- {
- String dirPath = Paths.get(BLOB_PATH).getParent().toString();
-
- EasyMock.expect(azureStorage.emptyCloudBlobDirectory(CONTAINER_NAME, dirPath)).andThrow(
- new BlobStorageException(
- "",
- null,
- null
- )
- );
-
- replayAll();
-
- AzureDataSegmentKiller killer = new AzureDataSegmentKiller(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
-
- killer.kill(DATA_SEGMENT);
-
- verifyAll();
- }
-
@Test
public void killBatchTest() throws SegmentLoadingException, BlobStorageException
{
@@ -321,42 +321,53 @@ public void killBatchTest() throws SegmentLoadingException, BlobStorageException
verifyAll();
- Assert.assertEquals(
- ImmutableSet.of(BLOB_PATH, BLOB_PATH_2),
- new HashSet<>(deletedFilesCapture.getValue())
- );
+ assertEquals(ImmutableSet.of(BLOB_PATH, BLOB_PATH_2), new HashSet<>(deletedFilesCapture.getValue()));
}
- @Test(expected = RuntimeException.class)
+ @Test
public void test_killBatch_runtimeException()
- throws SegmentLoadingException, BlobStorageException
{
-
EasyMock.expect(azureStorage.batchDeleteFiles(CONTAINER_NAME, ImmutableList.of(BLOB_PATH, BLOB_PATH_2), null))
.andThrow(new RuntimeException(""));
replayAll();
- AzureDataSegmentKiller killer = new AzureDataSegmentKiller(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
+ final AzureDataSegmentKiller killer = new AzureDataSegmentKiller(
+ segmentConfig,
+ inputDataConfig,
+ accountConfig,
+ azureStorage,
+ azureCloudBlobIterableFactory
+ );
- killer.kill(ImmutableList.of(DATA_SEGMENT, DATA_SEGMENT_2));
+ assertThrows(
+ RuntimeException.class,
+ () -> killer.kill(ImmutableList.of(DATA_SEGMENT, DATA_SEGMENT_2))
+ );
verifyAll();
}
- @Test(expected = SegmentLoadingException.class)
+ @Test
public void test_killBatch_SegmentLoadingExceptionOnError()
- throws SegmentLoadingException, BlobStorageException
{
-
EasyMock.expect(azureStorage.batchDeleteFiles(CONTAINER_NAME, ImmutableList.of(BLOB_PATH, BLOB_PATH_2), null))
.andReturn(false);
replayAll();
- AzureDataSegmentKiller killer = new AzureDataSegmentKiller(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
+ AzureDataSegmentKiller killer = new AzureDataSegmentKiller(
+ segmentConfig,
+ inputDataConfig,
+ accountConfig,
+ azureStorage,
+ azureCloudBlobIterableFactory
+ );
- killer.kill(ImmutableList.of(DATA_SEGMENT, DATA_SEGMENT_2));
+ assertThrows(
+ SegmentLoadingException.class,
+ () -> killer.kill(ImmutableList.of(DATA_SEGMENT, DATA_SEGMENT_2))
+ );
verifyAll();
}
@@ -364,7 +375,6 @@ public void test_killBatch_SegmentLoadingExceptionOnError()
@Test
public void killBatch_emptyList() throws SegmentLoadingException, BlobStorageException
{
-
AzureDataSegmentKiller killer = new AzureDataSegmentKiller(segmentConfig, inputDataConfig, accountConfig, azureStorage, azureCloudBlobIterableFactory);
killer.kill(ImmutableList.of());
}
@@ -372,7 +382,6 @@ public void killBatch_emptyList() throws SegmentLoadingException, BlobStorageExc
@Test
public void killBatch_singleSegment() throws SegmentLoadingException, BlobStorageException
{
-
List deletedFiles = new ArrayList<>();
final String dirPath = Paths.get(BLOB_PATH).getParent().toString();
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java
index ebcefd79571de..a53e5bba405dd 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPullerTest.java
@@ -25,26 +25,32 @@
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import java.io.File;
-import java.io.FileInputStream;
import java.io.IOException;
import java.io.InputStream;
+import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+import java.util.zip.ZipEntry;
+import java.util.zip.ZipOutputStream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
public class AzureDataSegmentPullerTest extends EasyMockSupport
{
-
- private static final String SEGMENT_FILE_NAME = "segment";
private static final String CONTAINER_NAME = "container";
private static final String BLOB_PATH = "path/to/storage/index.zip";
- private static final String BLOB_PATH_HADOOP = AzureUtils.AZURE_STORAGE_HOST_ADDRESS + "/path/to/storage/index.zip";
private AzureStorage azureStorage;
private AzureByteSourceFactory byteSourceFactory;
- @Before
+ @BeforeEach
public void before()
{
azureStorage = createMock(AzureStorage.class);
@@ -52,136 +58,135 @@ public void before()
}
@Test
- public void test_getSegmentFiles_success()
- throws SegmentLoadingException, BlobStorageException, IOException
+ public void test_getSegmentFiles_success(@TempDir Path sourcePath, @TempDir Path targetPath)
+ throws IOException, SegmentLoadingException
{
+ final String segmentFileName = "segment";
final String value = "bucket";
- final File pulledFile = AzureTestUtils.createZipTempFile(SEGMENT_FILE_NAME, value);
- final File toDir = FileUtils.createTempDir();
- try {
- final InputStream zipStream = new FileInputStream(pulledFile);
- final AzureAccountConfig config = new AzureAccountConfig();
- EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
- EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream);
+ final File pulledFile = createZipTempFile(sourcePath, segmentFileName, value);
+
+ final InputStream zipStream = Files.newInputStream(pulledFile.toPath());
+ final AzureAccountConfig config = new AzureAccountConfig();
- replayAll();
+ EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage))
+ .andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream);
- AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
+ replayAll();
- FileUtils.FileCopyResult result = puller.getSegmentFiles(CONTAINER_NAME, BLOB_PATH, toDir);
+ AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
- File expected = new File(toDir, SEGMENT_FILE_NAME);
- Assert.assertEquals(value.length(), result.size());
- Assert.assertTrue(expected.exists());
- Assert.assertEquals(value.length(), expected.length());
+ FileUtils.FileCopyResult result = puller.getSegmentFiles(CONTAINER_NAME, BLOB_PATH, targetPath.toFile());
- verifyAll();
- }
- finally {
- pulledFile.delete();
- FileUtils.deleteDirectory(toDir);
- }
+ File expected = new File(targetPath.toFile(), segmentFileName);
+ assertEquals(value.length(), result.size());
+ assertTrue(expected.exists());
+ assertEquals(value.length(), expected.length());
+
+ verifyAll();
}
@Test
- public void test_getSegmentFiles_blobPathIsHadoop_success()
- throws SegmentLoadingException, BlobStorageException, IOException
+ public void test_getSegmentFiles_blobPathIsHadoop_success(@TempDir Path sourcePath, @TempDir Path targetPath)
+ throws IOException, SegmentLoadingException
{
+ final String segmentFileName = "segment";
final String value = "bucket";
- final File pulledFile = AzureTestUtils.createZipTempFile(SEGMENT_FILE_NAME, value);
- final File toDir = FileUtils.createTempDir();
- try {
- final InputStream zipStream = new FileInputStream(pulledFile);
- final AzureAccountConfig config = new AzureAccountConfig();
- EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
- EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream);
+ final File pulledFile = createZipTempFile(sourcePath, segmentFileName, value);
- replayAll();
+ final InputStream zipStream = Files.newInputStream(pulledFile.toPath());
+ final AzureAccountConfig config = new AzureAccountConfig();
- AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
+ EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage))
+ .andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andReturn(zipStream);
- FileUtils.FileCopyResult result = puller.getSegmentFiles(CONTAINER_NAME, BLOB_PATH_HADOOP, toDir);
+ replayAll();
- File expected = new File(toDir, SEGMENT_FILE_NAME);
- Assert.assertEquals(value.length(), result.size());
- Assert.assertTrue(expected.exists());
- Assert.assertEquals(value.length(), expected.length());
+ AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
- verifyAll();
- }
- finally {
- pulledFile.delete();
- FileUtils.deleteDirectory(toDir);
- }
+ final String blobPathHadoop = AzureUtils.AZURE_STORAGE_HOST_ADDRESS + "/path/to/storage/index.zip";
+ FileUtils.FileCopyResult result = puller.getSegmentFiles(CONTAINER_NAME, blobPathHadoop, targetPath.toFile());
+
+ File expected = new File(targetPath.toFile(), segmentFileName);
+ assertEquals(value.length(), result.size());
+ assertTrue(expected.exists());
+ assertEquals(value.length(), expected.length());
+
+ verifyAll();
}
- @Test(expected = RuntimeException.class)
- public void test_getSegmentFiles_nonRecoverableErrorRaisedWhenPullingSegmentFiles_doNotDeleteOutputDirectory()
- throws IOException, BlobStorageException, SegmentLoadingException
+ @Test
+ public void test_getSegmentFiles_nonRecoverableErrorRaisedWhenPullingSegmentFiles_doNotDeleteOutputDirectory(
+ @TempDir Path tempPath
+ )
{
final AzureAccountConfig config = new AzureAccountConfig();
- final File outDir = FileUtils.createTempDir();
- try {
- EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
- EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andThrow(
- new RuntimeException(
- "error"
- )
- );
+ EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage))
+ .andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH))
+ .andThrow(new RuntimeException("error"));
- replayAll();
+ AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
- AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
+ replayAll();
- puller.getSegmentFiles(CONTAINER_NAME, BLOB_PATH, outDir);
- }
- catch (Exception e) {
- Assert.assertTrue(outDir.exists());
- verifyAll();
- throw e;
- }
- finally {
- FileUtils.deleteDirectory(outDir);
- }
+ assertThrows(
+ RuntimeException.class,
+ () -> puller.getSegmentFiles(CONTAINER_NAME, BLOB_PATH, tempPath.toFile())
+ );
+ assertTrue(tempPath.toFile().exists());
+
+ verifyAll();
}
- @Test(expected = SegmentLoadingException.class)
- public void test_getSegmentFiles_recoverableErrorRaisedWhenPullingSegmentFiles_deleteOutputDirectory()
- throws IOException, BlobStorageException, SegmentLoadingException
+ @Test
+ public void test_getSegmentFiles_recoverableErrorRaisedWhenPullingSegmentFiles_deleteOutputDirectory(
+ @TempDir Path tempPath
+ )
{
final AzureAccountConfig config = new AzureAccountConfig();
- final File outDir = FileUtils.createTempDir();
- try {
- HttpResponse httpResponse = createMock(HttpResponse.class);
- EasyMock.expect(httpResponse.getStatusCode()).andReturn(500).anyTimes();
- EasyMock.replay(httpResponse);
- EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage)).andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
- EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andThrow(
- new BlobStorageException("", httpResponse, null)
- ).atLeastOnce();
+ final HttpResponse httpResponse = createMock(HttpResponse.class);
+ EasyMock.expect(httpResponse.getStatusCode()).andReturn(500).anyTimes();
+ EasyMock.replay(httpResponse);
+ EasyMock.expect(byteSourceFactory.create(CONTAINER_NAME, BLOB_PATH, azureStorage))
+ .andReturn(new AzureByteSource(azureStorage, CONTAINER_NAME, BLOB_PATH));
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(0L, CONTAINER_NAME, BLOB_PATH)).andThrow(
+ new BlobStorageException("", httpResponse, null)
+ ).atLeastOnce();
- EasyMock.replay(azureStorage);
- EasyMock.replay(byteSourceFactory);
+ EasyMock.replay(azureStorage);
+ EasyMock.replay(byteSourceFactory);
- AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
+ AzureDataSegmentPuller puller = new AzureDataSegmentPuller(byteSourceFactory, azureStorage, config);
- puller.getSegmentFiles(CONTAINER_NAME, BLOB_PATH, outDir);
+ assertThrows(
+ SegmentLoadingException.class,
+ () -> puller.getSegmentFiles(CONTAINER_NAME, BLOB_PATH, tempPath.toFile())
+ );
- Assert.assertFalse(outDir.exists());
+ assertFalse(tempPath.toFile().exists());
+ verifyAll();
+ }
- verifyAll();
- }
- catch (Exception e) {
- Assert.assertFalse(outDir.exists());
- verifyAll();
- throw e;
- }
- finally {
- FileUtils.deleteDirectory(outDir);
+ @SuppressWarnings("SameParameterValue")
+ private static File createZipTempFile(
+ final Path tempPath,
+ final String entry,
+ final String entryValue
+ ) throws IOException
+ {
+ final File zipFile = Files.createFile(tempPath.resolve("index.zip")).toFile();
+
+ try (ZipOutputStream zipStream = new ZipOutputStream(Files.newOutputStream(zipFile.toPath()))) {
+ zipStream.putNextEntry(new ZipEntry(entry));
+ zipStream.write(entryValue.getBytes(StandardCharsets.UTF_8));
}
+
+ return zipFile;
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java
index 2c8e357b2c5a6..cc1bd8bd57250 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureDataSegmentPusherTest.java
@@ -26,28 +26,29 @@
import org.apache.druid.java.util.common.MapUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.partition.NoneShardSpec;
+import org.apache.druid.timeline.partition.LinearShardSpec;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.io.IOException;
+import java.nio.file.Path;
import java.util.ArrayList;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
import java.util.regex.Pattern;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
public class AzureDataSegmentPusherTest extends EasyMockSupport
{
- @Rule
- public final TemporaryFolder tempFolder = new TemporaryFolder();
-
private static final String ACCOUNT = "account";
private static final String CONTAINER_NAME = "container";
private static final String PREFIX = "prefix";
@@ -59,7 +60,7 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
ImmutableMap.of("containerName", CONTAINER_NAME, "blobPath", BLOB_PATH),
null,
null,
- NoneShardSpec.instance(),
+ new LinearShardSpec(0),
0,
1
);
@@ -77,7 +78,7 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
new HashMap<>(),
new ArrayList<>(),
new ArrayList<>(),
- NoneShardSpec.instance(),
+ new LinearShardSpec(0),
0,
DATA.length
);
@@ -87,7 +88,7 @@ public class AzureDataSegmentPusherTest extends EasyMockSupport
private AzureDataSegmentConfig segmentConfigWithPrefix;
private AzureDataSegmentConfig segmentConfigWithoutPrefix;
- @Before
+ @BeforeEach
public void before()
{
azureStorage = createMock(AzureStorage.class);
@@ -104,15 +105,14 @@ public void before()
}
@Test
- public void test_push_nonUniquePathNoPrefix_succeeds() throws Exception
+ public void test_push_nonUniquePathNoPrefix_succeeds(@TempDir Path tempPath) throws Exception
{
boolean useUniquePath = false;
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithoutPrefix
);
// Create a mock segment on disk
- File tmp = tempFolder.newFile("version.bin");
-
+ File tmp = tempPath.resolve("version.bin").toFile();
Files.write(DATA, tmp);
String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath);
@@ -121,28 +121,29 @@ public void test_push_nonUniquePathNoPrefix_succeeds() throws Exception
replayAll();
- DataSegment segment = pusher.push(tempFolder.getRoot(), SEGMENT_TO_PUSH, useUniquePath);
+ DataSegment segment = pusher.push(tempPath.toFile(), SEGMENT_TO_PUSH, useUniquePath);
- Assert.assertTrue(
- segment.getLoadSpec().get("blobPath").toString(),
- Pattern.compile(NON_UNIQUE_NO_PREFIX_MATCHER).matcher(segment.getLoadSpec().get("blobPath").toString()).matches()
+ assertTrue(
+ Pattern.compile(NON_UNIQUE_NO_PREFIX_MATCHER)
+ .matcher(segment.getLoadSpec().get("blobPath").toString())
+ .matches(),
+ segment.getLoadSpec().get("blobPath").toString()
);
- Assert.assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
+ assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
verifyAll();
}
@Test
- public void test_push_nonUniquePathWithPrefix_succeeds() throws Exception
+ public void test_push_nonUniquePathWithPrefix_succeeds(@TempDir Path tempPath) throws Exception
{
boolean useUniquePath = false;
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix
);
// Create a mock segment on disk
- File tmp = tempFolder.newFile("version.bin");
-
+ File tmp = tempPath.resolve("version.bin").toFile();
Files.write(DATA, tmp);
String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath);
@@ -156,26 +157,24 @@ public void test_push_nonUniquePathWithPrefix_succeeds() throws Exception
replayAll();
- DataSegment segment = pusher.push(tempFolder.getRoot(), SEGMENT_TO_PUSH, useUniquePath);
+ DataSegment segment = pusher.push(tempPath.toFile(), SEGMENT_TO_PUSH, useUniquePath);
- Assert.assertTrue(
- segment.getLoadSpec().get("blobPath").toString(),
- Pattern.compile(NON_UNIQUE_WITH_PREFIX_MATCHER).matcher(segment.getLoadSpec().get("blobPath").toString()).matches()
- );
+ assertTrue(Pattern.compile(NON_UNIQUE_WITH_PREFIX_MATCHER).matcher(segment.getLoadSpec().get("blobPath").toString()).matches(),
+ segment.getLoadSpec().get("blobPath").toString());
- Assert.assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
+ assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
verifyAll();
}
@Test
- public void test_push_uniquePathNoPrefix_succeeds() throws Exception
+ public void test_push_uniquePathNoPrefix_succeeds(@TempDir Path tempPath) throws Exception
{
boolean useUniquePath = true;
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithoutPrefix);
// Create a mock segment on disk
- File tmp = tempFolder.newFile("version.bin");
+ File tmp = tempPath.resolve("version.bin").toFile();
Files.write(DATA, tmp);
@@ -190,26 +189,27 @@ public void test_push_uniquePathNoPrefix_succeeds() throws Exception
replayAll();
- DataSegment segment = pusher.push(tempFolder.getRoot(), SEGMENT_TO_PUSH, useUniquePath);
+ DataSegment segment = pusher.push(tempPath.toFile(), SEGMENT_TO_PUSH, useUniquePath);
- Assert.assertTrue(
- segment.getLoadSpec().get("blobPath").toString(),
- Pattern.compile(UNIQUE_MATCHER_NO_PREFIX).matcher(segment.getLoadSpec().get("blobPath").toString()).matches()
- );
+ assertTrue(
+ Pattern.compile(UNIQUE_MATCHER_NO_PREFIX)
+ .matcher(segment.getLoadSpec().get("blobPath").toString())
+ .matches(),
+ segment.getLoadSpec().get("blobPath").toString());
- Assert.assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
+ assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
verifyAll();
}
@Test
- public void test_push_uniquePath_succeeds() throws Exception
+ public void test_push_uniquePath_succeeds(@TempDir Path tempPath) throws Exception
{
boolean useUniquePath = true;
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix);
// Create a mock segment on disk
- File tmp = tempFolder.newFile("version.bin");
+ File tmp = tempPath.resolve("version.bin").toFile();
Files.write(DATA, tmp);
@@ -224,57 +224,52 @@ public void test_push_uniquePath_succeeds() throws Exception
replayAll();
- DataSegment segment = pusher.push(tempFolder.getRoot(), SEGMENT_TO_PUSH, useUniquePath);
+ DataSegment segment = pusher.push(tempPath.toFile(), SEGMENT_TO_PUSH, useUniquePath);
- Assert.assertTrue(
- segment.getLoadSpec().get("blobPath").toString(),
- Pattern.compile(UNIQUE_MATCHER_PREFIX).matcher(segment.getLoadSpec().get("blobPath").toString()).matches()
+ assertTrue(
+ Pattern.compile(UNIQUE_MATCHER_PREFIX)
+ .matcher(segment.getLoadSpec().get("blobPath").toString())
+ .matches(),
+ segment.getLoadSpec().get("blobPath").toString()
);
- Assert.assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
+ assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
verifyAll();
}
- @Test(expected = RuntimeException.class)
- public void test_push_exception_throwsException() throws Exception
+ @Test
+ public void test_push_exception_throwsException(@TempDir Path tempPath) throws Exception
{
boolean useUniquePath = true;
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix);
// Create a mock segment on disk
- File tmp = tempFolder.newFile("version.bin");
+ File tmp = tempPath.resolve("version.bin").toFile();
Files.write(DATA, tmp);
- final long size = DATA.length;
- String azurePath = pusher.getAzurePath(SEGMENT_TO_PUSH, useUniquePath);
azureStorage.uploadBlockBlob(EasyMock.anyObject(File.class), EasyMock.eq(CONTAINER_NAME), EasyMock.anyString(), EasyMock.eq(MAX_TRIES));
EasyMock.expectLastCall().andThrow(new BlobStorageException("", null, null));
replayAll();
- DataSegment segment = pusher.push(tempFolder.getRoot(), SEGMENT_TO_PUSH, useUniquePath);
-
- Assert.assertTrue(
- segment.getLoadSpec().get("blobPath").toString(),
- Pattern.compile(UNIQUE_MATCHER_NO_PREFIX).matcher(segment.getLoadSpec().get("blobPath").toString()).matches()
+ assertThrows(
+ RuntimeException.class,
+ () -> pusher.push(tempPath.toFile(), SEGMENT_TO_PUSH, useUniquePath)
);
- Assert.assertEquals(SEGMENT_TO_PUSH.getSize(), segment.getSize());
-
verifyAll();
}
@Test
public void getAzurePathsTest()
{
-
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix);
final String storageDir = pusher.getStorageDir(DATA_SEGMENT, false);
final String azurePath = pusher.getAzurePath(DATA_SEGMENT, false);
- Assert.assertEquals(
+ assertEquals(
StringUtils.format("%s/%s", storageDir, AzureStorageDruidModule.INDEX_ZIP_FILE_NAME),
azurePath
);
@@ -301,11 +296,11 @@ public void uploadDataSegmentTest() throws BlobStorageException, IOException
azurePath
);
- Assert.assertEquals(compressedSegmentData.length(), pushedDataSegment.getSize());
- Assert.assertEquals(binaryVersion, (int) pushedDataSegment.getBinaryVersion());
+ assertEquals(compressedSegmentData.length(), pushedDataSegment.getSize());
+ assertEquals(binaryVersion, (int) pushedDataSegment.getBinaryVersion());
Map loadSpec = pushedDataSegment.getLoadSpec();
- Assert.assertEquals(AzureStorageDruidModule.SCHEME, MapUtils.getString(loadSpec, "type"));
- Assert.assertEquals(azurePath, MapUtils.getString(loadSpec, "blobPath"));
+ assertEquals(AzureStorageDruidModule.SCHEME, MapUtils.getString(loadSpec, "type"));
+ assertEquals(azurePath, MapUtils.getString(loadSpec, "blobPath"));
verifyAll();
}
@@ -315,7 +310,7 @@ public void getPathForHadoopWithPrefixTest()
{
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix);
String hadoopPath = pusher.getPathForHadoop();
- Assert.assertEquals("wasbs://container@account.blob.core.windows.net/prefix/", hadoopPath);
+ assertEquals("wasbs://container@account.blob.core.windows.net/prefix/", hadoopPath);
}
@Test
@@ -323,23 +318,7 @@ public void getPathForHadoopWithoutPrefixTest()
{
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithoutPrefix);
String hadoopPath = pusher.getPathForHadoop();
- Assert.assertEquals("wasbs://container@account.blob.core.windows.net/", hadoopPath);
- }
-
- @Test
- public void test_getPathForHadoop_noArgsWithoutPrefix_succeeds()
- {
- AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithoutPrefix);
- String hadoopPath = pusher.getPathForHadoop("");
- Assert.assertEquals("wasbs://container@account.blob.core.windows.net/", hadoopPath);
- }
-
- @Test
- public void test_getPathForHadoop_noArgsWithPrefix_succeeds()
- {
- AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix);
- String hadoopPath = pusher.getPathForHadoop("");
- Assert.assertEquals("wasbs://container@account.blob.core.windows.net/prefix/", hadoopPath);
+ assertEquals("wasbs://container@account.blob.core.windows.net/", hadoopPath);
}
@Test
@@ -347,7 +326,7 @@ public void test_getAllowedPropertyPrefixesForHadoop_returnsExpcetedPropertyPref
{
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix);
List actualPropertyPrefixes = pusher.getAllowedPropertyPrefixesForHadoop();
- Assert.assertEquals(AzureDataSegmentPusher.ALLOWED_PROPERTY_PREFIXES_FOR_HADOOP, actualPropertyPrefixes);
+ assertEquals(AzureDataSegmentPusher.ALLOWED_PROPERTY_PREFIXES_FOR_HADOOP, actualPropertyPrefixes);
}
@Test
@@ -356,6 +335,6 @@ public void storageDirContainsNoColonsTest()
AzureDataSegmentPusher pusher = new AzureDataSegmentPusher(azureStorage, azureAccountConfig, segmentConfigWithPrefix);
DataSegment withColons = DATA_SEGMENT.withVersion("2018-01-05T14:54:09.295Z");
String segmentPath = pusher.getStorageDir(withColons, false);
- Assert.assertFalse("Path should not contain any columns", segmentPath.contains(":"));
+ assertFalse(segmentPath.contains(":"), "Path should not contain any columns");
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureIngestClientFactoryTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureIngestClientFactoryTest.java
index d982cf2253e1b..1d82b845f28ea 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureIngestClientFactoryTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureIngestClientFactoryTest.java
@@ -26,21 +26,22 @@
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.azure.AzureStorageAccountInputSourceConfig;
import org.easymock.EasyMock;
-import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockExtension;
import org.easymock.EasyMockSupport;
import org.easymock.Mock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
import java.net.MalformedURLException;
import java.net.URL;
-@RunWith(EasyMockRunner.class)
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+
+@ExtendWith(EasyMockExtension.class)
public class AzureIngestClientFactoryTest extends EasyMockSupport
{
- private AzureIngestClientFactory azureIngestClientFactory;
private static final String ACCOUNT = "account";
private static final String KEY = "key";
private static final String TOKEN = "token";
@@ -51,7 +52,7 @@ public class AzureIngestClientFactoryTest extends EasyMockSupport
@Mock
private static AzureStorageAccountInputSourceConfig azureStorageAccountInputSourceConfig;
- @Before
+ @BeforeEach
public void setup()
{
EasyMock.expect(accountConfig.getBlobStorageEndpoint()).andReturn("blob.core.windows.net").anyTimes();
@@ -67,12 +68,13 @@ public void test_blobServiceClient_accountName()
null,
null
);
- azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
+
+ final AzureIngestClientFactory azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
replayAll();
BlobServiceClient blobServiceClient = azureIngestClientFactory.getBlobServiceClient(3, ACCOUNT);
verifyAll();
- Assert.assertEquals(ACCOUNT, blobServiceClient.getAccountName());
+ assertEquals(ACCOUNT, blobServiceClient.getAccountName());
}
@Test
@@ -85,7 +87,8 @@ public void test_blobServiceClientBuilder_key() throws MalformedURLException
null,
null
);
- azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
+
+ final AzureIngestClientFactory azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
replayAll();
BlobServiceClient blobServiceClient = azureIngestClientFactory.getBlobServiceClient(3, ACCOUNT);
@@ -93,12 +96,14 @@ public void test_blobServiceClientBuilder_key() throws MalformedURLException
StorageSharedKeyCredential storageSharedKeyCredential = StorageSharedKeyCredential.getSharedKeyCredentialFromPipeline(
blobServiceClient.getHttpPipeline()
);
- Assert.assertNotNull(storageSharedKeyCredential);
+ assertNotNull(storageSharedKeyCredential);
// Azure doesn't let us look at the key in the StorageSharedKeyCredential so make sure the authorization header generated is what we expect.
- Assert.assertEquals(
- new StorageSharedKeyCredential(ACCOUNT, KEY).generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of()),
- storageSharedKeyCredential.generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of())
+ assertEquals(
+ new StorageSharedKeyCredential(ACCOUNT, KEY)
+ .generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of()),
+ storageSharedKeyCredential
+ .generateAuthorizationHeader(new URL("http://druid.com"), "POST", ImmutableMap.of())
);
}
@@ -112,7 +117,8 @@ public void test_blobServiceClientBuilder_sasToken()
null,
null
);
- azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
+
+ final AzureIngestClientFactory azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
replayAll();
BlobServiceClient blobServiceClient = azureIngestClientFactory.getBlobServiceClient(3, ACCOUNT);
verifyAll();
@@ -124,7 +130,7 @@ public void test_blobServiceClientBuilder_sasToken()
}
}
- Assert.assertNotNull(azureSasCredentialPolicy);
+ assertNotNull(azureSasCredentialPolicy);
}
@Test
@@ -137,7 +143,8 @@ public void test_blobServiceClientBuilder_useAppRegistration()
"clientSecret",
"tenantId"
);
- azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
+
+ final AzureIngestClientFactory azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
replayAll();
BlobServiceClient blobServiceClient = azureIngestClientFactory.getBlobServiceClient(3, ACCOUNT);
verifyAll();
@@ -148,16 +155,16 @@ public void test_blobServiceClientBuilder_useAppRegistration()
}
}
- Assert.assertNotNull(bearerTokenAuthenticationPolicy);
+ assertNotNull(bearerTokenAuthenticationPolicy);
}
-
@Test
public void test_blobServiceClientBuilder_useAzureAccountConfig_asDefaultMaxTries()
{
// We should only call getKey twice (both times in the first call to getBlobServiceClient)
EasyMock.expect(azureStorageAccountInputSourceConfig.getKey()).andReturn(KEY).times(2);
- azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
+
+ final AzureIngestClientFactory azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
EasyMock.expect(accountConfig.getMaxTries()).andReturn(5);
replayAll();
azureIngestClientFactory.getBlobServiceClient(null, ACCOUNT);
@@ -181,7 +188,8 @@ public void test_blobServiceClientBuilder_fallbackToAzureAccountConfig()
null,
null
);
- azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
+
+ final AzureIngestClientFactory azureIngestClientFactory = new AzureIngestClientFactory(accountConfig, azureStorageAccountInputSourceConfig);
EasyMock.expect(accountConfig.getKey()).andReturn(KEY).times(2);
replayAll();
azureIngestClientFactory.getBlobServiceClient(5, ACCOUNT);
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageDruidModuleTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageDruidModuleTest.java
index 5b18b6c5b61b5..ab1ab379a8a53 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageDruidModuleTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageDruidModuleTest.java
@@ -19,14 +19,10 @@
package org.apache.druid.storage.azure;
-import com.azure.storage.blob.BlobServiceClient;
-import com.google.common.base.Supplier;
import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
import com.google.inject.Guice;
import com.google.inject.Injector;
import com.google.inject.Key;
-import com.google.inject.Module;
import com.google.inject.ProvisionException;
import com.google.inject.TypeLiteral;
import org.apache.druid.data.input.azure.AzureEntityFactory;
@@ -40,11 +36,11 @@
import org.apache.druid.segment.loading.OmniDataSegmentKiller;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.ExpectedException;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Named;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.params.ParameterizedTest;
+import org.junit.jupiter.params.provider.MethodSource;
import javax.validation.Validation;
import javax.validation.Validator;
@@ -52,12 +48,18 @@
import java.nio.charset.StandardCharsets;
import java.util.Base64;
import java.util.Properties;
+import java.util.stream.Stream;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertNotNull;
+import static org.junit.jupiter.api.Assertions.assertNotSame;
+import static org.junit.jupiter.api.Assertions.assertNull;
+import static org.junit.jupiter.api.Assertions.assertSame;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
public class AzureStorageDruidModuleTest extends EasyMockSupport
{
- @Rule
- public ExpectedException expectedException = ExpectedException.none();
-
private static final String AZURE_ACCOUNT_NAME;
private static final String AZURE_ACCOUNT_KEY;
private static final String AZURE_SHARED_ACCESS_TOKEN;
@@ -72,7 +74,6 @@ public class AzureStorageDruidModuleTest extends EasyMockSupport
private CloudObjectLocation cloudObjectLocation1;
private CloudObjectLocation cloudObjectLocation2;
private AzureStorage azureStorage;
- private Injector injector;
static {
try {
@@ -91,7 +92,7 @@ public class AzureStorageDruidModuleTest extends EasyMockSupport
}
}
- @Before
+ @BeforeEach
public void setup()
{
cloudObjectLocation1 = createMock(CloudObjectLocation.class);
@@ -102,11 +103,11 @@ public void setup()
@Test
public void testGetAzureAccountConfigExpectedConfig()
{
- injector = makeInjectorWithProperties(PROPERTIES);
+ final Injector injector = makeInjectorWithProperties(PROPERTIES);
AzureAccountConfig azureAccountConfig = injector.getInstance(AzureAccountConfig.class);
- Assert.assertEquals(AZURE_ACCOUNT_NAME, azureAccountConfig.getAccount());
- Assert.assertEquals(AZURE_ACCOUNT_KEY, azureAccountConfig.getKey());
+ assertEquals(AZURE_ACCOUNT_NAME, azureAccountConfig.getAccount());
+ assertEquals(AZURE_ACCOUNT_KEY, azureAccountConfig.getKey());
}
@Test
@@ -116,42 +117,42 @@ public void testGetAzureAccountConfigExpectedConfigWithSAS()
properties.setProperty("druid.azure.sharedAccessStorageToken", AZURE_SHARED_ACCESS_TOKEN);
properties.remove("druid.azure.key");
- injector = makeInjectorWithProperties(properties);
+ final Injector injector = makeInjectorWithProperties(properties);
AzureAccountConfig azureAccountConfig = injector.getInstance(AzureAccountConfig.class);
- Assert.assertEquals(AZURE_ACCOUNT_NAME, azureAccountConfig.getAccount());
- Assert.assertEquals(AZURE_SHARED_ACCESS_TOKEN, azureAccountConfig.getSharedAccessStorageToken());
+ assertEquals(AZURE_ACCOUNT_NAME, azureAccountConfig.getAccount());
+ assertEquals(AZURE_SHARED_ACCESS_TOKEN, azureAccountConfig.getSharedAccessStorageToken());
}
@Test
public void testGetAzureDataSegmentConfigExpectedConfig()
{
- injector = makeInjectorWithProperties(PROPERTIES);
+ final Injector injector = makeInjectorWithProperties(PROPERTIES);
AzureDataSegmentConfig segmentConfig = injector.getInstance(AzureDataSegmentConfig.class);
- Assert.assertEquals(AZURE_CONTAINER, segmentConfig.getContainer());
- Assert.assertEquals(AZURE_PREFIX, segmentConfig.getPrefix());
+ assertEquals(AZURE_CONTAINER, segmentConfig.getContainer());
+ assertEquals(AZURE_PREFIX, segmentConfig.getPrefix());
}
@Test
public void testGetAzureInputDataConfigExpectedConfig()
{
- injector = makeInjectorWithProperties(PROPERTIES);
+ final Injector injector = makeInjectorWithProperties(PROPERTIES);
AzureInputDataConfig inputDataConfig = injector.getInstance(AzureInputDataConfig.class);
- Assert.assertEquals(AZURE_MAX_LISTING_LENGTH, inputDataConfig.getMaxListingLength());
+ assertEquals(AZURE_MAX_LISTING_LENGTH, inputDataConfig.getMaxListingLength());
}
@Test
public void testGetAzureByteSourceFactoryCanCreateAzureByteSource()
{
- injector = makeInjectorWithProperties(PROPERTIES);
+ final Injector injector = makeInjectorWithProperties(PROPERTIES);
AzureByteSourceFactory factory = injector.getInstance(AzureByteSourceFactory.class);
Object object1 = factory.create("container1", "blob1", azureStorage);
Object object2 = factory.create("container2", "blob2", azureStorage);
- Assert.assertNotNull(object1);
- Assert.assertNotNull(object2);
- Assert.assertNotSame(object1, object2);
+ assertNotNull(object1);
+ assertNotNull(object2);
+ assertNotSame(object1, object2);
}
@Test
@@ -163,40 +164,40 @@ public void testGetAzureEntityFactoryCanCreateAzureEntity()
EasyMock.expect(cloudObjectLocation2.getPath()).andReturn(PATH);
replayAll();
- injector = makeInjectorWithProperties(PROPERTIES);
+ final Injector injector = makeInjectorWithProperties(PROPERTIES);
AzureEntityFactory factory = injector.getInstance(AzureEntityFactory.class);
Object object1 = factory.create(cloudObjectLocation1, azureStorage, AzureInputSource.SCHEME);
Object object2 = factory.create(cloudObjectLocation2, azureStorage, AzureInputSource.SCHEME);
Object object3 = factory.create(cloudObjectLocation1, azureStorage, AzureStorageAccountInputSource.SCHEME);
- Assert.assertNotNull(object1);
- Assert.assertNotNull(object2);
- Assert.assertNotNull(object3);
- Assert.assertNotSame(object1, object2);
- Assert.assertNotSame(object1, object3);
+ assertNotNull(object1);
+ assertNotNull(object2);
+ assertNotNull(object3);
+ assertNotSame(object1, object2);
+ assertNotSame(object1, object3);
}
@Test
public void testGetAzureCloudBlobIteratorFactoryCanCreateAzureCloudBlobIterator()
{
- injector = makeInjectorWithProperties(PROPERTIES);
+ final Injector injector = makeInjectorWithProperties(PROPERTIES);
AzureCloudBlobIteratorFactory factory = injector.getInstance(AzureCloudBlobIteratorFactory.class);
Object object1 = factory.create(EMPTY_PREFIXES_ITERABLE, 10, azureStorage);
Object object2 = factory.create(EMPTY_PREFIXES_ITERABLE, 10, azureStorage);
- Assert.assertNotNull(object1);
- Assert.assertNotNull(object2);
- Assert.assertNotSame(object1, object2);
+ assertNotNull(object1);
+ assertNotNull(object2);
+ assertNotSame(object1, object2);
}
@Test
public void testGetAzureCloudBlobIterableFactoryCanCreateAzureCloudBlobIterable()
{
- injector = makeInjectorWithProperties(PROPERTIES);
+ final Injector injector = makeInjectorWithProperties(PROPERTIES);
AzureCloudBlobIterableFactory factory = injector.getInstance(AzureCloudBlobIterableFactory.class);
AzureCloudBlobIterable object1 = factory.create(EMPTY_PREFIXES_ITERABLE, 10, azureStorage);
AzureCloudBlobIterable object2 = factory.create(EMPTY_PREFIXES_ITERABLE, 10, azureStorage);
- Assert.assertNotNull(object1);
- Assert.assertNotNull(object2);
- Assert.assertNotSame(object1, object2);
+ assertNotNull(object1);
+ assertNotNull(object2);
+ assertNotSame(object1, object2);
}
@Test
@@ -204,79 +205,75 @@ public void testSegmentKillerBoundSingleton()
{
Injector injector = makeInjectorWithProperties(PROPERTIES);
OmniDataSegmentKiller killer = injector.getInstance(OmniDataSegmentKiller.class);
- Assert.assertTrue(killer.getKillers().containsKey(AzureStorageDruidModule.SCHEME));
- Assert.assertSame(
+ assertTrue(killer.getKillers().containsKey(AzureStorageDruidModule.SCHEME));
+ assertSame(
AzureDataSegmentKiller.class,
killer.getKillers().get(AzureStorageDruidModule.SCHEME).get().getClass()
);
- Assert.assertSame(
+ assertSame(
killer.getKillers().get(AzureStorageDruidModule.SCHEME).get(),
killer.getKillers().get(AzureStorageDruidModule.SCHEME).get()
);
}
- @Test
- public void testMultipleCredentialsSet()
+ @ParameterizedTest
+ @MethodSource("propertiesWithMultipleCredentials")
+ public void testMultipleCredentialsSet(final Properties properties)
{
- String message = "Set only one of 'key' or 'sharedAccessStorageToken' or 'useAzureCredentialsChain' in the azure config.";
- Properties properties = initializePropertes();
- properties.setProperty("druid.azure.sharedAccessStorageToken", AZURE_SHARED_ACCESS_TOKEN);
- expectedException.expect(ProvisionException.class);
- expectedException.expectMessage(message);
- makeInjectorWithProperties(properties).getInstance(
- Key.get(new TypeLiteral()
- {
- })
- );
-
- properties = initializePropertes();
- properties.setProperty("druid.azure.managedIdentityClientId", AZURE_MANAGED_CREDENTIAL_CLIENT_ID);
- expectedException.expect(ProvisionException.class);
- expectedException.expectMessage(message);
- makeInjectorWithProperties(properties).getInstance(
- Key.get(new TypeLiteral>()
- {
- })
+ final ProvisionException exception = assertThrows(
+ ProvisionException.class,
+ () -> makeInjectorWithProperties(properties).getInstance(
+ Key.get(new TypeLiteral()
+ {
+ })
+ )
);
- properties = initializePropertes();
- properties.remove("druid.azure.key");
- properties.setProperty("druid.azure.managedIdentityClientId", AZURE_MANAGED_CREDENTIAL_CLIENT_ID);
- properties.setProperty("druid.azure.sharedAccessStorageToken", AZURE_SHARED_ACCESS_TOKEN);
- expectedException.expect(ProvisionException.class);
- expectedException.expectMessage(message);
- makeInjectorWithProperties(properties).getInstance(
- Key.get(new TypeLiteral()
- {
- })
+ assertEquals(
+ "Set only one of 'key' or 'sharedAccessStorageToken' or 'useAzureCredentialsChain' in the azure config. Please refer to azure documentation.",
+ exception.getCause().getMessage()
);
}
@Test
public void testAllCredentialsUnset()
{
- Properties properties = initializePropertes();
+ final Properties properties = initializePropertes();
properties.remove("druid.azure.key");
- expectedException.expect(ProvisionException.class);
- expectedException.expectMessage("Either set 'key' or 'sharedAccessStorageToken' or 'useAzureCredentialsChain' in the azure config.");
- makeInjectorWithProperties(properties).getInstance(
- Key.get(new TypeLiteral()
- {
- })
+
+ final ProvisionException exception = assertThrows(
+ ProvisionException.class,
+ () -> makeInjectorWithProperties(properties).getInstance(
+ Key.get(new TypeLiteral()
+ {
+ })
+ )
+ );
+
+ assertEquals(
+ "Either set 'key' or 'sharedAccessStorageToken' or 'useAzureCredentialsChain' in the azure config. Please refer to azure documentation.",
+ exception.getCause().getMessage()
);
}
@Test
public void testAccountUnset()
{
- Properties properties = initializePropertes();
+ final Properties properties = initializePropertes();
properties.remove("druid.azure.account");
- expectedException.expect(ProvisionException.class);
- expectedException.expectMessage("Set 'account' to the storage account that needs to be configured in the azure config. Please refer to azure documentation.");
- makeInjectorWithProperties(properties).getInstance(
- Key.get(new TypeLiteral()
- {
- })
+
+ final ProvisionException exception = assertThrows(
+ ProvisionException.class,
+ () -> makeInjectorWithProperties(properties).getInstance(
+ Key.get(new TypeLiteral()
+ {
+ })
+ )
+ );
+
+ assertEquals(
+ "Set 'account' to the storage account that needs to be configured in the azure config. Please refer to azure documentation.",
+ exception.getCause().getMessage()
);
}
@@ -285,9 +282,9 @@ public void testGetBlobStorageEndpointWithDefaultProperties()
{
Properties properties = initializePropertes();
AzureAccountConfig config = makeInjectorWithProperties(properties).getInstance(AzureAccountConfig.class);
- Assert.assertNull(config.getEndpointSuffix());
- Assert.assertEquals(config.getStorageAccountEndpointSuffix(), AzureUtils.AZURE_STORAGE_HOST_ADDRESS);
- Assert.assertEquals(config.getBlobStorageEndpoint(), AzureUtils.AZURE_STORAGE_HOST_ADDRESS);
+ assertNull(config.getEndpointSuffix());
+ assertEquals(config.getStorageAccountEndpointSuffix(), AzureUtils.AZURE_STORAGE_HOST_ADDRESS);
+ assertEquals(config.getBlobStorageEndpoint(), AzureUtils.AZURE_STORAGE_HOST_ADDRESS);
}
@Test
@@ -297,8 +294,8 @@ public void testGetBlobStorageEndpointWithCustomBlobPath()
final String customSuffix = "core.usgovcloudapi.net";
properties.setProperty("druid.azure.endpointSuffix", customSuffix);
AzureAccountConfig config = makeInjectorWithProperties(properties).getInstance(AzureAccountConfig.class);
- Assert.assertEquals(config.getEndpointSuffix(), customSuffix);
- Assert.assertEquals(config.getBlobStorageEndpoint(), "blob." + customSuffix);
+ assertEquals(config.getEndpointSuffix(), customSuffix);
+ assertEquals(config.getBlobStorageEndpoint(), "blob." + customSuffix);
}
private Injector makeInjectorWithProperties(final Properties props)
@@ -307,15 +304,10 @@ private Injector makeInjectorWithProperties(final Properties props)
ImmutableList.of(
new DruidGuiceExtensions(),
new JacksonModule(),
- new Module()
- {
- @Override
- public void configure(Binder binder)
- {
- binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator());
- binder.bind(JsonConfigurator.class).in(LazySingleton.class);
- binder.bind(Properties.class).toInstance(props);
- }
+ binder -> {
+ binder.bind(Validator.class).toInstance(Validation.buildDefaultValidatorFactory().getValidator());
+ binder.bind(JsonConfigurator.class).in(LazySingleton.class);
+ binder.bind(Properties.class).toInstance(props);
},
new AzureStorageDruidModule()
));
@@ -324,11 +316,31 @@ public void configure(Binder binder)
private static Properties initializePropertes()
{
final Properties props = new Properties();
- props.put("druid.azure.account", AZURE_ACCOUNT_NAME);
- props.put("druid.azure.key", AZURE_ACCOUNT_KEY);
- props.put("druid.azure.container", AZURE_CONTAINER);
- props.put("druid.azure.prefix", AZURE_PREFIX);
- props.put("druid.azure.maxListingLength", String.valueOf(AZURE_MAX_LISTING_LENGTH));
+ props.setProperty("druid.azure.account", AZURE_ACCOUNT_NAME);
+ props.setProperty("druid.azure.key", AZURE_ACCOUNT_KEY);
+ props.setProperty("druid.azure.container", AZURE_CONTAINER);
+ props.setProperty("druid.azure.prefix", AZURE_PREFIX);
+ props.setProperty("druid.azure.maxListingLength", String.valueOf(AZURE_MAX_LISTING_LENGTH));
return props;
}
+
+ private static Stream> propertiesWithMultipleCredentials()
+ {
+ final Properties propertiesWithKeyAndToken = initializePropertes();
+ propertiesWithKeyAndToken.setProperty("druid.azure.sharedAccessStorageToken", AZURE_SHARED_ACCESS_TOKEN);
+
+ final Properties propertiesWithKeyAndCredentialChain = initializePropertes();
+ propertiesWithKeyAndCredentialChain.setProperty("druid.azure.useAzureCredentialsChain", Boolean.TRUE.toString());
+
+ final Properties propertiesWithTokenAndCredentialChain = initializePropertes();
+ propertiesWithTokenAndCredentialChain.remove("druid.azure.key");
+ propertiesWithTokenAndCredentialChain.setProperty("druid.azure.useAzureCredentialsChain", Boolean.TRUE.toString());
+ propertiesWithTokenAndCredentialChain.setProperty("druid.azure.sharedAccessStorageToken", AZURE_SHARED_ACCESS_TOKEN);
+
+ return Stream.of(
+ Named.of("Key and storage token", propertiesWithKeyAndToken),
+ Named.of("Key and credential chain", propertiesWithKeyAndCredentialChain),
+ Named.of("Storage token and credential chain", propertiesWithTokenAndCredentialChain)
+ );
+ }
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java
index 6f66063897187..b61e72340148e 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureStorageTest.java
@@ -31,11 +31,9 @@
import com.azure.storage.blob.models.DeleteSnapshotsOptionType;
import com.google.common.collect.ImmutableList;
import org.apache.druid.common.guava.SettableSupplier;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import org.mockito.ArgumentCaptor;
import org.mockito.ArgumentMatchers;
import org.mockito.Mockito;
@@ -43,14 +41,19 @@
import java.io.File;
import java.io.IOException;
import java.io.InputStream;
+import java.nio.file.Files;
+import java.nio.file.Path;
import java.util.ArrayList;
import java.util.List;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
// Using Mockito for the whole test class since azure classes (e.g. BlobContainerClient) are final and can't be mocked with EasyMock
public class AzureStorageTest
{
-
AzureStorage azureStorage;
BlobClient blobClient = Mockito.mock(BlobClient.class);
BlobServiceClient blobServiceClient = Mockito.mock(BlobServiceClient.class);
@@ -60,12 +63,8 @@ public class AzureStorageTest
private final String STORAGE_ACCOUNT = "storageAccount";
private final String CONTAINER = "container";
private final String BLOB_NAME = "blobName";
- private final Integer MAX_ATTEMPTS = 3;
- @Rule
- public TemporaryFolder tempFolder = new TemporaryFolder();
-
- @Before
+ @BeforeEach
public void setup() throws BlobStorageException
{
azureStorage = new AzureStorage(azureClientFactory, STORAGE_ACCOUNT);
@@ -83,9 +82,11 @@ public void testListDir_retriable() throws BlobStorageException
ArgumentMatchers.any()
);
Mockito.doReturn(blobContainerClient).when(blobServiceClient).createBlobContainerIfNotExists(CONTAINER);
- Mockito.doReturn(blobServiceClient).when(azureClientFactory).getBlobServiceClient(MAX_ATTEMPTS, STORAGE_ACCOUNT);
- Assert.assertEquals(ImmutableList.of(BLOB_NAME), azureStorage.listDir(CONTAINER, "", MAX_ATTEMPTS));
+ final Integer maxAttempts = 3;
+ Mockito.doReturn(blobServiceClient).when(azureClientFactory).getBlobServiceClient(maxAttempts, STORAGE_ACCOUNT);
+
+ assertEquals(ImmutableList.of(BLOB_NAME), azureStorage.listDir(CONTAINER, "", maxAttempts));
}
@Test
@@ -102,7 +103,7 @@ public void testListDir_nullMaxAttempts() throws BlobStorageException
Mockito.doReturn(blobContainerClient).when(blobServiceClient).createBlobContainerIfNotExists(CONTAINER);
Mockito.doReturn(blobServiceClient).when(azureClientFactory).getBlobServiceClient(null, STORAGE_ACCOUNT);
- Assert.assertEquals(ImmutableList.of(BLOB_NAME), azureStorage.listDir(CONTAINER, "", null));
+ assertEquals(ImmutableList.of(BLOB_NAME), azureStorage.listDir(CONTAINER, "", null));
}
@Test
@@ -150,8 +151,8 @@ public void testBatchDeleteFiles_emptyResponse() throws BlobStorageException
);
boolean deleteSuccessful = azureStorage.batchDeleteFiles(CONTAINER, ImmutableList.of(BLOB_NAME), null);
- Assert.assertEquals(captor.getValue().get(0), containerUrl + "/" + BLOB_NAME);
- Assert.assertTrue(deleteSuccessful);
+ assertEquals(captor.getValue().get(0), containerUrl + "/" + BLOB_NAME);
+ assertTrue(deleteSuccessful);
}
@Test
@@ -174,8 +175,8 @@ public void testBatchDeleteFiles_error() throws BlobStorageException
);
boolean deleteSuccessful = azureStorage.batchDeleteFiles(CONTAINER, ImmutableList.of(BLOB_NAME), null);
- Assert.assertEquals(captor.getValue().get(0), containerUrl + "/" + BLOB_NAME);
- Assert.assertFalse(deleteSuccessful);
+ assertEquals(captor.getValue().get(0), containerUrl + "/" + BLOB_NAME);
+ assertFalse(deleteSuccessful);
}
@Test
@@ -207,15 +208,15 @@ public void testBatchDeleteFiles_emptyResponse_multipleResponses() throws BlobSt
boolean deleteSuccessful = azureStorage.batchDeleteFiles(CONTAINER, blobNameList, null);
List> deletedValues = captor.getAllValues();
- Assert.assertEquals(deletedValues.get(0).size(), 256);
- Assert.assertEquals(deletedValues.get(1).size(), 2);
- Assert.assertTrue(deleteSuccessful);
+ assertEquals(deletedValues.get(0).size(), 256);
+ assertEquals(deletedValues.get(1).size(), 2);
+ assertTrue(deleteSuccessful);
}
@Test
- public void testUploadBlob_usesOverwrite() throws BlobStorageException, IOException
+ public void testUploadBlob_usesOverwrite(@TempDir Path tempPath) throws BlobStorageException, IOException
{
- File tempFile = tempFolder.newFile("tempFile.txt");
+ final File tempFile = Files.createFile(tempPath.resolve("tempFile.txt")).toFile();
String blobPath = "blob";
ArgumentCaptor captor = ArgumentCaptor.forClass(InputStream.class);
@@ -229,7 +230,7 @@ public void testUploadBlob_usesOverwrite() throws BlobStorageException, IOExcept
azureStorage.uploadBlockBlob(tempFile, CONTAINER, blobPath, null);
Mockito.verify(blobClient).upload(captor.capture(), captor2.capture(), overrideArgument.capture());
- Assert.assertTrue(overrideArgument.getValue());
+ assertTrue(overrideArgument.getValue());
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java
index 92010952802ad..ca661fab438a7 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTaskLogsTest.java
@@ -25,15 +25,14 @@
import com.google.common.collect.ImmutableMap;
import org.apache.commons.io.IOUtils;
import org.apache.druid.common.utils.CurrentTimeMillisSupplier;
-import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.storage.azure.blob.CloudBlobHolder;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
-import org.junit.After;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
+import org.junit.jupiter.api.AfterEach;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import java.io.ByteArrayInputStream;
import java.io.File;
@@ -42,10 +41,16 @@
import java.io.StringWriter;
import java.net.URI;
import java.nio.charset.StandardCharsets;
+import java.nio.file.Files;
+import java.nio.file.Path;
+
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
public class AzureTaskLogsTest extends EasyMockSupport
{
-
private static final String CONTAINER = "test";
private static final String PREFIX = "test/log";
private static final String TASK_ID = "taskid";
@@ -70,7 +75,7 @@ public class AzureTaskLogsTest extends EasyMockSupport
private CurrentTimeMillisSupplier timeSupplier;
private AzureTaskLogs azureTaskLogs;
- @Before
+ @BeforeEach
public void before()
{
inputDataConfig = createMock(AzureInputDataConfig.class);
@@ -84,174 +89,138 @@ public void before()
accountConfig,
azureStorage,
azureCloudBlobIterableFactory,
- timeSupplier);
+ timeSupplier
+ );
}
-
@Test
- public void test_PushTaskLog_uploadsBlob() throws Exception
+ public void test_PushTaskLog_uploadsBlob(@TempDir Path tempPath) throws IOException
{
- final File tmpDir = FileUtils.createTempDir();
+ final File logFile = Files.createFile(tempPath.resolve("log")).toFile();
- try {
- final File logFile = new File(tmpDir, "log");
+ azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log", MAX_TRIES);
+ EasyMock.expectLastCall();
- azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log", MAX_TRIES);
- EasyMock.expectLastCall();
-
- EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
+ EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- replayAll();
+ replayAll();
- azureTaskLogs.pushTaskLog(TASK_ID, logFile);
+ azureTaskLogs.pushTaskLog(TASK_ID, logFile);
- verifyAll();
- }
- finally {
- FileUtils.deleteDirectory(tmpDir);
- }
+ verifyAll();
}
- @Test(expected = RuntimeException.class)
- public void test_PushTaskLog_exception_rethrowsException() throws Exception
+ @Test
+ public void test_PushTaskLog_exception_rethrowsException(@TempDir Path tempPath) throws IOException
{
- final File tmpDir = FileUtils.createTempDir();
+ final File logFile = Files.createFile(tempPath.resolve("log")).toFile();
- try {
- final File logFile = new File(tmpDir, "log");
-
- EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log", MAX_TRIES);
- EasyMock.expectLastCall().andThrow(new IOException());
+ EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
+ azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/log", MAX_TRIES);
+ EasyMock.expectLastCall().andThrow(new IOException());
- replayAll();
+ replayAll();
- azureTaskLogs.pushTaskLog(TASK_ID, logFile);
+ assertThrows(
+ RuntimeException.class,
+ () -> azureTaskLogs.pushTaskLog(TASK_ID, logFile)
+ );
- verifyAll();
- }
- finally {
- FileUtils.deleteDirectory(tmpDir);
- }
+ verifyAll();
}
@Test
- public void test_PushTaskReports_uploadsBlob() throws Exception
+ public void test_PushTaskReports_uploadsBlob(@TempDir Path tempPath) throws IOException
{
- final File tmpDir = FileUtils.createTempDir();
+ final File logFile = Files.createFile(tempPath.resolve("log")).toFile();
- try {
- final File logFile = new File(tmpDir, "log");
-
- EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json", MAX_TRIES);
- EasyMock.expectLastCall();
+ EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
+ azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json", MAX_TRIES);
+ EasyMock.expectLastCall();
- replayAll();
+ replayAll();
- azureTaskLogs.pushTaskReports(TASK_ID, logFile);
+ azureTaskLogs.pushTaskReports(TASK_ID, logFile);
- verifyAll();
- }
- finally {
- FileUtils.deleteDirectory(tmpDir);
- }
+ verifyAll();
}
@Test
- public void test_PushTaskStatus_uploadsBlob() throws Exception
+ public void test_PushTaskStatus_uploadsBlob(@TempDir Path tempPath) throws IOException
{
- final File tmpDir = FileUtils.createTempDir();
+ final File logFile = Files.createFile(tempPath.resolve("status.json")).toFile();
- try {
- final File logFile = new File(tmpDir, "status.json");
-
- EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/status.json", MAX_TRIES);
- EasyMock.expectLastCall();
+ EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
+ azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/status.json", MAX_TRIES);
+ EasyMock.expectLastCall();
- replayAll();
+ replayAll();
- azureTaskLogs.pushTaskStatus(TASK_ID, logFile);
+ azureTaskLogs.pushTaskStatus(TASK_ID, logFile);
- verifyAll();
- }
- finally {
- FileUtils.deleteDirectory(tmpDir);
- }
+ verifyAll();
}
@Test
- public void test_PushTaskPayload_uploadsBlob() throws Exception
+ public void test_PushTaskPayload_uploadsBlob(@TempDir Path tempPath) throws IOException
{
- final File tmpDir = FileUtils.createTempDir();
+ final File taskFile = Files.createFile(tempPath.resolve("task.json")).toFile();
- try {
- final File taskFile = new File(tmpDir, "task.json");
-
- EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- azureStorage.uploadBlockBlob(taskFile, CONTAINER, PREFIX + "/" + TASK_ID + "/task.json", MAX_TRIES);
- EasyMock.expectLastCall();
+ EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
+ azureStorage.uploadBlockBlob(taskFile, CONTAINER, PREFIX + "/" + TASK_ID + "/task.json", MAX_TRIES);
+ EasyMock.expectLastCall();
- replayAll();
+ replayAll();
- azureTaskLogs.pushTaskPayload(TASK_ID, taskFile);
+ azureTaskLogs.pushTaskPayload(TASK_ID, taskFile);
- verifyAll();
- }
- finally {
- FileUtils.deleteDirectory(tmpDir);
- }
+ verifyAll();
}
- @Test(expected = RuntimeException.class)
- public void test_PushTaskReports_exception_rethrowsException() throws Exception
+ @Test
+ public void test_PushTaskReports_exception_rethrowsException(@TempDir Path tempPath) throws IOException
{
- final File tmpDir = FileUtils.createTempDir();
+ final File logFile = Files.createFile(tempPath.resolve("log")).toFile();
- try {
- final File logFile = new File(tmpDir, "log");
-
- EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
- azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json", MAX_TRIES);
- EasyMock.expectLastCall().andThrow(new IOException());
+ EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
+ azureStorage.uploadBlockBlob(logFile, CONTAINER, PREFIX + "/" + TASK_ID + "/report.json", MAX_TRIES);
+ EasyMock.expectLastCall().andThrow(new IOException());
- replayAll();
+ replayAll();
- azureTaskLogs.pushTaskReports(TASK_ID, logFile);
+ assertThrows(
+ RuntimeException.class,
+ () -> azureTaskLogs.pushTaskReports(TASK_ID, logFile)
+ );
- verifyAll();
- }
- finally {
- FileUtils.deleteDirectory(tmpDir);
- }
+ verifyAll();
}
@Test
- public void testStreamTaskLogWithoutOffset() throws Exception
+ public void testStreamTaskLogWithoutOffset() throws IOException
{
final String testLog = "hello this is a log";
final String blobPath = PREFIX + "/" + TASK_ID + "/log";
EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true);
EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length());
- EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn(
- new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8)));
-
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath))
+ .andReturn(new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8)));
replayAll();
final Optional stream = azureTaskLogs.streamTaskLog(TASK_ID, 0);
+ assertTrue(stream.isPresent());
final StringWriter writer = new StringWriter();
IOUtils.copy(stream.get(), writer, "UTF-8");
- Assert.assertEquals(writer.toString(), testLog);
+ assertEquals(writer.toString(), testLog);
verifyAll();
}
@Test
- public void testStreamTaskLogWithPositiveOffset() throws Exception
+ public void testStreamTaskLogWithPositiveOffset() throws IOException
{
final String testLog = "hello this is a log";
@@ -261,43 +230,43 @@ public void testStreamTaskLogWithPositiveOffset() throws Exception
EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn(
new ByteArrayInputStream(testLog.getBytes(StandardCharsets.UTF_8)));
-
replayAll();
final Optional stream = azureTaskLogs.streamTaskLog(TASK_ID, 5);
+ assertTrue(stream.isPresent());
final StringWriter writer = new StringWriter();
IOUtils.copy(stream.get(), writer, "UTF-8");
- Assert.assertEquals(writer.toString(), testLog.substring(5));
+ assertEquals(writer.toString(), testLog.substring(5));
verifyAll();
}
@Test
- public void testStreamTaskLogWithNegative() throws Exception
+ public void testStreamTaskLogWithNegative() throws IOException
{
final String testLog = "hello this is a log";
final String blobPath = PREFIX + "/" + TASK_ID + "/log";
EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true);
EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length());
- EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn(
- new ByteArrayInputStream(StringUtils.toUtf8(testLog)));
-
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath))
+ .andReturn(new ByteArrayInputStream(StringUtils.toUtf8(testLog)));
replayAll();
final Optional stream = azureTaskLogs.streamTaskLog(TASK_ID, -3);
+ assertTrue(stream.isPresent());
final StringWriter writer = new StringWriter();
IOUtils.copy(stream.get(), writer, "UTF-8");
- Assert.assertEquals(writer.toString(), testLog.substring(testLog.length() - 3));
+ assertEquals(writer.toString(), testLog.substring(testLog.length() - 3));
verifyAll();
}
@Test
- public void test_streamTaskReports_blobExists_succeeds() throws Exception
+ public void test_streamTaskReports_blobExists_succeeds() throws IOException
{
final String testLog = "hello this is a log";
@@ -311,19 +280,18 @@ public void test_streamTaskReports_blobExists_succeeds() throws Exception
replayAll();
final Optional stream = azureTaskLogs.streamTaskReports(TASK_ID);
+ assertTrue(stream.isPresent());
final StringWriter writer = new StringWriter();
IOUtils.copy(stream.get(), writer, "UTF-8");
- Assert.assertEquals(writer.toString(), testLog);
+ assertEquals(writer.toString(), testLog);
verifyAll();
}
@Test
- public void test_streamTaskReports_blobDoesNotExist_returnsAbsent() throws Exception
+ public void test_streamTaskReports_blobDoesNotExist_returnsAbsent() throws IOException
{
- final String testLog = "hello this is a log";
-
final String blobPath = PREFIX + "/" + TASK_ID_NOT_FOUND + "/report.json";
EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(false);
@@ -331,72 +299,76 @@ public void test_streamTaskReports_blobDoesNotExist_returnsAbsent() throws Excep
final Optional stream = azureTaskLogs.streamTaskReports(TASK_ID_NOT_FOUND);
-
- Assert.assertFalse(stream.isPresent());
+ assertFalse(stream.isPresent());
verifyAll();
}
- @Test(expected = IOException.class)
- public void test_streamTaskReports_exceptionWhenGettingStream_throwsException() throws Exception
+ @Test
+ public void test_streamTaskReports_exceptionWhenGettingStream_throwsException()
{
final String testLog = "hello this is a log";
final String blobPath = PREFIX + "/" + TASK_ID + "/report.json";
- EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true);
- EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) testLog.length());
- EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andThrow(
- new BlobStorageException("", null, null));
-
+ EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath))
+ .andReturn(true);
+ EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath))
+ .andReturn((long) testLog.length());
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath))
+ .andThrow(new BlobStorageException("", null, null));
replayAll();
- final Optional stream = azureTaskLogs.streamTaskReports(TASK_ID);
+ assertThrows(
+ IOException.class,
+ () -> azureTaskLogs.streamTaskReports(TASK_ID)
+ );
- final StringWriter writer = new StringWriter();
- IOUtils.copy(stream.get(), writer, "UTF-8");
verifyAll();
}
- @Test(expected = IOException.class)
- public void test_streamTaskReports_exceptionWhenCheckingBlobExistence_throwsException() throws Exception
+ @Test
+ public void test_streamTaskReports_exceptionWhenCheckingBlobExistence_throwsException()
{
-
final String blobPath = PREFIX + "/" + TASK_ID + "/report.json";
- EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andThrow(new BlobStorageException("", null, null));
+ EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath))
+ .andThrow(new BlobStorageException("", null, null));
replayAll();
- azureTaskLogs.streamTaskReports(TASK_ID);
+ assertThrows(
+ IOException.class,
+ () -> azureTaskLogs.streamTaskReports(TASK_ID)
+ );
verifyAll();
}
@Test
- public void test_streamTaskStatus_blobExists_succeeds() throws Exception
+ public void test_streamTaskStatus_blobExists_succeeds() throws IOException
{
final String taskStatus = "{}";
final String blobPath = PREFIX + "/" + TASK_ID + "/status.json";
EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true);
EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) taskStatus.length());
- EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn(
- new ByteArrayInputStream(taskStatus.getBytes(StandardCharsets.UTF_8)));
-
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath))
+ .andReturn(new ByteArrayInputStream(taskStatus.getBytes(StandardCharsets.UTF_8)));
replayAll();
final Optional stream = azureTaskLogs.streamTaskStatus(TASK_ID);
+ assertTrue(stream.isPresent());
final StringWriter writer = new StringWriter();
IOUtils.copy(stream.get(), writer, "UTF-8");
- Assert.assertEquals(writer.toString(), taskStatus);
+ assertEquals(writer.toString(), taskStatus);
verifyAll();
}
@Test
- public void test_streamTaskStatus_blobDoesNotExist_returnsAbsent() throws Exception
+ public void test_streamTaskStatus_blobDoesNotExist_returnsAbsent() throws IOException
{
final String blobPath = PREFIX + "/" + TASK_ID_NOT_FOUND + "/status.json";
EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(false);
@@ -406,47 +378,53 @@ public void test_streamTaskStatus_blobDoesNotExist_returnsAbsent() throws Except
final Optional stream = azureTaskLogs.streamTaskStatus(TASK_ID_NOT_FOUND);
- Assert.assertFalse(stream.isPresent());
+ assertFalse(stream.isPresent());
verifyAll();
}
- @Test(expected = IOException.class)
- public void test_streamTaskStatus_exceptionWhenGettingStream_throwsException() throws Exception
+ @Test
+ public void test_streamTaskStatus_exceptionWhenGettingStream_throwsException()
{
final String taskStatus = "{}";
final String blobPath = PREFIX + "/" + TASK_ID + "/status.json";
- EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true);
- EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) taskStatus.length());
- EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andThrow(
- new BlobStorageException("", null, null));
-
+ EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath))
+ .andReturn(true);
+ EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath))
+ .andReturn((long) taskStatus.length());
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath))
+ .andThrow(new BlobStorageException("", null, null));
replayAll();
- final Optional stream = azureTaskLogs.streamTaskStatus(TASK_ID);
+ assertThrows(
+ IOException.class,
+ () -> azureTaskLogs.streamTaskStatus(TASK_ID)
+ );
- final StringWriter writer = new StringWriter();
- IOUtils.copy(stream.get(), writer, "UTF-8");
verifyAll();
}
- @Test(expected = IOException.class)
- public void test_streamTaskStatus_exceptionWhenCheckingBlobExistence_throwsException() throws Exception
+ @Test
+ public void test_streamTaskStatus_exceptionWhenCheckingBlobExistence_throwsException()
{
final String blobPath = PREFIX + "/" + TASK_ID + "/status.json";
- EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andThrow(new BlobStorageException("", null, null));
+ EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath))
+ .andThrow(new BlobStorageException("", null, null));
replayAll();
- azureTaskLogs.streamTaskStatus(TASK_ID);
+ assertThrows(
+ IOException.class,
+ () -> azureTaskLogs.streamTaskStatus(TASK_ID)
+ );
verifyAll();
}
@Test
- public void test_streamTaskPayload_blobExists_succeeds() throws Exception
+ public void test_streamTaskPayload_blobExists_succeeds() throws IOException
{
final String taskPayload = "{}";
@@ -456,20 +434,20 @@ public void test_streamTaskPayload_blobExists_succeeds() throws Exception
EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andReturn(
new ByteArrayInputStream(taskPayload.getBytes(StandardCharsets.UTF_8)));
-
replayAll();
final Optional stream = azureTaskLogs.streamTaskPayload(TASK_ID);
+ assertTrue(stream.isPresent());
final StringWriter writer = new StringWriter();
IOUtils.copy(stream.get(), writer, "UTF-8");
- Assert.assertEquals(writer.toString(), taskPayload);
+ assertEquals(writer.toString(), taskPayload);
verifyAll();
}
@Test
- public void test_streamTaskPayload_blobDoesNotExist_returnsAbsent() throws Exception
+ public void test_streamTaskPayload_blobDoesNotExist_returnsAbsent() throws IOException
{
final String blobPath = PREFIX + "/" + TASK_ID_NOT_FOUND + "/task.json";
EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(false);
@@ -479,48 +457,53 @@ public void test_streamTaskPayload_blobDoesNotExist_returnsAbsent() throws Excep
final Optional stream = azureTaskLogs.streamTaskPayload(TASK_ID_NOT_FOUND);
- Assert.assertFalse(stream.isPresent());
+ assertFalse(stream.isPresent());
verifyAll();
}
- @Test(expected = IOException.class)
- public void test_streamTaskPayload_exceptionWhenGettingStream_throwsException() throws Exception
+ @Test
+ public void test_streamTaskPayload_exceptionWhenGettingStream_throwsException()
{
final String taskPayload = "{}";
final String blobPath = PREFIX + "/" + TASK_ID + "/task.json";
- EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andReturn(true);
- EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath)).andReturn((long) taskPayload.length());
- EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath)).andThrow(
- new BlobStorageException("", null, null));
-
+ EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath))
+ .andReturn(true);
+ EasyMock.expect(azureStorage.getBlockBlobLength(CONTAINER, blobPath))
+ .andReturn((long) taskPayload.length());
+ EasyMock.expect(azureStorage.getBlockBlobInputStream(CONTAINER, blobPath))
+ .andThrow(new BlobStorageException("", null, null));
replayAll();
- final Optional stream = azureTaskLogs.streamTaskPayload(TASK_ID);
+ assertThrows(
+ IOException.class,
+ () -> azureTaskLogs.streamTaskPayload(TASK_ID)
+ );
- final StringWriter writer = new StringWriter();
- IOUtils.copy(stream.get(), writer, "UTF-8");
verifyAll();
}
- @Test(expected = IOException.class)
- public void test_streamTaskPayload_exceptionWhenCheckingBlobExistence_throwsException() throws Exception
+ @Test
+ public void test_streamTaskPayload_exceptionWhenCheckingBlobExistence_throwsException()
{
final String blobPath = PREFIX + "/" + TASK_ID + "/task.json";
- EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath)).andThrow(new BlobStorageException("", null, null));
+ EasyMock.expect(azureStorage.getBlockBlobExists(CONTAINER, blobPath))
+ .andThrow(new BlobStorageException("", null, null));
replayAll();
- azureTaskLogs.streamTaskPayload(TASK_ID);
+ assertThrows(
+ IOException.class,
+ () -> azureTaskLogs.streamTaskPayload(TASK_ID)
+ );
verifyAll();
}
-
@Test
- public void test_killAll_noException_deletesAllTaskLogs() throws Exception
+ public void test_killAll_noException_deletesAllTaskLogs() throws IOException
{
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_KEYS);
EasyMock.expect(timeSupplier.getAsLong()).andReturn(TIME_NOW);
@@ -544,9 +527,25 @@ public void test_killAll_noException_deletesAllTaskLogs() throws Exception
ImmutableMap.of(),
MAX_TRIES
);
- EasyMock.replay(inputDataConfig, accountConfig, timeSupplier, azureCloudBlobIterable, azureCloudBlobIterableFactory, azureStorage);
+ EasyMock.replay(
+ inputDataConfig,
+ accountConfig,
+ timeSupplier,
+ azureCloudBlobIterable,
+ azureCloudBlobIterableFactory,
+ azureStorage
+ );
azureTaskLogs.killAll();
- EasyMock.verify(inputDataConfig, accountConfig, timeSupplier, object1, object2, azureCloudBlobIterable, azureCloudBlobIterableFactory, azureStorage);
+ EasyMock.verify(
+ inputDataConfig,
+ accountConfig,
+ timeSupplier,
+ object1,
+ object2,
+ azureCloudBlobIterable,
+ azureCloudBlobIterableFactory,
+ azureStorage
+ );
}
@Test
@@ -590,7 +589,7 @@ public void test_killAll_nonrecoverableExceptionWhenListingObjects_doesntDeleteA
catch (IOException e) {
ioExceptionThrown = true;
}
- Assert.assertTrue(ioExceptionThrown);
+ assertTrue(ioExceptionThrown);
EasyMock.verify(
inputDataConfig,
accountConfig,
@@ -603,7 +602,7 @@ public void test_killAll_nonrecoverableExceptionWhenListingObjects_doesntDeleteA
}
@Test
- public void test_killOlderThan_noException_deletesOnlyTaskLogsOlderThan() throws Exception
+ public void test_killOlderThan_noException_deletesOnlyTaskLogsOlderThan() throws IOException
{
EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_KEYS);
EasyMock.expect(accountConfig.getMaxTries()).andReturn(MAX_TRIES).anyTimes();
@@ -626,9 +625,25 @@ public void test_killOlderThan_noException_deletesOnlyTaskLogsOlderThan() throws
ImmutableMap.of(),
MAX_TRIES
);
- EasyMock.replay(inputDataConfig, accountConfig, timeSupplier, azureCloudBlobIterable, azureCloudBlobIterableFactory, azureStorage);
+ EasyMock.replay(
+ inputDataConfig,
+ accountConfig,
+ timeSupplier,
+ azureCloudBlobIterable,
+ azureCloudBlobIterableFactory,
+ azureStorage
+ );
azureTaskLogs.killOlderThan(TIME_NOW);
- EasyMock.verify(inputDataConfig, accountConfig, timeSupplier, object1, object2, azureCloudBlobIterable, azureCloudBlobIterableFactory, azureStorage);
+ EasyMock.verify(
+ inputDataConfig,
+ accountConfig,
+ timeSupplier,
+ object1,
+ object2,
+ azureCloudBlobIterable,
+ azureCloudBlobIterableFactory,
+ azureStorage
+ );
}
@Test
@@ -671,7 +686,7 @@ public void test_killOlderThan_nonrecoverableExceptionWhenListingObjects_doesntD
catch (IOException e) {
ioExceptionThrown = true;
}
- Assert.assertTrue(ioExceptionThrown);
+ assertTrue(ioExceptionThrown);
EasyMock.verify(
inputDataConfig,
accountConfig,
@@ -683,15 +698,7 @@ public void test_killOlderThan_nonrecoverableExceptionWhenListingObjects_doesntD
);
}
- /*
- @Test (expected = UnsupportedOperationException.class)
- public void test_killOlderThan_throwsUnsupportedOperationException() throws IOException
- {
- azureTaskLogs.killOlderThan(0);
- }
- */
-
- @After
+ @AfterEach
public void cleanup()
{
resetAll();
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTestUtils.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTestUtils.java
index e6c048dbdd225..ddbb70b1309d0 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTestUtils.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureTestUtils.java
@@ -25,34 +25,14 @@
import org.easymock.EasyMockSupport;
import org.easymock.IExpectationSetters;
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
import java.net.URI;
-import java.nio.charset.StandardCharsets;
-import java.nio.file.Files;
import java.util.Date;
import java.util.HashMap;
import java.util.List;
import java.util.Map;
-import java.util.zip.ZipEntry;
-import java.util.zip.ZipOutputStream;
public class AzureTestUtils extends EasyMockSupport
{
- public static File createZipTempFile(final String segmentFileName, final String content) throws IOException
- {
- final File zipFile = Files.createTempFile("index", ".zip").toFile();
- final byte[] value = content.getBytes(StandardCharsets.UTF_8);
-
- try (ZipOutputStream zipStream = new ZipOutputStream(new FileOutputStream(zipFile))) {
- zipStream.putNextEntry(new ZipEntry(segmentFileName));
- zipStream.write(value);
- }
-
- return zipFile;
- }
-
public static AzureCloudBlobIterable expectListObjects(
AzureCloudBlobIterableFactory azureCloudBlobIterableFactory,
int maxListingLength,
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureUtilsTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureUtilsTest.java
index 4a28c4de4ccc1..6491c61521c12 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureUtilsTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/AzureUtilsTest.java
@@ -23,19 +23,22 @@
import com.azure.storage.blob.models.BlobStorageException;
import org.apache.druid.data.input.azure.AzureInputSource;
import org.easymock.EasyMock;
-import org.easymock.EasyMockRunner;
+import org.easymock.EasyMockExtension;
import org.easymock.EasyMockSupport;
import org.easymock.Mock;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.extension.ExtendWith;
import java.io.IOException;
import java.net.URI;
import java.net.URISyntaxException;
import java.util.concurrent.TimeoutException;
-@RunWith(EasyMockRunner.class)
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
+@ExtendWith(EasyMockExtension.class)
public class AzureUtilsTest extends EasyMockSupport
{
private static final String CONTAINER_NAME = "container1";
@@ -79,28 +82,28 @@ public class AzureUtilsTest extends EasyMockSupport
public void test_extractAzureKey_pathHasLeadingSlash_returnsPathWithLeadingSlashRemoved()
{
String extractedKey = AzureUtils.extractAzureKey(URI_WITH_PATH_WITH_LEADING_SLASH);
- Assert.assertEquals(BLOB_NAME, extractedKey);
+ assertEquals(BLOB_NAME, extractedKey);
}
@Test
public void test_maybeRemoveAzurePathPrefix_pathHasLeadingAzurePathPrefix_returnsPathWithLeadingAzurePathRemoved()
{
String path = AzureUtils.maybeRemoveAzurePathPrefix(BLOB_PATH_WITH_LEADING_AZURE_PREFIX, AzureUtils.AZURE_STORAGE_HOST_ADDRESS);
- Assert.assertEquals(BLOB_NAME, path);
+ assertEquals(BLOB_NAME, path);
}
@Test
public void test_maybeRemoveAzurePathPrefix_pathDoesNotHaveAzurePathPrefix__returnsPathWithLeadingAzurePathRemoved()
{
String path = AzureUtils.maybeRemoveAzurePathPrefix(BLOB_NAME, AzureUtils.AZURE_STORAGE_HOST_ADDRESS);
- Assert.assertEquals(BLOB_NAME, path);
+ assertEquals(BLOB_NAME, path);
}
@Test
public void test_azureRetry_URISyntaxException_returnsFalse()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(URI_SYNTAX_EXCEPTION);
- Assert.assertFalse(retry);
+ assertFalse(retry);
}
@Test
@@ -112,7 +115,7 @@ public void test_azureRetry_StorageException_500ErrorCode_returnsTrue()
BlobStorageException blobStorageException = new BlobStorageException("storage exception", httpResponse, null);
boolean retry = AzureUtils.AZURE_RETRY.apply(blobStorageException);
verifyAll();
- Assert.assertTrue(retry);
+ assertTrue(retry);
}
@Test
@@ -124,7 +127,7 @@ public void test_azureRetry_StorageException_429ErrorCode_returnsTrue()
BlobStorageException blobStorageException = new BlobStorageException("storage exception", httpResponse, null);
boolean retry = AzureUtils.AZURE_RETRY.apply(blobStorageException);
verifyAll();
- Assert.assertTrue(retry);
+ assertTrue(retry);
}
@Test
@@ -136,7 +139,7 @@ public void test_azureRetry_StorageException_503ErrorCode_returnsTrue()
BlobStorageException blobStorageException = new BlobStorageException("storage exception", httpResponse, null);
boolean retry = AzureUtils.AZURE_RETRY.apply(blobStorageException);
verifyAll();
- Assert.assertTrue(retry);
+ assertTrue(retry);
}
@Test
@@ -148,76 +151,76 @@ public void test_azureRetry_StorageException_400ErrorCode_returnsFalse()
BlobStorageException blobStorageException = new BlobStorageException("storage exception", httpResponse, null);
boolean retry = AzureUtils.AZURE_RETRY.apply(blobStorageException);
verifyAll();
- Assert.assertFalse(retry);
+ assertFalse(retry);
}
@Test
public void test_azureRetry_nestedIOException_returnsTrue()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(new RuntimeException("runtime", new IOException("ioexception")));
- Assert.assertTrue(retry);
+ assertTrue(retry);
}
@Test
public void test_azureRetry_nestedTimeoutException_returnsTrue()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(new RuntimeException("runtime", new TimeoutException("timeout exception")));
- Assert.assertTrue(retry);
+ assertTrue(retry);
}
@Test
public void test_azureRetry_IOException_returnsTrue()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(IO_EXCEPTION);
- Assert.assertTrue(retry);
+ assertTrue(retry);
}
@Test
public void test_azureRetry_nullException_returnsFalse()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(null);
- Assert.assertFalse(retry);
+ assertFalse(retry);
}
@Test
public void test_azureRetry_RunTimeException_returnsFalse()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(RUNTIME_EXCEPTION);
- Assert.assertFalse(retry);
+ assertFalse(retry);
}
@Test
public void test_azureRetry_nullExceptionWrappedInRunTimeException_returnsFalse()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(NULL_EXCEPTION_WRAPPED_IN_RUNTIME_EXCEPTION);
- Assert.assertFalse(retry);
+ assertFalse(retry);
}
@Test
public void test_azureRetry_IOExceptionWrappedInRunTimeException_returnsTrue()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(IO_EXCEPTION_WRAPPED_IN_RUNTIME_EXCEPTION);
- Assert.assertTrue(retry);
+ assertTrue(retry);
}
@Test
public void test_azureRetry_RunTimeExceptionWrappedInRunTimeException_returnsFalse()
{
boolean retry = AzureUtils.AZURE_RETRY.apply(RUNTIME_EXCEPTION_WRAPPED_IN_RUNTIME_EXCEPTON);
- Assert.assertFalse(retry);
+ assertFalse(retry);
}
@Test
public void testRemoveAzurePathPrefixDefaultEndpoint()
{
String outputBlob = AzureUtils.maybeRemoveAzurePathPrefix("blob.core.windows.net/container/blob", "blob.core.windows.net");
- Assert.assertEquals("container/blob", outputBlob);
+ assertEquals("container/blob", outputBlob);
}
@Test
public void testRemoveAzurePathPrefixCustomEndpoint()
{
String outputBlob = AzureUtils.maybeRemoveAzurePathPrefix("blob.core.usgovcloudapi.net/container/blob", "blob.core.usgovcloudapi.net");
- Assert.assertEquals("container/blob", outputBlob);
+ assertEquals("container/blob", outputBlob);
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java
index 4753132d1c488..722d28f583499 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureInputRangeTest.java
@@ -20,7 +20,7 @@
package org.apache.druid.storage.azure.output;
import nl.jqno.equalsverifier.EqualsVerifier;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
public class AzureInputRangeTest
{
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java
index ab3104adf4ea7..058887316ec9e 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputConfigTest.java
@@ -24,62 +24,60 @@
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.ISE;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
import java.io.File;
import java.io.IOException;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
public class AzureOutputConfigTest
{
-
- @Rule
- public final TemporaryFolder temporaryFolder = new TemporaryFolder();
-
private static final String CONTAINER = "container";
private static final String PREFIX = "prefix";
private static final int MAX_RETRY_COUNT = 0;
@Test
- public void testTooLargeChunkSize()
+ public void testTooLargeChunkSize(@TempDir File tempDir)
{
HumanReadableBytes chunkSize = new HumanReadableBytes("4001MiB");
- Assert.assertThrows(
+
+ //noinspection ResultOfObjectAllocationIgnored
+ assertThrows(
DruidException.class,
- () -> new AzureOutputConfig(CONTAINER, PREFIX, temporaryFolder.newFolder(), chunkSize, MAX_RETRY_COUNT)
+ () -> new AzureOutputConfig(CONTAINER, PREFIX, tempDir, chunkSize, MAX_RETRY_COUNT)
);
}
@Test
- public void testTempDirectoryNotWritable() throws IOException
+ public void testTempDirectoryNotWritable(@TempDir File tempDir)
{
- File tempDir = temporaryFolder.newFolder();
if (!tempDir.setWritable(false)) {
throw new ISE("Unable to change the permission of temp folder for %s", this.getClass().getName());
}
+
//noinspection ResultOfObjectAllocationIgnored
- Assert.assertThrows(
+ assertThrows(
DruidException.class,
() -> new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT)
);
}
@Test
- public void testTempDirectoryNotPresentButWritable() throws IOException
+ public void testTempDirectoryNotPresentButWritable(@TempDir File tempDir)
{
- File tempDir = new File(temporaryFolder.newFolder() + "/notPresent1/notPresent2/notPresent3");
+ File temporaryFolder = new File(tempDir + "/notPresent1/notPresent2/notPresent3");
//noinspection ResultOfObjectAllocationIgnored
- new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT);
+ new AzureOutputConfig(CONTAINER, PREFIX, temporaryFolder, null, MAX_RETRY_COUNT);
}
@Test
- public void testTempDirectoryPresent() throws IOException
+ public void testTempDirectoryPresent(@TempDir File tempDir) throws IOException
{
- File tempDir = new File(temporaryFolder.newFolder() + "/notPresent1/notPresent2/notPresent3");
- FileUtils.mkdirp(tempDir);
+ File temporaryFolder = new File(tempDir + "/notPresent1/notPresent2/notPresent3");
+ FileUtils.mkdirp(temporaryFolder);
//noinspection ResultOfObjectAllocationIgnored
- new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, MAX_RETRY_COUNT);
+ new AzureOutputConfig(CONTAINER, PREFIX, temporaryFolder, null, MAX_RETRY_COUNT);
}
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java
index ecf99666ce70e..aea5232217a76 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureOutputSerdeTest.java
@@ -25,12 +25,14 @@
import com.fasterxml.jackson.databind.exc.ValueInstantiationException;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.StringUtils;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
import java.io.File;
import java.io.IOException;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
public class AzureOutputSerdeTest
{
@@ -55,12 +57,9 @@ public void sanity() throws IOException
2
);
- Assert.assertEquals(
- json,
- MAPPER.writeValueAsString(azureOutputConfig)
- );
+ assertEquals(json, MAPPER.writeValueAsString(azureOutputConfig));
- Assert.assertEquals(azureOutputConfig, MAPPER.readValue(json, AzureOutputConfig.class));
+ assertEquals(azureOutputConfig, MAPPER.readValue(json, AzureOutputConfig.class));
}
@Test
@@ -72,7 +71,7 @@ public void noPrefix()
+ " \"chunkSize\":104857600,\n"
+ " \"maxRetry\": 2\n"
+ "}\n");
- Assert.assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
+ assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
}
@Test
@@ -84,7 +83,7 @@ public void noContainer()
+ " \"chunkSize\":104857600,\n"
+ " \"maxRetry\": 2\n"
+ "}\n");
- Assert.assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
+ assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
}
@Test
@@ -96,7 +95,7 @@ public void noTempDir()
+ " \"chunkSize\":104857600,\n"
+ " \"maxRetry\": 2\n"
+ "}\n");
- Assert.assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
+ assertThrows(MismatchedInputException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
}
@Test
@@ -115,7 +114,7 @@ public void leastArguments() throws JsonProcessingException
null,
null
);
- Assert.assertEquals(azureOutputConfig, MAPPER.readValue(json, AzureOutputConfig.class));
+ assertEquals(azureOutputConfig, MAPPER.readValue(json, AzureOutputConfig.class));
}
@@ -130,7 +129,7 @@ public void testChunkValidation()
+ " \"chunkSize\":104,\n"
+ " \"maxRetry\": 2\n"
+ "}\n");
- Assert.assertThrows(ValueInstantiationException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
+ assertThrows(ValueInstantiationException.class, () -> MAPPER.readValue(json, AzureOutputConfig.class));
}
private static String jsonStringReadyForAssert(String input)
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java
index 7014fe64f0d9e..0b76f02af29e3 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorProviderTest.java
@@ -34,12 +34,15 @@
import org.apache.druid.storage.azure.AzureStorage;
import org.apache.druid.storage.azure.AzureStorageDruidModule;
import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Test;
+import org.junit.jupiter.api.Test;
import java.io.File;
import java.util.Properties;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertInstanceOf;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+
public class AzureStorageConnectorProviderTest
{
private static final String CUSTOM_NAMESPACE = "custom";
@@ -55,12 +58,12 @@ public void createAzureStorageFactoryWithRequiredProperties()
properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp");
StorageConnectorProvider storageConnectorProvider = getStorageConnectorProvider(properties);
- Assert.assertTrue(storageConnectorProvider instanceof AzureStorageConnectorProvider);
- Assert.assertTrue(storageConnectorProvider.get() instanceof AzureStorageConnector);
- Assert.assertEquals("container", ((AzureStorageConnectorProvider) storageConnectorProvider).getContainer());
- Assert.assertEquals("prefix", ((AzureStorageConnectorProvider) storageConnectorProvider).getPrefix());
- Assert.assertEquals(new File("/tmp"), ((AzureStorageConnectorProvider) storageConnectorProvider).getTempDir());
-
+ assertInstanceOf(AzureStorageConnectorProvider.class, storageConnectorProvider);
+ assertInstanceOf(AzureStorageConnector.class, storageConnectorProvider.get());
+ assertEquals("container", ((AzureStorageConnectorProvider) storageConnectorProvider).getContainer());
+ assertEquals("prefix", ((AzureStorageConnectorProvider) storageConnectorProvider).getPrefix());
+ assertEquals(new File("/tmp"),
+ ((AzureStorageConnectorProvider) storageConnectorProvider).getTempDir());
}
@Test
@@ -71,10 +74,10 @@ public void createAzureStorageFactoryWithMissingPrefix()
properties.setProperty(CUSTOM_NAMESPACE + ".type", "azure");
properties.setProperty(CUSTOM_NAMESPACE + ".container", "container");
properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp");
- Assert.assertThrows(
- "Missing required creator property 'prefix'",
+ assertThrows(
ProvisionException.class,
- () -> getStorageConnectorProvider(properties)
+ () -> getStorageConnectorProvider(properties),
+ "Missing required creator property 'prefix'"
);
}
@@ -87,10 +90,10 @@ public void createAzureStorageFactoryWithMissingContainer()
properties.setProperty(CUSTOM_NAMESPACE + ".type", "azure");
properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix");
properties.setProperty(CUSTOM_NAMESPACE + ".tempDir", "/tmp");
- Assert.assertThrows(
- "Missing required creator property 'container'",
+ assertThrows(
ProvisionException.class,
- () -> getStorageConnectorProvider(properties)
+ () -> getStorageConnectorProvider(properties),
+ "Missing required creator property 'container'"
);
}
@@ -103,10 +106,10 @@ public void createAzureStorageFactoryWithMissingTempDir()
properties.setProperty(CUSTOM_NAMESPACE + ".container", "container");
properties.setProperty(CUSTOM_NAMESPACE + ".prefix", "prefix");
- Assert.assertThrows(
- "Missing required creator property 'tempDir'",
+ assertThrows(
ProvisionException.class,
- () -> getStorageConnectorProvider(properties)
+ () -> getStorageConnectorProvider(properties),
+ "Missing required creator property 'tempDir'"
);
}
diff --git a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java
index 17cee1855383e..5219f2b5962df 100644
--- a/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java
+++ b/extensions-core/azure-extensions/src/test/java/org/apache/druid/storage/azure/output/AzureStorageConnectorTest.java
@@ -28,41 +28,40 @@
import org.apache.druid.storage.azure.AzureStorage;
import org.easymock.Capture;
import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
+import org.junit.jupiter.api.BeforeEach;
+import org.junit.jupiter.api.Test;
+import org.junit.jupiter.api.io.TempDir;
+import java.io.File;
import java.io.IOException;
import java.io.InputStream;
import java.nio.charset.StandardCharsets;
import java.util.Collections;
import java.util.List;
+import static org.junit.jupiter.api.Assertions.assertEquals;
+import static org.junit.jupiter.api.Assertions.assertFalse;
+import static org.junit.jupiter.api.Assertions.assertThrows;
+import static org.junit.jupiter.api.Assertions.assertTrue;
+
public class AzureStorageConnectorTest
{
-
private static final String CONTAINER = "CONTAINER";
private static final String PREFIX = "P/R/E/F/I/X";
public static final String TEST_FILE = "test.csv";
- @Rule
- public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
private StorageConnector storageConnector;
private final AzureStorage azureStorage = EasyMock.createMock(AzureStorage.class);
- @Before
- public void setup() throws IOException
+ @BeforeEach
+ public void setup(@TempDir File tempDir)
{
storageConnector = new AzureStorageConnector(
- new AzureOutputConfig(CONTAINER, PREFIX, temporaryFolder.newFolder(), null, null),
+ new AzureOutputConfig(CONTAINER, PREFIX, tempDir, null, null),
azureStorage
);
}
-
@Test
public void testPathExistsSuccess() throws BlobStorageException, IOException
{
@@ -72,9 +71,9 @@ public void testPathExistsSuccess() throws BlobStorageException, IOException
EasyMock.expect(azureStorage.getBlockBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt()))
.andReturn(true);
EasyMock.replay(azureStorage);
- Assert.assertTrue(storageConnector.pathExists(TEST_FILE));
- Assert.assertEquals(CONTAINER, bucket.getValue());
- Assert.assertEquals(PREFIX + "/" + TEST_FILE, path.getValue());
+ assertTrue(storageConnector.pathExists(TEST_FILE));
+ assertEquals(CONTAINER, bucket.getValue());
+ assertEquals(PREFIX + "/" + TEST_FILE, path.getValue());
EasyMock.verify(azureStorage);
}
@@ -87,9 +86,9 @@ public void testPathExistsNotFound() throws BlobStorageException, IOException
EasyMock.expect(azureStorage.getBlockBlobExists(EasyMock.capture(bucket), EasyMock.capture(path), EasyMock.anyInt()))
.andReturn(false);
EasyMock.replay(azureStorage);
- Assert.assertFalse(storageConnector.pathExists(TEST_FILE));
- Assert.assertEquals(CONTAINER, bucket.getValue());
- Assert.assertEquals(PREFIX + "/" + TEST_FILE, path.getValue());
+ assertFalse(storageConnector.pathExists(TEST_FILE));
+ assertEquals(CONTAINER, bucket.getValue());
+ assertEquals(PREFIX + "/" + TEST_FILE, path.getValue());
EasyMock.verify(azureStorage);
}
@@ -114,9 +113,9 @@ public void testRead() throws BlobStorageException, IOException
EasyMock.replay(azureStorage);
InputStream is = storageConnector.read(TEST_FILE);
byte[] dataBytes = new byte[data.length()];
- Assert.assertEquals(data.length(), is.read(dataBytes));
- Assert.assertEquals(-1, is.read());
- Assert.assertEquals(data, new String(dataBytes, StandardCharsets.UTF_8));
+ assertEquals(data.length(), is.read(dataBytes));
+ assertEquals(-1, is.read());
+ assertEquals(data, new String(dataBytes, StandardCharsets.UTF_8));
EasyMock.reset(azureStorage);
}
@@ -142,9 +141,9 @@ public void testReadRange() throws BlobStorageException, IOException
InputStream is = storageConnector.readRange(TEST_FILE, start, length);
byte[] dataBytes = new byte[((Long) length).intValue()];
- Assert.assertEquals(length, is.read(dataBytes));
- Assert.assertEquals(-1, is.read());
- Assert.assertEquals(dataQueried, new String(dataBytes, StandardCharsets.UTF_8));
+ assertEquals(length, is.read(dataBytes));
+ assertEquals(-1, is.read());
+ assertEquals(dataQueried, new String(dataBytes, StandardCharsets.UTF_8));
EasyMock.reset(azureStorage);
}
}
@@ -163,8 +162,8 @@ public void testDeleteSinglePath() throws BlobStorageException, IOException
)).andReturn(true);
EasyMock.replay(azureStorage);
storageConnector.deleteFile(TEST_FILE);
- Assert.assertEquals(CONTAINER, containerCapture.getValue());
- Assert.assertEquals(Collections.singletonList(PREFIX + "/" + TEST_FILE), pathsCapture.getValue());
+ assertEquals(CONTAINER, containerCapture.getValue());
+ assertEquals(Collections.singletonList(PREFIX + "/" + TEST_FILE), pathsCapture.getValue());
EasyMock.reset(azureStorage);
}
@@ -181,8 +180,8 @@ public void testDeleteMultiplePaths() throws BlobStorageException, IOException
)).andReturn(true);
EasyMock.replay(azureStorage);
storageConnector.deleteFiles(ImmutableList.of(TEST_FILE + "_1.part", TEST_FILE + "_2.part"));
- Assert.assertEquals(CONTAINER, containerCapture.getValue());
- Assert.assertEquals(
+ assertEquals(CONTAINER, containerCapture.getValue());
+ assertEquals(
ImmutableList.of(
PREFIX + "/" + TEST_FILE + "_1.part",
PREFIX + "/" + TEST_FILE + "_2.part"
@@ -200,7 +199,7 @@ public void testListDir() throws BlobStorageException, IOException
.andReturn(ImmutableList.of(PREFIX + "/x/y/z/" + TEST_FILE, PREFIX + "/p/q/r/" + TEST_FILE));
EasyMock.replay(azureStorage);
List ret = Lists.newArrayList(storageConnector.listDir(""));
- Assert.assertEquals(ImmutableList.of("x/y/z/" + TEST_FILE, "p/q/r/" + TEST_FILE), ret);
+ assertEquals(ImmutableList.of("x/y/z/" + TEST_FILE, "p/q/r/" + TEST_FILE), ret);
EasyMock.reset(azureStorage);
}
@@ -212,7 +211,7 @@ public void test_deleteFile_blobStorageException()
azureStorage.batchDeleteFiles(EasyMock.anyString(), EasyMock.anyObject(), EasyMock.anyInt());
EasyMock.expectLastCall().andThrow(new BlobStorageException("error", mockHttpResponse, null));
EasyMock.replay(azureStorage);
- Assert.assertThrows(IOException.class, () -> storageConnector.deleteFile("file"));
+ assertThrows(IOException.class, () -> storageConnector.deleteFile("file"));
EasyMock.verify(azureStorage);
EasyMock.reset(azureStorage);
}
@@ -225,7 +224,7 @@ public void test_deleteFiles_blobStorageException()
azureStorage.batchDeleteFiles(EasyMock.anyString(), EasyMock.anyObject(), EasyMock.anyInt());
EasyMock.expectLastCall().andThrow(new BlobStorageException("error", mockHttpResponse, null));
EasyMock.replay(azureStorage);
- Assert.assertThrows(IOException.class, () -> storageConnector.deleteFiles(ImmutableList.of()));
+ assertThrows(IOException.class, () -> storageConnector.deleteFiles(ImmutableList.of()));
EasyMock.verify(azureStorage);
EasyMock.reset(azureStorage);
}
diff --git a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java
index 1ffa89ab2471c..b3f6ac20d53e0 100644
--- a/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java
+++ b/extensions-core/multi-stage-query/src/test/java/org/apache/druid/msq/exec/MSQWindowTest.java
@@ -217,11 +217,11 @@ public void testWindowOnFooWithFirstWindowPartitionNextWindowEmpty(String contex
.add("w1", ColumnType.DOUBLE)
.build(),
ImmutableList.of(
+ new NaivePartitioningOperatorFactory(ImmutableList.of()),
+ new WindowOperatorFactory(proc1),
new NaiveSortOperatorFactory(ImmutableList.of(ColumnWithDirection.ascending("d0"))),
new NaivePartitioningOperatorFactory(ImmutableList.of("d0")),
- new WindowOperatorFactory(proc),
- new NaivePartitioningOperatorFactory(ImmutableList.of()),
- new WindowOperatorFactory(proc1)
+ new WindowOperatorFactory(proc)
),
null
);
@@ -245,12 +245,12 @@ public void testWindowOnFooWithFirstWindowPartitionNextWindowEmpty(String contex
.build())
.setExpectedRowSignature(rowSignature)
.setExpectedResultRows(ImmutableList.of(
- new Object[]{1.0f, 1.0, 1.0, 1.0},
- new Object[]{2.0f, 2.0, 2.0, 2.0},
- new Object[]{3.0f, 3.0, 3.0, 3.0},
- new Object[]{4.0f, 4.0, 4.0, 4.0},
- new Object[]{5.0f, 5.0, 5.0, 5.0},
- new Object[]{6.0f, 6.0, 6.0, 6.0}
+ new Object[]{1.0f, 1.0, 1.0, 21.0},
+ new Object[]{2.0f, 2.0, 2.0, 21.0},
+ new Object[]{3.0f, 3.0, 3.0, 21.0},
+ new Object[]{4.0f, 4.0, 4.0, 21.0},
+ new Object[]{5.0f, 5.0, 5.0, 21.0},
+ new Object[]{6.0f, 6.0, 6.0, 21.0}
))
.setQueryContext(context)
.setExpectedCountersForStageWorkerChannel(
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java
index 6986ec683a568..98ab50cff7883 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/common/actions/SegmentAllocationQueue.java
@@ -268,7 +268,7 @@ private void processBatchesDue()
catch (Throwable t) {
currentBatch.failPendingRequests(t);
processed = true;
- log.error(t, "Error while processing batch [%s]", currentBatch.key);
+ log.error(t, "Error while processing batch[%s].", currentBatch.key);
}
// Requeue if not fully processed yet
@@ -619,7 +619,7 @@ void failPendingRequests(String reason)
void failPendingRequests(Throwable cause)
{
if (!requestToFuture.isEmpty()) {
- log.warn("Failing [%d] requests in batch [%s], reason [%s].", size(), cause.getMessage(), key);
+ log.warn("Failing [%d] requests in batch[%s], reason[%s].", size(), key, cause.getMessage());
requestToFuture.values().forEach(future -> future.completeExceptionally(cause));
requestToFuture.keySet().forEach(
request -> emitTaskMetric("task/action/failed/count", 1L, request)
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java
index 7776663330b5b..2155ac2c26558 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskLockbox.java
@@ -657,7 +657,7 @@ private TaskLockPosse createOrFindLockPosse(LockRequest request, Task task, bool
}
} else {
- log.info("Task[%s] already present in TaskLock[%s]", task.getId(), posseToUse.getTaskLock().getGroupId());
+ log.debug("Task[%s] already present in TaskLock[%s].", task.getId(), posseToUse.getTaskLock().getGroupId());
}
return posseToUse;
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
index 6036fe630baf5..ea8f1a56ed859 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/TaskQueue.java
@@ -35,6 +35,7 @@
import org.apache.druid.common.utils.IdUtils;
import org.apache.druid.error.DruidException;
import org.apache.druid.error.EntryAlreadyExists;
+import org.apache.druid.error.InvalidInput;
import org.apache.druid.indexer.RunnerTaskState;
import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskStatus;
@@ -98,6 +99,9 @@ public class TaskQueue
private static final long MANAGEMENT_WAIT_TIMEOUT_NANOS = TimeUnit.SECONDS.toNanos(60);
private static final long MIN_WAIT_TIME_MS = 100;
+ // 60 MB warning threshold since 64 MB is the default max_allowed_packet size in MySQL 8+
+ private static final long TASK_SIZE_WARNING_THRESHOLD = 1024 * 1024 * 60;
+
// Task ID -> Task, for tasks that are active in some way (submitted, running, or finished and to-be-cleaned-up).
@GuardedBy("giant")
private final LinkedHashMap tasks = new LinkedHashMap<>();
@@ -508,6 +512,7 @@ public boolean add(final Task task)
if (taskStorage.getTask(task.getId()).isPresent()) {
throw EntryAlreadyExists.exception("Task[%s] already exists", task.getId());
}
+ validateTaskPayload(task);
// Set forceTimeChunkLock before adding task spec to taskStorage, so that we can see always consistent task spec.
task.addToContextIfAbsent(Tasks.FORCE_TIME_CHUNK_LOCK_KEY, lockConfig.isForceTimeChunkLock());
@@ -1018,4 +1023,34 @@ List getTasks()
giant.unlock();
}
}
+
+ private void validateTaskPayload(Task task)
+ {
+ try {
+ String payload = passwordRedactingMapper.writeValueAsString(task);
+ if (config.getMaxTaskPayloadSize() != null && config.getMaxTaskPayloadSize().getBytesInInt() < payload.length()) {
+ throw InvalidInput.exception(
+ "Task[%s] has payload of size[%d] but max allowed size is [%d]. " +
+ "Reduce the size of the task payload or increase 'druid.indexer.queue.maxTaskPayloadSize'.",
+ task.getId(), payload.length(), config.getMaxTaskPayloadSize()
+ );
+ } else if (payload.length() > TASK_SIZE_WARNING_THRESHOLD) {
+ log.warn(
+ "Task[%s] of datasource[%s] has payload size[%d] larger than the recommended maximum[%d]. " +
+ "Large task payloads may cause stability issues in the Overlord and may fail while persisting to the metadata store." +
+ "Such tasks may be rejected by the Overlord in future Druid versions.",
+ task.getId(),
+ task.getDataSource(),
+ payload.length(),
+ TASK_SIZE_WARNING_THRESHOLD
+ );
+ }
+ }
+ catch (JsonProcessingException e) {
+ log.error(e, "Failed to parse task payload for validation");
+ throw DruidException.defensive(
+ "Failed to parse task payload for validation"
+ );
+ }
+ }
}
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskQueueConfig.java b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskQueueConfig.java
index 1e7bb1ffa2d83..5c10dfacf979c 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskQueueConfig.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/overlord/config/TaskQueueConfig.java
@@ -22,9 +22,12 @@
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import org.apache.druid.common.config.Configs;
+import org.apache.druid.java.util.common.HumanReadableBytes;
import org.joda.time.Duration;
import org.joda.time.Period;
+import javax.annotation.Nullable;
+
public class TaskQueueConfig
{
@JsonProperty
@@ -42,13 +45,17 @@ public class TaskQueueConfig
@JsonProperty
private int taskCompleteHandlerNumThreads;
+ @JsonProperty
+ private HumanReadableBytes maxTaskPayloadSize;
+
@JsonCreator
public TaskQueueConfig(
@JsonProperty("maxSize") final Integer maxSize,
@JsonProperty("startDelay") final Period startDelay,
@JsonProperty("restartDelay") final Period restartDelay,
@JsonProperty("storageSyncRate") final Period storageSyncRate,
- @JsonProperty("taskCompleteHandlerNumThreads") final Integer taskCompleteHandlerNumThreads
+ @JsonProperty("taskCompleteHandlerNumThreads") final Integer taskCompleteHandlerNumThreads,
+ @JsonProperty("maxTaskPayloadSize") @Nullable final HumanReadableBytes maxTaskPayloadSize
)
{
this.maxSize = Configs.valueOrDefault(maxSize, Integer.MAX_VALUE);
@@ -56,6 +63,7 @@ public TaskQueueConfig(
this.startDelay = defaultDuration(startDelay, "PT1M");
this.restartDelay = defaultDuration(restartDelay, "PT30S");
this.storageSyncRate = defaultDuration(storageSyncRate, "PT1M");
+ this.maxTaskPayloadSize = maxTaskPayloadSize;
}
public int getMaxSize()
@@ -83,6 +91,11 @@ public Duration getStorageSyncRate()
return storageSyncRate;
}
+ public HumanReadableBytes getMaxTaskPayloadSize()
+ {
+ return maxTaskPayloadSize;
+ }
+
private static Duration defaultDuration(final Period period, final String theDefault)
{
return (period == null ? new Period(theDefault) : period).toStandardDuration();
diff --git a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
index d413a9bec3e5a..ec4de45cac716 100644
--- a/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
+++ b/indexing-service/src/main/java/org/apache/druid/indexing/seekablestream/supervisor/SeekableStreamSupervisor.java
@@ -83,6 +83,7 @@
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.RetryUtils;
+import org.apache.druid.java.util.common.Stopwatch;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.concurrent.Execs;
import org.apache.druid.java.util.emitter.EmittingLogger;
@@ -96,13 +97,12 @@
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.joda.time.DateTime;
+import org.joda.time.Duration;
import javax.annotation.Nonnull;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
import java.io.IOException;
-import java.time.Duration;
-import java.time.Instant;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
@@ -1203,19 +1203,17 @@ public void tryInit()
}
try {
- Instant handleNoticeStartTime = Instant.now();
+ final Stopwatch noticeHandleTime = Stopwatch.createStarted();
notice.handle();
- Instant handleNoticeEndTime = Instant.now();
- Duration timeElapsed = Duration.between(handleNoticeStartTime, handleNoticeEndTime);
String noticeType = notice.getType();
+ emitNoticeProcessTime(noticeType, noticeHandleTime.millisElapsed());
if (log.isDebugEnabled()) {
log.debug(
- "Handled notice [%s] from notices queue in [%d] ms, "
- + "current notices queue size [%d] for datasource [%s]",
- noticeType, timeElapsed.toMillis(), getNoticesQueueSize(), dataSource
+ "Handled notice[%s] from notices queue in [%d] ms, "
+ + "current notices queue size [%d] for datasource[%s].",
+ noticeType, noticeHandleTime.millisElapsed(), getNoticesQueueSize(), dataSource
);
}
- emitNoticeProcessTime(noticeType, timeElapsed.toMillis());
}
catch (Throwable e) {
stateManager.recordThrowableEvent(e);
@@ -2837,10 +2835,7 @@ private boolean updatePartitionDataFromStream()
earlyStopTime = DateTimes.nowUtc().plus(tuningConfig.getRepartitionTransitionDuration());
log.info(
"Previous partition set [%s] has changed to [%s] - requesting that tasks stop after [%s] at [%s]",
- previousPartitionIds,
- partitionIds,
- tuningConfig.getRepartitionTransitionDuration(),
- earlyStopTime
+ previousPartitionIds, partitionIds, tuningConfig.getRepartitionTransitionDuration(), earlyStopTime
);
break;
}
@@ -3161,57 +3156,52 @@ private void checkTaskDuration() throws ExecutionException, InterruptedException
final List>> futures = new ArrayList<>();
final List futureGroupIds = new ArrayList<>();
- boolean stopTasksEarly;
- if (earlyStopTime != null && (earlyStopTime.isBeforeNow() || earlyStopTime.isEqualNow())) {
- log.info("Early stop requested - signalling tasks to complete");
-
+ final boolean stopTasksEarly;
+ if (earlyStopTime != null && !earlyStopTime.isAfterNow()) {
+ log.info("Early stop requested, signalling tasks to complete.");
earlyStopTime = null;
stopTasksEarly = true;
} else {
stopTasksEarly = false;
}
- AtomicInteger stoppedTasks = new AtomicInteger();
+ final AtomicInteger numStoppedTasks = new AtomicInteger();
// Sort task groups by start time to prioritize early termination of earlier groups, then iterate for processing
- activelyReadingTaskGroups
- .entrySet().stream().sorted(
+ activelyReadingTaskGroups.entrySet().stream().sorted(
Comparator.comparingLong(
- (Entry entry) ->
- computeEarliestTaskStartTime(entry.getValue())
- .getMillis()))
+ taskGroupEntry -> computeEarliestTaskStartTime(taskGroupEntry.getValue()).getMillis()
+ )
+ )
.forEach(entry -> {
Integer groupId = entry.getKey();
TaskGroup group = entry.getValue();
- if (stopTasksEarly) {
+ final DateTime earliestTaskStart = computeEarliestTaskStartTime(group);
+ final Duration runDuration = Duration.millis(DateTimes.nowUtc().getMillis() - earliestTaskStart.getMillis());
+ if (stopTasksEarly || group.getHandoffEarly()) {
+ // If handoffEarly has been set, stop tasks irrespective of stopTaskCount
log.info(
- "Stopping task group [%d] early. It has run for [%s]",
- groupId,
- ioConfig.getTaskDuration()
+ "Stopping taskGroup[%d] early after running for duration[%s].",
+ groupId, runDuration
);
futureGroupIds.add(groupId);
futures.add(checkpointTaskGroup(group, true));
- } else {
- DateTime earliestTaskStart = computeEarliestTaskStartTime(group);
-
- if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow() || group.getHandoffEarly()) {
- // if this task has run longer than the configured duration
- // as long as the pending task groups are less than the configured stop task count.
- // If shutdownEarly has been set, ignore stopTaskCount since this is a manual operator action.
- if (pendingCompletionTaskGroups.values()
- .stream()
- .mapToInt(CopyOnWriteArrayList::size)
- .sum() + stoppedTasks.get()
- < ioConfig.getMaxAllowedStops() || group.getHandoffEarly()) {
- log.info(
- "Task group [%d] has run for [%s]. Stopping.",
- groupId,
- ioConfig.getTaskDuration()
- );
- futureGroupIds.add(groupId);
- futures.add(checkpointTaskGroup(group, true));
- stoppedTasks.getAndIncrement();
- }
+ if (group.getHandoffEarly()) {
+ numStoppedTasks.getAndIncrement();
+ }
+ } else if (earliestTaskStart.plus(ioConfig.getTaskDuration()).isBeforeNow()) {
+ // Stop this task group if it has run longer than the configured duration
+ // and the pending task groups are less than the configured stop task count.
+ int numPendingCompletionTaskGroups = pendingCompletionTaskGroups.values().stream()
+ .mapToInt(List::size).sum();
+ if (numPendingCompletionTaskGroups + numStoppedTasks.get() < ioConfig.getMaxAllowedStops()) {
+ log.info(
+ "Stopping taskGroup[%d] as it has already run for duration[%s], configured task duration[%s].",
+ groupId, runDuration, ioConfig.getTaskDuration()
+ );
+ futureGroupIds.add(groupId);
+ futures.add(checkpointTaskGroup(group, true));
+ numStoppedTasks.getAndIncrement();
}
}
});
@@ -3384,7 +3374,7 @@ public Map apply(List> iTasks = taskGroup.tasks.entrySet().iterator();
while (iTasks.hasNext()) {
@@ -3589,7 +3579,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep
// stop and remove bad tasks from the task group
if (!isTaskCurrent(groupId, taskId, activeTaskMap)) {
- log.info("Stopping task [%s] which does not match the expected sequence range and ingestion spec", taskId);
+ log.info("Stopping task[%s] as it does not match the expected sequence range and ingestion spec.", taskId);
futures.add(stopTask(taskId, false));
iTasks.remove();
continue;
@@ -3613,7 +3603,7 @@ private void checkCurrentTaskState() throws ExecutionException, InterruptedExcep
break;
}
}
- log.debug("Task group [%d] post-pruning: %s", groupId, taskGroup.taskIds());
+ log.debug("After pruning, taskGroup[%d] has tasks[%s].", groupId, taskGroup.taskIds());
}
// Ignore return value; just await.
@@ -3627,10 +3617,9 @@ private void checkIfStreamInactiveAndTurnSupervisorIdle()
}
Map latestSequencesFromStream = getLatestSequencesFromStream();
- long nowTime = Instant.now().toEpochMilli();
- boolean idle;
- long idleTime;
-
+ final long nowTime = DateTimes.nowUtc().getMillis();
+ final boolean idle;
+ final long idleTime;
if (lastActiveTimeMillis > 0
&& previousSequencesFromStream.equals(latestSequencesFromStream)
&& computeTotalLag() == 0) {
@@ -3684,7 +3673,7 @@ private void createNewTasks() throws JsonProcessingException
// check that there is a current task group for each group of partitions in [partitionGroups]
for (Integer groupId : partitionGroups.keySet()) {
if (!activelyReadingTaskGroups.containsKey(groupId)) {
- log.info("Creating new task group [%d] for partitions %s", groupId, partitionGroups.get(groupId));
+ log.info("Creating new taskGroup[%d] for partitions[%s].", groupId, partitionGroups.get(groupId));
Optional minimumMessageTime;
if (ioConfig.getLateMessageRejectionStartDateTime().isPresent()) {
minimumMessageTime = Optional.of(ioConfig.getLateMessageRejectionStartDateTime().get());
@@ -3771,13 +3760,13 @@ private void createNewTasks() throws JsonProcessingException
if (taskGroup.startingSequences == null ||
taskGroup.startingSequences.size() == 0 ||
taskGroup.startingSequences.values().stream().allMatch(x -> x == null || isEndOfShard(x))) {
- log.debug("Nothing to read in any partition for taskGroup [%d], skipping task creation", groupId);
+ log.debug("Nothing to read in any partition for taskGroup[%d], skipping task creation.", groupId);
continue;
}
if (ioConfig.getReplicas() > taskGroup.tasks.size()) {
log.info(
- "Number of tasks [%d] does not match configured numReplicas [%d] in task group [%d], creating more tasks",
+ "Number of tasks[%d] does not match configured numReplicas[%d] in taskGroup[%d], creating more tasks.",
taskGroup.tasks.size(), ioConfig.getReplicas(), groupId
);
createTasksForGroup(groupId, ioConfig.getReplicas() - taskGroup.tasks.size());
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java
index 273339fe7f3fd..a62d477ef09e2 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndAppendTest.java
@@ -138,7 +138,7 @@ public void setup()
);
taskQueue = new TaskQueue(
new TaskLockConfig(),
- new TaskQueueConfig(null, new Period(0L), null, null, null),
+ new TaskQueueConfig(null, new Period(0L), null, null, null, null),
new DefaultTaskConfig(),
getTaskStorage(),
taskRunner,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java
index 7da5a3d19fe84..4f599e2449390 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/common/task/concurrent/ConcurrentReplaceAndStreamingAppendTest.java
@@ -151,7 +151,7 @@ public void setUpIngestionTestBase() throws IOException
);
taskQueue = new TaskQueue(
new TaskLockConfig(),
- new TaskQueueConfig(null, new Period(0L), null, null, null),
+ new TaskQueueConfig(null, new Period(0L), null, null, null, null),
new DefaultTaskConfig(),
getTaskStorage(),
taskRunner,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java
index 9d408365daa6b..69c2e1fe8cab8 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskLockConfigTest.java
@@ -109,7 +109,7 @@ public boolean isForceTimeChunkLock()
} else {
lockConfig = new TaskLockConfig();
}
- final TaskQueueConfig queueConfig = new TaskQueueConfig(null, null, null, null, null);
+ final TaskQueueConfig queueConfig = new TaskQueueConfig(null, null, null, null, null, null);
final TaskRunner taskRunner = EasyMock.createNiceMock(RemoteTaskRunner.class);
final TaskActionClientFactory actionClientFactory = EasyMock.createNiceMock(LocalTaskActionClientFactory.class);
final TaskLockbox lockbox = new TaskLockbox(taskStorage, new TestIndexerMetadataStorageCoordinator());
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java
index b9dcf97a54ca5..f67e9fc286144 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueScaleTest.java
@@ -122,7 +122,7 @@ public RetType submit(TaskAction taskAction)
taskQueue = new TaskQueue(
new TaskLockConfig(),
- new TaskQueueConfig(null, Period.millis(1), null, null, null),
+ new TaskQueueConfig(null, Period.millis(1), null, null, null, null),
new DefaultTaskConfig(),
taskStorage,
taskRunner,
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java
index 6125a4d73bea6..8984e73b6b9c7 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/TaskQueueTest.java
@@ -71,6 +71,7 @@
import org.apache.druid.indexing.worker.Worker;
import org.apache.druid.indexing.worker.config.WorkerConfig;
import org.apache.druid.jackson.DefaultObjectMapper;
+import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.granularity.Granularity;
@@ -94,6 +95,7 @@
import javax.annotation.Nullable;
import java.io.IOException;
import java.net.URI;
+import java.util.Arrays;
import java.util.Collections;
import java.util.HashMap;
import java.util.List;
@@ -119,7 +121,7 @@ public void setUpIngestionTestBase() throws IOException
taskQueue = new TaskQueue(
new TaskLockConfig(),
- new TaskQueueConfig(3, null, null, null, null),
+ new TaskQueueConfig(3, null, null, null, null, null),
new DefaultTaskConfig()
{
@Override
@@ -214,6 +216,73 @@ public void testAddThrowsExceptionWhenQueueIsFull()
);
}
+ @Test
+ public void testAddThrowsExceptionWhenPayloadIsTooLarge()
+ {
+ HumanReadableBytes maxPayloadSize10Mib = HumanReadableBytes.valueOf(10 * 1024 * 1024);
+ TaskQueue maxPayloadTaskQueue = new TaskQueue(
+ new TaskLockConfig(),
+ new TaskQueueConfig(3, null, null, null, null, maxPayloadSize10Mib),
+ new DefaultTaskConfig()
+ {
+ @Override
+ public Map getContext()
+ {
+ return defaultTaskContext;
+ }
+ },
+ getTaskStorage(),
+ new SimpleTaskRunner(),
+ actionClientFactory,
+ getLockbox(),
+ serviceEmitter,
+ getObjectMapper(),
+ new NoopTaskContextEnricher()
+ );
+ maxPayloadTaskQueue.setActive();
+
+ // 1 MB is not too large
+ char[] context = new char[1024 * 1024];
+ Arrays.fill(context, 'a');
+ maxPayloadTaskQueue.add(
+ new TestTask(
+ "tx",
+ Intervals.of("2021-01/P1M"),
+ ImmutableMap.of(
+ "contextKey", new String(context)
+ )
+ )
+ );
+
+ // 100 MB is too large
+ char[] contextLarge = new char[100 * 1024 * 1024];
+ Arrays.fill(contextLarge, 'a');
+
+ Assert.assertThrows(
+ DruidException.class,
+ () -> maxPayloadTaskQueue.add(
+ new TestTask(
+ "tx2",
+ Intervals.of("2021-01/P1M"),
+ ImmutableMap.of(
+ "contextKey", new String(contextLarge)
+ )
+ )
+ )
+ );
+
+ // If no limit is set, don't throw anything
+ taskQueue.add(
+ new TestTask(
+ "tx3",
+ Intervals.of("2021-01/P1M"),
+ ImmutableMap.of(
+ "contextKey", new String(contextLarge)
+ )
+ )
+ );
+ }
+
@Test
public void testAddedTaskUsesLineageBasedSegmentAllocationByDefault()
{
@@ -336,7 +405,7 @@ public void testKilledTasksEmitRuntimeMetricWithHttpRemote() throws InterruptedE
EasyMock.replay(workerHolder);
final TaskQueue taskQueue = new TaskQueue(
new TaskLockConfig(),
- new TaskQueueConfig(null, null, null, null, null),
+ new TaskQueueConfig(null, null, null, null, null, null),
new DefaultTaskConfig(),
getTaskStorage(),
taskRunner,
@@ -424,7 +493,7 @@ public void testGetTaskStatus()
final TaskQueue taskQueue = new TaskQueue(
new TaskLockConfig(),
- new TaskQueueConfig(null, null, null, null, null),
+ new TaskQueueConfig(null, null, null, null, null, null),
new DefaultTaskConfig(),
taskStorage,
taskRunner,
@@ -469,7 +538,7 @@ public void testGetActiveTaskRedactsPassword() throws JsonProcessingException
final TaskQueue taskQueue = new TaskQueue(
new TaskLockConfig(),
- new TaskQueueConfig(null, null, null, null, null),
+ new TaskQueueConfig(null, null, null, null, null, null),
new DefaultTaskConfig(),
taskStorage,
EasyMock.createMock(HttpRemoteTaskRunner.class),
diff --git a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java
index 862d3aace3ce1..aab3639b2e635 100644
--- a/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java
+++ b/indexing-service/src/test/java/org/apache/druid/indexing/overlord/http/OverlordTest.java
@@ -236,7 +236,7 @@ public MockTaskRunner get()
taskMaster = new TaskMaster(
new TaskLockConfig(),
- new TaskQueueConfig(null, new Period(1), null, new Period(10), null),
+ new TaskQueueConfig(null, new Period(1), null, new Period(10), null, null),
new DefaultTaskConfig(),
taskLockbox,
taskStorage,
diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuMonitor.java
index af2408a156ed7..9486bc5328224 100644
--- a/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuMonitor.java
+++ b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupCpuMonitor.java
@@ -20,28 +20,64 @@
package org.apache.druid.java.util.metrics;
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.Cpu;
+import org.apache.druid.java.util.metrics.cgroups.ProcSelfCgroupDiscoverer;
+import java.io.BufferedReader;
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.nio.charset.StandardCharsets;
+import java.time.Instant;
import java.util.Map;
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 dimensions;
+ private Long userHz;
+ private KeyedDiff jiffies = new KeyedDiff();
+ private long prevJiffiesSnapshotAt = 0;
public CgroupCpuMonitor(CgroupDiscoverer cgroupDiscoverer, final Map dimensions, String feed)
{
super(feed);
this.cgroupDiscoverer = cgroupDiscoverer;
this.dimensions = dimensions;
+ try {
+ Process p = new ProcessBuilder("getconf", "CLK_TCK").start();
+ try (BufferedReader in = new BufferedReader(new InputStreamReader(p.getInputStream(), StandardCharsets.UTF_8))) {
+ String line = in.readLine();
+ if (line != null) {
+ userHz = Long.valueOf(line.trim());
+ }
+ }
+ }
+ catch (IOException | NumberFormatException e) {
+ LOG.warn(e, "Error getting the USER_HZ value");
+ }
+ finally {
+ if (userHz == null) {
+ LOG.warn("Using default value for USER_HZ");
+ userHz = DEFAULT_USER_HZ;
+ }
+ }
}
public CgroupCpuMonitor(final Map dimensions, String feed)
{
- this(null, dimensions, feed);
+ this(new ProcSelfCgroupDiscoverer(), dimensions, feed);
}
public CgroupCpuMonitor(final Map dimensions)
@@ -58,7 +94,8 @@ public CgroupCpuMonitor()
public boolean doMonitor(ServiceEmitter emitter)
{
final Cpu cpu = new Cpu(cgroupDiscoverer);
- final Cpu.CpuAllocationMetric cpuSnapshot = cpu.snapshot();
+ final Cpu.CpuMetrics cpuSnapshot = cpu.snapshot();
+ long now = Instant.now().getEpochSecond();
final ServiceMetricEvent.Builder builder = builder();
MonitorUtils.addDimensionsToBuilder(builder, dimensions);
@@ -68,6 +105,26 @@ public boolean doMonitor(ServiceEmitter emitter)
computeProcessorQuota(cpuSnapshot.getQuotaUs(), cpuSnapshot.getPeriodUs())
));
+ long elapsedJiffiesSnapshotSecs = now - prevJiffiesSnapshotAt;
+ if (elapsedJiffiesSnapshotSecs > 0) {
+ prevJiffiesSnapshotAt = now;
+ final Map elapsedJiffies = jiffies.to(
+ "usage",
+ ImmutableMap.builder()
+ .put(USER, cpuSnapshot.getUserJiffies())
+ .put(SYSTEM, cpuSnapshot.getSystemJiffies())
+ .put(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));
+ }
+ }
return true;
}
diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupDiskMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupDiskMonitor.java
new file mode 100644
index 0000000000000..07ead5a68c262
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupDiskMonitor.java
@@ -0,0 +1,85 @@
+/*
+ * 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.collect.ImmutableMap;
+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.ProcSelfCgroupDiscoverer;
+
+import java.util.Map;
+
+public class CgroupDiskMonitor extends FeedDefiningMonitor
+{
+ final CgroupDiscoverer cgroupDiscoverer;
+ final Map dimensions;
+ private final KeyedDiff diff = new KeyedDiff();
+
+ public CgroupDiskMonitor(CgroupDiscoverer cgroupDiscoverer, final Map dimensions, String feed)
+ {
+ super(feed);
+ this.cgroupDiscoverer = cgroupDiscoverer;
+ this.dimensions = dimensions;
+ }
+
+ public CgroupDiskMonitor(final Map dimensions, String feed)
+ {
+ this(new ProcSelfCgroupDiscoverer(), dimensions, feed);
+ }
+
+ public CgroupDiskMonitor(final Map dimensions)
+ {
+ this(dimensions, DEFAULT_METRICS_FEED);
+ }
+
+ public CgroupDiskMonitor()
+ {
+ this(ImmutableMap.of());
+ }
+
+ @Override
+ public boolean doMonitor(ServiceEmitter emitter)
+ {
+ Map snapshot = new Disk(cgroupDiscoverer).snapshot();
+ for (Map.Entry entry : snapshot.entrySet()) {
+ final Map stats = diff.to(
+ entry.getKey(),
+ ImmutableMap.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())
+ .build()
+ );
+
+ if (stats != null) {
+ final ServiceMetricEvent.Builder builder = builder()
+ .setDimension("diskName", entry.getValue().getDiskName());
+ MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+ for (Map.Entry stat : stats.entrySet()) {
+ emitter.emit(builder.setMetric(stat.getKey(), stat.getValue()));
+ }
+ }
+ }
+ return true;
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitor.java b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitor.java
index d282cf5cddd0d..442087198ae7e 100644
--- a/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitor.java
+++ b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitor.java
@@ -25,6 +25,7 @@
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.Memory;
+import org.apache.druid.java.util.metrics.cgroups.ProcSelfCgroupDiscoverer;
import java.util.Map;
@@ -42,7 +43,7 @@ public CgroupMemoryMonitor(CgroupDiscoverer cgroupDiscoverer, final Map dimensions, String feed)
{
- this(null, dimensions, feed);
+ this(new ProcSelfCgroupDiscoverer(), dimensions, feed);
}
public CgroupMemoryMonitor(final Map dimensions)
@@ -60,16 +61,18 @@ public boolean doMonitor(ServiceEmitter emitter)
{
final Memory memory = new Memory(cgroupDiscoverer);
final Memory.MemoryStat stat = memory.snapshot();
+ final ServiceMetricEvent.Builder builder = builder();
+ MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+ emitter.emit(builder.setMetric("cgroup/memory/usage/bytes", stat.getUsage()));
+ emitter.emit(builder.setMetric("cgroup/memory/limit/bytes", stat.getLimit()));
+
stat.getMemoryStats().forEach((key, value) -> {
- final ServiceMetricEvent.Builder builder = builder();
- MonitorUtils.addDimensionsToBuilder(builder, dimensions);
// See https://www.kernel.org/doc/Documentation/cgroup-v1/memory.txt
// There are inconsistent units for these. Most are bytes.
emitter.emit(builder.setMetric(StringUtils.format("cgroup/memory/%s", key), value));
});
stat.getNumaMemoryStats().forEach((key, value) -> {
- final ServiceMetricEvent.Builder builder = builder().setDimension("numaZone", Long.toString(key));
- MonitorUtils.addDimensionsToBuilder(builder, dimensions);
+ builder().setDimension("numaZone", Long.toString(key));
value.forEach((k, v) -> emitter.emit(builder.setMetric(StringUtils.format("cgroup/memory_numa/%s/pages", k), v)));
});
return true;
diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupUtil.java b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupUtil.java
index b9b29cdbdfcbc..bd1e8ed20f559 100644
--- a/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupUtil.java
+++ b/processing/src/main/java/org/apache/druid/java/util/metrics/CgroupUtil.java
@@ -19,10 +19,32 @@
package org.apache.druid.java.util.metrics;
+import com.google.common.primitives.Longs;
+import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
+
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.List;
+import java.util.Objects;
import java.util.regex.Pattern;
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 long readLongValue(CgroupDiscoverer discoverer, String cgroup, String fileName, long defaultValue)
+ {
+ try {
+ List lines = Files.readAllLines(Paths.get(discoverer.discover(cgroup).toString(), fileName));
+ return lines.stream().map(Longs::tryParse).filter(Objects::nonNull).findFirst().orElse(defaultValue);
+ }
+ catch (RuntimeException | IOException ex) {
+ LOG.warn(ex, "Unable to fetch %s", fileName);
+ return defaultValue;
+ }
+ }
}
diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Cpu.java b/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Cpu.java
index a742db2c3cb9e..05228bda04d9c 100644
--- a/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Cpu.java
+++ b/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Cpu.java
@@ -21,12 +21,13 @@
import com.google.common.primitives.Longs;
import org.apache.druid.java.util.common.logger.Logger;
+import org.apache.druid.java.util.metrics.CgroupUtil;
+import java.io.BufferedReader;
import java.io.IOException;
import java.nio.file.Files;
import java.nio.file.Paths;
-import java.util.List;
-import java.util.Objects;
+import java.util.regex.Pattern;
/**
* Collect CPU share and quota information from cpu cgroup files.
@@ -35,6 +36,7 @@ public class 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";
private static final String CPU_PERIOD_FILE = "cpu.cfs_period_us";
@@ -51,28 +53,43 @@ public Cpu(CgroupDiscoverer cgroupDiscoverer)
*
* @return A snapshot with the data populated.
*/
- public CpuAllocationMetric snapshot()
+ public CpuMetrics snapshot()
{
- return new CpuAllocationMetric(
- readLongValue(CPU_SHARES_FILE, -1),
- readLongValue(CPU_QUOTA_FILE, 0),
- readLongValue(CPU_PERIOD_FILE, 0)
- );
- }
-
- private long readLongValue(String fileName, long defaultValeue)
- {
- try {
- List lines = Files.readAllLines(Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), fileName));
- return lines.stream().map(Longs::tryParse).filter(Objects::nonNull).findFirst().orElse(defaultValeue);
+ long userJiffies = -1L;
+ long systemJiffies = -1L;
+ try (final BufferedReader reader = Files.newBufferedReader(
+ Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), CPUACCT_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;
+ }
+ switch (parts[0]) {
+ case "user":
+ userJiffies = Longs.tryParse(parts[1]);
+ break;
+ case "system":
+ systemJiffies = Longs.tryParse(parts[1]);
+ break;
+ }
+ }
}
- catch (RuntimeException | IOException ex) {
- LOG.error(ex, "Unable to fetch %s", fileName);
- return defaultValeue;
+ catch (IOException | RuntimeException ex) {
+ LOG.error(ex, "Unable to fetch cpu snapshot");
}
+
+
+ return new CpuMetrics(
+ CgroupUtil.readLongValue(cgroupDiscoverer, CGROUP, CPU_SHARES_FILE, -1),
+ CgroupUtil.readLongValue(cgroupDiscoverer, CGROUP, CPU_QUOTA_FILE, 0),
+ CgroupUtil.readLongValue(cgroupDiscoverer, CGROUP, CPU_PERIOD_FILE, 0),
+ systemJiffies, userJiffies
+ );
}
- public static class CpuAllocationMetric
+ public static class CpuMetrics
{
// Maps to cpu.shares - the share of CPU given to the process
private final long shares;
@@ -85,11 +102,19 @@ public static class CpuAllocationMetric
// bandwidth decisions
private final long periodUs;
- CpuAllocationMetric(long shares, long quotaUs, long periodUs)
+ // Maps to user value at cpuacct.stat
+ private final long userJiffies;
+
+ // Maps to system value at cpuacct.stat
+ private final long systemJiffies;
+
+ CpuMetrics(long shares, long quotaUs, long periodUs, long systemJiffis, long userJiffies)
{
this.shares = shares;
this.quotaUs = quotaUs;
this.periodUs = periodUs;
+ this.userJiffies = userJiffies;
+ this.systemJiffies = systemJiffis;
}
public final long getShares()
@@ -106,5 +131,20 @@ public final long getPeriodUs()
{
return periodUs;
}
+
+ public long getUserJiffies()
+ {
+ return userJiffies;
+ }
+
+ public long getSystemJiffies()
+ {
+ return systemJiffies;
+ }
+
+ public long getTotalJiffies()
+ {
+ return userJiffies + systemJiffies;
+ }
}
}
diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Disk.java b/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Disk.java
new file mode 100644
index 0000000000000..b74fb322ba95e
--- /dev/null
+++ b/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Disk.java
@@ -0,0 +1,179 @@
+/*
+ * 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.primitives.Longs;
+import org.apache.druid.java.util.common.logger.Logger;
+
+import java.io.BufferedReader;
+import java.io.File;
+import java.io.IOException;
+import java.nio.file.Files;
+import java.nio.file.Paths;
+import java.util.HashMap;
+import java.util.Map;
+import java.util.Objects;
+import java.util.regex.Pattern;
+
+public class Disk
+{
+ private static final Logger LOG = new Logger(Disk.class);
+ private static final String CGROUP = "blkio";
+ private static final String IO_SERVICED_FILE = "blkio.throttle.io_serviced";
+ private static final String IO_SERVICE_BYTES_FILE = "blkio.throttle.io_service_bytes";
+ private static final String READ = "Read";
+ private static final String WRITE = "Write";
+ private final CgroupDiscoverer cgroupDiscoverer;
+
+ public Disk(CgroupDiscoverer cgroupDiscoverer)
+ {
+ this.cgroupDiscoverer = cgroupDiscoverer;
+ }
+
+ /**
+ * Take a snapshot of cpu cgroup data
+ *
+ * @return A snapshot with the data populated.
+ */
+ public Map snapshot()
+ {
+ Map statsByDisk = new HashMap<>();
+
+ try (final BufferedReader reader = Files.newBufferedReader(
+ Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), IO_SERVICED_FILE))) {
+ for (String line = reader.readLine(); line != null; line = reader.readLine()) {
+ final String[] parts = line.split(Pattern.quote(" "));
+ if (parts.length != 3) {
+ // ignore
+ continue;
+ }
+ Metrics metrics = statsByDisk.computeIfAbsent(parts[0], majorMinor -> new Metrics(majorMinor));
+ switch (parts[1]) {
+ case WRITE:
+ metrics.writeCount = Longs.tryParse(parts[2]);
+ break;
+ case READ:
+ metrics.readCount = Longs.tryParse(parts[2]);
+ break;
+ }
+ }
+ }
+ catch (IOException | RuntimeException ex) {
+ LOG.error(ex, "Unable to fetch disk snapshot");
+ }
+
+ try (final BufferedReader reader = Files.newBufferedReader(
+ Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), IO_SERVICE_BYTES_FILE))) {
+ for (String line = reader.readLine(); line != null; line = reader.readLine()) {
+ final String[] parts = line.split(Pattern.quote(" "));
+ if (parts.length != 3) {
+ // ignore
+ continue;
+ }
+ Metrics metrics = statsByDisk.computeIfAbsent(parts[0], majorMinor -> new Metrics(majorMinor));
+ switch (parts[1]) {
+ case WRITE:
+ metrics.writeBytes = Longs.tryParse(parts[2]);
+ break;
+ case READ:
+ metrics.readBytes = Longs.tryParse(parts[2]);
+ break;
+ }
+ }
+ }
+ catch (IOException | RuntimeException ex) {
+ LOG.error(ex, "Unable to fetch memory snapshot");
+ }
+
+ return statsByDisk;
+ }
+
+ public static class Metrics
+ {
+ String diskName;
+ long readCount;
+ long writeCount;
+ long readBytes;
+ long writeBytes;
+
+ public Metrics(String majorMinor)
+ {
+ try {
+ File deviceFile = new File("/sys/dev/block/" + majorMinor);
+ if (deviceFile.exists()) {
+ diskName = deviceFile.getCanonicalPath();
+ }
+ }
+ catch (IOException e) {
+ LOG.warn("Unable to get disk name for " + majorMinor);
+ }
+ finally {
+ if (diskName == null) {
+ diskName = majorMinor;
+ }
+ }
+ }
+
+ public long getReadCount()
+ {
+ return readCount;
+ }
+ public long getWriteCount()
+ {
+ return writeCount;
+ }
+
+ public long getReadBytes()
+ {
+ return readBytes;
+ }
+ public long getWriteBytes()
+ {
+ return writeBytes;
+ }
+ public String getDiskName()
+ {
+ return diskName;
+ }
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ Metrics metrics = (Metrics) o;
+ return readCount == metrics.readCount
+ && writeCount == metrics.writeCount
+ && readBytes == metrics.readBytes
+ && writeBytes == metrics.writeBytes
+ && Objects.equals(diskName, metrics.diskName);
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(diskName, readCount, writeCount, readBytes, writeBytes);
+ }
+ }
+}
diff --git a/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Memory.java b/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Memory.java
index bc4487bb478d4..01e7fed9c9f5c 100644
--- a/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Memory.java
+++ b/processing/src/main/java/org/apache/druid/java/util/metrics/cgroups/Memory.java
@@ -22,6 +22,7 @@
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.metrics.CgroupUtil;
import java.io.BufferedReader;
import java.io.IOException;
@@ -36,6 +37,8 @@ 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,6 +50,8 @@ public Memory(CgroupDiscoverer cgroupDiscoverer)
public MemoryStat snapshot()
{
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);
try (final BufferedReader reader = Files.newBufferedReader(
Paths.get(cgroupDiscoverer.discover(CGROUP).toString(), CGROUP_MEMORY_FILE)
@@ -102,6 +107,8 @@ public static class MemoryStat
{
private final Map memoryStats = new HashMap<>();
private final Map> numaMemoryStats = new HashMap<>();
+ private long usage;
+ private long limit;
public Map getMemoryStats()
{
@@ -113,5 +120,15 @@ public Map> getNumaMemoryStats()
// They can modify the inner map... but why?
return ImmutableMap.copyOf(numaMemoryStats);
}
+
+ public long getUsage()
+ {
+ return usage;
+ }
+
+ public long getLimit()
+ {
+ return limit;
+ }
}
}
diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuMonitorTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuMonitorTest.java
index d6379840a4951..04f147481288a 100644
--- a/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuMonitorTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupCpuMonitorTest.java
@@ -20,6 +20,7 @@
package org.apache.druid.java.util.metrics;
import com.google.common.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.emitter.core.Event;
import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
@@ -36,6 +37,7 @@
import java.io.IOException;
import java.util.List;
import java.util.Map;
+import java.util.stream.Collectors;
public class CgroupCpuMonitorTest
{
@@ -45,6 +47,7 @@ public class CgroupCpuMonitorTest
public TemporaryFolder temporaryFolder = new TemporaryFolder();
private File procDir;
private File cgroupDir;
+ private File statFile;
private CgroupDiscoverer discoverer;
@Before
@@ -60,13 +63,15 @@ public void setUp() throws IOException
);
FileUtils.mkdirp(cpuDir);
+ statFile = new File(cpuDir, "cpuacct.stat");
TestUtils.copyOrReplaceResource("/cpu.shares", new File(cpuDir, "cpu.shares"));
TestUtils.copyOrReplaceResource("/cpu.cfs_quota_us", new File(cpuDir, "cpu.cfs_quota_us"));
TestUtils.copyOrReplaceResource("/cpu.cfs_period_us", new File(cpuDir, "cpu.cfs_period_us"));
+ TestUtils.copyOrReplaceResource("/cpuacct.stat", statFile);
}
@Test
- public void testMonitor()
+ public void testMonitor() throws IOException, InterruptedException
{
final CgroupCpuMonitor monitor = new CgroupCpuMonitor(discoverer, ImmutableMap.of(), "some_feed");
final StubServiceEmitter emitter = new StubServiceEmitter("service", "host");
@@ -79,6 +84,26 @@ public void testMonitor()
Assert.assertEquals(1024L, sharesEvent.get("value"));
Assert.assertEquals("cgroup/cpu/cores_quota", coresEvent.get("metric"));
Assert.assertEquals(3.0D, coresEvent.get("value"));
+ emitter.flush();
+
+ TestUtils.copyOrReplaceResource("/cpuacct.stat-2", statFile);
+ // We need to pass atleast a second for the calculation to trigger
+ // to avoid divide by zero.
+ Thread.sleep(1000);
+
+ Assert.assertTrue(monitor.doMonitor(emitter));
+ Assert.assertTrue(
+ emitter
+ .getEvents()
+ .stream()
+ .map(e -> e.toMap().get("metric"))
+ .collect(Collectors.toList())
+ .containsAll(
+ ImmutableSet.of(
+ CgroupCpuMonitor.TOTAL_USAGE_METRIC,
+ CgroupCpuMonitor.USER_USAGE_METRIC,
+ CgroupCpuMonitor.SYS_USAGE_METRIC
+ )));
}
@Test
diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupDiskMonitorTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupDiskMonitorTest.java
new file mode 100644
index 0000000000000..21a834587292b
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupDiskMonitorTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.collect.ImmutableMap;
+import org.apache.druid.java.util.common.FileUtils;
+import org.apache.druid.java.util.metrics.cgroups.CgroupDiscoverer;
+import org.apache.druid.java.util.metrics.cgroups.ProcCgroupDiscoverer;
+import org.apache.druid.java.util.metrics.cgroups.TestUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.io.IOException;
+
+public class CgroupDiskMonitorTest
+{
+ @Rule
+ public TemporaryFolder temporaryFolder = new TemporaryFolder();
+ private File procDir;
+ private File cgroupDir;
+ private File servicedFile;
+ private File serviceBytesFile;
+ private CgroupDiscoverer discoverer;
+
+ @Before
+ public void setUp() throws IOException
+ {
+ cgroupDir = temporaryFolder.newFolder();
+ procDir = temporaryFolder.newFolder();
+ discoverer = new ProcCgroupDiscoverer(procDir.toPath());
+ TestUtils.setUpCgroups(procDir, cgroupDir);
+ final File blkioDir = new File(
+ cgroupDir,
+ "blkio/system.slice/some.service/"
+ );
+
+ FileUtils.mkdirp(blkioDir);
+ servicedFile = new File(blkioDir, "blkio.throttle.io_serviced");
+ serviceBytesFile = new File(blkioDir, "blkio.throttle.io_service_bytes");
+ TestUtils.copyResource("/blkio.throttle.io_service_bytes", serviceBytesFile);
+ TestUtils.copyResource("/blkio.throttle.io_serviced", servicedFile);
+ }
+
+ @Test
+ public void testMonitor() throws IOException
+ {
+ final CgroupDiskMonitor monitor = new CgroupDiskMonitor(discoverer, ImmutableMap.of(), "some_feed");
+ final StubServiceEmitter emitter = new StubServiceEmitter("service", "host");
+ Assert.assertTrue(monitor.doMonitor(emitter));
+ Assert.assertEquals(0, emitter.getEvents().size());
+
+ TestUtils.copyOrReplaceResource("/blkio.throttle.io_service_bytes-2", serviceBytesFile);
+ TestUtils.copyOrReplaceResource("/blkio.throttle.io_serviced-2", servicedFile);
+
+ Assert.assertTrue(monitor.doMonitor(emitter));
+ Assert.assertEquals(8, emitter.getEvents().size());
+ Assert.assertTrue(
+ emitter
+ .getEvents()
+ .stream()
+ .map(e -> e.toMap().get("value"))
+ .allMatch(val -> Long.valueOf(10).equals(val)));
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitorTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitorTest.java
index 39ff532a5a343..7827368ec9f52 100644
--- a/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitorTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/metrics/CgroupMemoryMonitorTest.java
@@ -61,6 +61,8 @@ public void setUp() throws IOException
FileUtils.mkdirp(memoryDir);
TestUtils.copyResource("/memory.stat", new File(memoryDir, "memory.stat"));
TestUtils.copyResource("/memory.numa_stat", new File(memoryDir, "memory.numa_stat"));
+ TestUtils.copyResource("/memory.usage_in_bytes", new File(memoryDir, "memory.usage_in_bytes"));
+ TestUtils.copyResource("/memory.limit_in_bytes", new File(memoryDir, "memory.limit_in_bytes"));
}
@Test
@@ -70,6 +72,6 @@ public void testMonitor()
final StubServiceEmitter emitter = new StubServiceEmitter("service", "host");
Assert.assertTrue(monitor.doMonitor(emitter));
final List actualEvents = emitter.getEvents();
- Assert.assertEquals(44, actualEvents.size());
+ Assert.assertEquals(46, actualEvents.size());
}
}
diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuTest.java
index 817b4664c27f5..1af0ba427bf91 100644
--- a/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/CpuTest.java
@@ -52,6 +52,8 @@ public void setUp() throws IOException
TestUtils.copyOrReplaceResource("/cpu.shares", new File(cpuDir, "cpu.shares"));
TestUtils.copyOrReplaceResource("/cpu.cfs_quota_us", new File(cpuDir, "cpu.cfs_quota_us"));
TestUtils.copyOrReplaceResource("/cpu.cfs_period_us", new File(cpuDir, "cpu.cfs_period_us"));
+ TestUtils.copyOrReplaceResource("/cpuacct.usage", new File(cpuDir, "cpuacct.usage"));
+ TestUtils.copyOrReplaceResource("/cpuacct.stat", new File(cpuDir, "cpuacct.stat"));
}
@Test
@@ -60,19 +62,24 @@ public void testWontCrash()
final Cpu cpu = new Cpu(cgroup -> {
throw new RuntimeException("Should still continue");
});
- final Cpu.CpuAllocationMetric metric = cpu.snapshot();
+ final Cpu.CpuMetrics metric = cpu.snapshot();
Assert.assertEquals(-1L, metric.getShares());
Assert.assertEquals(0, metric.getQuotaUs());
Assert.assertEquals(0, metric.getPeriodUs());
+ Assert.assertEquals(-1L, metric.getSystemJiffies());
+ Assert.assertEquals(-1L, metric.getUserJiffies());
}
@Test
public void testSimpleLoad()
{
final Cpu cpu = new Cpu(discoverer);
- final Cpu.CpuAllocationMetric snapshot = cpu.snapshot();
+ final Cpu.CpuMetrics snapshot = cpu.snapshot();
Assert.assertEquals(1024, snapshot.getShares());
Assert.assertEquals(300000, snapshot.getQuotaUs());
Assert.assertEquals(100000, snapshot.getPeriodUs());
+ Assert.assertEquals(143871L, snapshot.getSystemJiffies());
+ Assert.assertEquals(251183L, snapshot.getUserJiffies());
+ Assert.assertEquals(395054L, snapshot.getTotalJiffies());
}
}
diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/DiskTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/DiskTest.java
new file mode 100644
index 0000000000000..514b4cea9ea18
--- /dev/null
+++ b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/DiskTest.java
@@ -0,0 +1,85 @@
+/*
+ * 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.collect.ImmutableMap;
+import com.google.common.collect.ImmutableSet;
+import org.apache.druid.java.util.common.FileUtils;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import java.io.File;
+import java.util.Map;
+
+public class DiskTest
+{
+ @Rule
+ public TemporaryFolder temporaryFolder = new TemporaryFolder();
+ private File procDir;
+ private File cgroupDir;
+ private CgroupDiscoverer discoverer;
+
+ @Before
+ public void setUp() throws Exception
+ {
+ cgroupDir = temporaryFolder.newFolder();
+ procDir = temporaryFolder.newFolder();
+ discoverer = new ProcCgroupDiscoverer(procDir.toPath());
+ TestUtils.setUpCgroups(procDir, cgroupDir);
+ final File blkioDir = new File(
+ cgroupDir,
+ "blkio/system.slice/some.service"
+ );
+
+ FileUtils.mkdirp(blkioDir);
+ TestUtils.copyResource("/blkio.throttle.io_serviced", new File(blkioDir, "blkio.throttle.io_serviced"));
+ TestUtils.copyResource("/blkio.throttle.io_service_bytes", new File(blkioDir, "blkio.throttle.io_service_bytes"));
+ }
+
+ @Test
+ public void testWontCrash()
+ {
+ final Disk disk = new Disk((cgroup) -> {
+ throw new RuntimeException("shouldContinue");
+ });
+ final Map stats = disk.snapshot();
+ Assert.assertEquals(ImmutableMap.of(), stats);
+ }
+
+ @Test
+ public void testSimpleSnapshot()
+ {
+ final Map stats = new Disk(discoverer).snapshot();
+ Assert.assertEquals(ImmutableSet.of("259:0", "259:7"), stats.keySet());
+
+ Assert.assertEquals(stats.get("259:0").getReadCount(), 98L);
+ Assert.assertEquals(stats.get("259:0").getWriteCount(), 756L);
+ Assert.assertEquals(stats.get("259:0").getReadBytes(), 55000L);
+ Assert.assertEquals(stats.get("259:0").getWriteBytes(), 6208512L);
+
+ Assert.assertEquals(stats.get("259:7").getReadCount(), 26L);
+ Assert.assertEquals(stats.get("259:7").getWriteCount(), 0L);
+ Assert.assertEquals(stats.get("259:7").getReadBytes(), 1773568L);
+ Assert.assertEquals(stats.get("259:7").getWriteBytes(), 0L);
+ }
+}
diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/MemoryTest.java b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/MemoryTest.java
index 87e68d521d68f..bd5b10c7225a0 100644
--- a/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/MemoryTest.java
+++ b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/MemoryTest.java
@@ -57,6 +57,8 @@ public void setUp() throws Exception
FileUtils.mkdirp(memoryDir);
TestUtils.copyResource("/memory.stat", new File(memoryDir, "memory.stat"));
TestUtils.copyResource("/memory.numa_stat", new File(memoryDir, "memory.numa_stat"));
+ TestUtils.copyResource("/memory.usage_in_bytes", new File(memoryDir, "memory.usage_in_bytes"));
+ TestUtils.copyResource("/memory.limit_in_bytes", new File(memoryDir, "memory.limit_in_bytes"));
}
@Test
@@ -75,6 +77,10 @@ public void testSimpleSnapshot()
{
final Memory memory = new Memory(discoverer);
final Memory.MemoryStat stat = memory.snapshot();
+
+ Assert.assertEquals(5000000, stat.getUsage());
+ Assert.assertEquals(8000000, stat.getLimit());
+
final Map expectedMemoryStats = new HashMap<>();
expectedMemoryStats.put("inactive_anon", 0L);
expectedMemoryStats.put("total_pgfault", 13137L);
diff --git a/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java
index e522ff679621a..1aca9e7f7c3fa 100644
--- a/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java
+++ b/processing/src/test/java/org/apache/druid/java/util/metrics/cgroups/TestUtils.java
@@ -26,6 +26,7 @@
import java.io.File;
import java.io.IOException;
import java.nio.file.Files;
+import java.nio.file.StandardCopyOption;
public class TestUtils
{
@@ -65,7 +66,7 @@ public static void copyResource(String resource, File out) throws IOException
public static void copyOrReplaceResource(String resource, File out) throws IOException
{
- Files.copy(TestUtils.class.getResourceAsStream(resource), out.toPath());
+ Files.copy(TestUtils.class.getResourceAsStream(resource), out.toPath(), StandardCopyOption.REPLACE_EXISTING);
Assert.assertTrue(out.exists());
Assert.assertNotEquals(0, out.length());
}
diff --git a/processing/src/test/resources/blkio.throttle.io_service_bytes b/processing/src/test/resources/blkio.throttle.io_service_bytes
new file mode 100644
index 0000000000000..66f285b698f8a
--- /dev/null
+++ b/processing/src/test/resources/blkio.throttle.io_service_bytes
@@ -0,0 +1,13 @@
+259:0 Read 55000
+259:0 Write 6208512
+259:0 Sync 6208512
+259:0 Async 0
+259:0 Discard 0
+259:0 Total 6263512
+259:7 Read 1773568
+259:7 Write 0
+259:7 Sync 1773568
+259:7 Async 0
+259:7 Discard 0
+259:7 Total 1773568
+Total 8037080
\ No newline at end of file
diff --git a/processing/src/test/resources/blkio.throttle.io_service_bytes-2 b/processing/src/test/resources/blkio.throttle.io_service_bytes-2
new file mode 100644
index 0000000000000..b17b7335354e4
--- /dev/null
+++ b/processing/src/test/resources/blkio.throttle.io_service_bytes-2
@@ -0,0 +1,13 @@
+259:0 Read 55010
+259:0 Write 6208522
+259:0 Sync 6208522
+259:0 Async 0
+259:0 Discard 0
+259:0 Total 6263532
+259:7 Read 1773578
+259:7 Write 10
+259:7 Sync 1773588
+259:7 Async 0
+259:7 Discard 0
+259:7 Total 1773588
+Total 8037120
\ No newline at end of file
diff --git a/processing/src/test/resources/blkio.throttle.io_serviced b/processing/src/test/resources/blkio.throttle.io_serviced
new file mode 100644
index 0000000000000..7c0a9f9adfff4
--- /dev/null
+++ b/processing/src/test/resources/blkio.throttle.io_serviced
@@ -0,0 +1,13 @@
+259:0 Read 98
+259:0 Write 756
+259:0 Sync 854
+259:0 Async 0
+259:0 Discard 0
+259:0 Total 854
+259:7 Read 26
+259:7 Write 0
+259:7 Sync 26
+259:7 Async 0
+259:7 Discard 0
+259:7 Total 26
+Total 880
\ No newline at end of file
diff --git a/processing/src/test/resources/blkio.throttle.io_serviced-2 b/processing/src/test/resources/blkio.throttle.io_serviced-2
new file mode 100644
index 0000000000000..5d41b635d94e4
--- /dev/null
+++ b/processing/src/test/resources/blkio.throttle.io_serviced-2
@@ -0,0 +1,13 @@
+259:0 Read 108
+259:0 Write 766
+259:0 Sync 874
+259:0 Async 0
+259:0 Discard 0
+259:0 Total 874
+259:7 Read 36
+259:7 Write 10
+259:7 Sync 46
+259:7 Async 0
+259:7 Discard 0
+259:7 Total 46
+Total 920
\ No newline at end of file
diff --git a/processing/src/test/resources/cpuacct.stat b/processing/src/test/resources/cpuacct.stat
new file mode 100644
index 0000000000000..b049b71ef1343
--- /dev/null
+++ b/processing/src/test/resources/cpuacct.stat
@@ -0,0 +1,2 @@
+user 251183
+system 143871
\ No newline at end of file
diff --git a/processing/src/test/resources/cpuacct.stat-2 b/processing/src/test/resources/cpuacct.stat-2
new file mode 100644
index 0000000000000..c93ee183c2159
--- /dev/null
+++ b/processing/src/test/resources/cpuacct.stat-2
@@ -0,0 +1,2 @@
+user 251208
+system 143896
\ No newline at end of file
diff --git a/processing/src/test/resources/cpuacct.usage b/processing/src/test/resources/cpuacct.usage
new file mode 100644
index 0000000000000..d4375a28ae7dd
--- /dev/null
+++ b/processing/src/test/resources/cpuacct.usage
@@ -0,0 +1 @@
+5000000
\ No newline at end of file
diff --git a/processing/src/test/resources/memory.limit_in_bytes b/processing/src/test/resources/memory.limit_in_bytes
new file mode 100644
index 0000000000000..10f38ec0043dc
--- /dev/null
+++ b/processing/src/test/resources/memory.limit_in_bytes
@@ -0,0 +1 @@
+8000000
\ No newline at end of file
diff --git a/processing/src/test/resources/memory.usage_in_bytes b/processing/src/test/resources/memory.usage_in_bytes
new file mode 100644
index 0000000000000..d4375a28ae7dd
--- /dev/null
+++ b/processing/src/test/resources/memory.usage_in_bytes
@@ -0,0 +1 @@
+5000000
\ No newline at end of file
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java
index e39fde9656042..fe46eabb42616 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/AutoCompactionSnapshot.java
@@ -35,27 +35,32 @@ public enum AutoCompactionScheduleStatus
}
@JsonProperty
- private String dataSource;
+ private final String dataSource;
@JsonProperty
- private AutoCompactionScheduleStatus scheduleStatus;
+ private final AutoCompactionScheduleStatus scheduleStatus;
@JsonProperty
- private long bytesAwaitingCompaction;
+ private final long bytesAwaitingCompaction;
@JsonProperty
- private long bytesCompacted;
+ private final long bytesCompacted;
@JsonProperty
- private long bytesSkipped;
+ private final long bytesSkipped;
@JsonProperty
- private long segmentCountAwaitingCompaction;
+ private final long segmentCountAwaitingCompaction;
@JsonProperty
- private long segmentCountCompacted;
+ private final long segmentCountCompacted;
@JsonProperty
- private long segmentCountSkipped;
+ private final long segmentCountSkipped;
@JsonProperty
- private long intervalCountAwaitingCompaction;
+ private final long intervalCountAwaitingCompaction;
@JsonProperty
- private long intervalCountCompacted;
+ private final long intervalCountCompacted;
@JsonProperty
- private long intervalCountSkipped;
+ private final long intervalCountSkipped;
+
+ public static Builder builder(String dataSource)
+ {
+ return new Builder(dataSource, AutoCompactionScheduleStatus.RUNNING);
+ }
@JsonCreator
public AutoCompactionSnapshot(
@@ -185,8 +190,9 @@ public int hashCode()
public static class Builder
{
- private String dataSource;
- private AutoCompactionScheduleStatus scheduleStatus;
+ private final String dataSource;
+ private final AutoCompactionScheduleStatus scheduleStatus;
+
private long bytesAwaitingCompaction;
private long bytesCompacted;
private long bytesSkipped;
@@ -197,12 +203,18 @@ public static class Builder
private long intervalCountCompacted;
private long intervalCountSkipped;
-
- public Builder(
+ private Builder(
@NotNull String dataSource,
@NotNull AutoCompactionScheduleStatus scheduleStatus
)
{
+ if (dataSource == null || dataSource.isEmpty()) {
+ throw new ISE("Invalid dataSource name");
+ }
+ if (scheduleStatus == null) {
+ throw new ISE("scheduleStatus cannot be null");
+ }
+
this.dataSource = dataSource;
this.scheduleStatus = scheduleStatus;
this.bytesAwaitingCompaction = 0;
@@ -272,12 +284,6 @@ public Builder incrementIntervalCountSkipped(long incrementValue)
public AutoCompactionSnapshot build()
{
- if (dataSource == null || dataSource.isEmpty()) {
- throw new ISE("Invalid dataSource name");
- }
- if (scheduleStatus == null) {
- throw new ISE("scheduleStatus cannot be null");
- }
return new AutoCompactionSnapshot(
dataSource,
scheduleStatus,
diff --git a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
index bb88b86dbf84d..27f6d17638dd2 100644
--- a/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
+++ b/server/src/main/java/org/apache/druid/server/coordinator/duty/CompactSegments.java
@@ -376,17 +376,16 @@ private int submitCompactionTasks(
// As these segments will be compacted, we will aggregate the statistic to the Compacted statistics
AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent(
dataSourceName,
- k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING)
+ AutoCompactionSnapshot::builder
);
- snapshotBuilder.incrementBytesCompacted(
- segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum()
- );
- snapshotBuilder.incrementIntervalCountCompacted(
- segmentsToCompact.stream()
- .map(DataSegment::getInterval)
- .distinct().count()
- );
- snapshotBuilder.incrementSegmentCountCompacted(segmentsToCompact.size());
+ snapshotBuilder
+ .incrementBytesCompacted(
+ segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum()
+ )
+ .incrementIntervalCountCompacted(
+ segmentsToCompact.stream().map(DataSegment::getInterval).distinct().count()
+ )
+ .incrementSegmentCountCompacted(segmentsToCompact.size());
final DataSourceCompactionConfig config = compactionConfigs.get(dataSourceName);
@@ -519,20 +518,16 @@ private void addCompactionSnapshotStats(
final String dataSourceName = segmentsToCompact.get(0).getDataSource();
AutoCompactionSnapshot.Builder snapshotBuilder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent(
dataSourceName,
- k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING)
- );
- snapshotBuilder.incrementBytesAwaitingCompaction(
- segmentsToCompact.stream()
- .mapToLong(DataSegment::getSize)
- .sum()
- );
- snapshotBuilder.incrementIntervalCountAwaitingCompaction(
- segmentsToCompact.stream()
- .map(DataSegment::getInterval)
- .distinct()
- .count()
+ AutoCompactionSnapshot::builder
);
- snapshotBuilder.incrementSegmentCountAwaitingCompaction(segmentsToCompact.size());
+ snapshotBuilder
+ .incrementBytesAwaitingCompaction(
+ segmentsToCompact.stream().mapToLong(DataSegment::getSize).sum()
+ )
+ .incrementIntervalCountAwaitingCompaction(
+ segmentsToCompact.stream().map(DataSegment::getInterval).distinct().count()
+ )
+ .incrementSegmentCountAwaitingCompaction(segmentsToCompact.size());
}
}
@@ -543,7 +538,7 @@ private void addCompactionSnapshotStats(
final CompactionStatistics dataSourceCompactedStatistics = compactionStatisticsEntry.getValue();
AutoCompactionSnapshot.Builder builder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent(
dataSource,
- k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING)
+ AutoCompactionSnapshot::builder
);
builder.incrementBytesCompacted(dataSourceCompactedStatistics.getTotalBytes());
builder.incrementSegmentCountCompacted(dataSourceCompactedStatistics.getNumSegments());
@@ -557,11 +552,11 @@ private void addCompactionSnapshotStats(
final CompactionStatistics dataSourceSkippedStatistics = compactionStatisticsEntry.getValue();
AutoCompactionSnapshot.Builder builder = currentRunAutoCompactionSnapshotBuilders.computeIfAbsent(
dataSource,
- k -> new AutoCompactionSnapshot.Builder(k, AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING)
+ AutoCompactionSnapshot::builder
);
- builder.incrementBytesSkipped(dataSourceSkippedStatistics.getTotalBytes());
- builder.incrementSegmentCountSkipped(dataSourceSkippedStatistics.getNumSegments());
- builder.incrementIntervalCountSkipped(dataSourceSkippedStatistics.getNumIntervals());
+ builder.incrementBytesSkipped(dataSourceSkippedStatistics.getTotalBytes())
+ .incrementSegmentCountSkipped(dataSourceSkippedStatistics.getNumSegments())
+ .incrementIntervalCountSkipped(dataSourceSkippedStatistics.getNumIntervals());
}
final Map currentAutoCompactionSnapshotPerDataSource = new HashMap<>();
diff --git a/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java
index 9415a8188faa0..e034459fc7414 100644
--- a/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java
+++ b/server/src/test/java/org/apache/druid/server/coordinator/AutoCompactionSnapshotTest.java
@@ -28,25 +28,22 @@ public class AutoCompactionSnapshotTest
public void testAutoCompactionSnapshotBuilder()
{
final String expectedDataSource = "data";
- final AutoCompactionSnapshot.AutoCompactionScheduleStatus expectedStatus = AutoCompactionSnapshot.AutoCompactionScheduleStatus.RUNNING;
- AutoCompactionSnapshot.Builder builder = new AutoCompactionSnapshot.Builder(expectedDataSource, expectedStatus);
+ final AutoCompactionSnapshot.Builder builder = AutoCompactionSnapshot.builder(expectedDataSource);
// Increment every stats twice
for (int i = 0; i < 2; i++) {
- builder.incrementIntervalCountSkipped(13);
- builder.incrementBytesSkipped(13);
- builder.incrementSegmentCountSkipped(13);
-
- builder.incrementIntervalCountCompacted(13);
- builder.incrementBytesCompacted(13);
- builder.incrementSegmentCountCompacted(13);
-
- builder.incrementIntervalCountAwaitingCompaction(13);
- builder.incrementBytesAwaitingCompaction(13);
- builder.incrementSegmentCountAwaitingCompaction(13);
+ builder.incrementIntervalCountSkipped(13)
+ .incrementBytesSkipped(13)
+ .incrementSegmentCountSkipped(13)
+ .incrementIntervalCountCompacted(13)
+ .incrementBytesCompacted(13)
+ .incrementSegmentCountCompacted(13)
+ .incrementIntervalCountAwaitingCompaction(13)
+ .incrementBytesAwaitingCompaction(13)
+ .incrementSegmentCountAwaitingCompaction(13);
}
- AutoCompactionSnapshot actual = builder.build();
+ final AutoCompactionSnapshot actual = builder.build();
Assert.assertNotNull(actual);
Assert.assertEquals(26, actual.getSegmentCountSkipped());
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
index 691e951865983..5bcec39a1e843 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/DruidQuery.java
@@ -1602,7 +1602,8 @@ private ScanQuery toScanQuery(final boolean considerSorting)
if (outputRowSignature.size() == 0) {
// Should never do a scan query without any columns that we're interested in. This is probably a planner bug.
- throw new ISE("Cannot convert to Scan query without any columns.");
+ this.plannerContext.setPlanningError("Cannot convert to Scan query without any columns.");
+ return null;
}
final Pair dataSourceFiltrationPair = getFiltration(
diff --git a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java
index 20c672ce924b0..4f0f0eda21b96 100644
--- a/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java
+++ b/sql/src/main/java/org/apache/druid/sql/calcite/rel/Windowing.java
@@ -68,9 +68,11 @@
import javax.annotation.Nonnull;
import java.util.ArrayList;
import java.util.Collections;
+import java.util.Comparator;
import java.util.Iterator;
import java.util.LinkedHashSet;
import java.util.List;
+import java.util.Objects;
/**
* Maps onto a {@link org.apache.druid.query.operator.WindowOperatorQuery}.
@@ -123,45 +125,14 @@ public static Windowing fromCalciteStuff(
{
final Window window = Preconditions.checkNotNull(partialQuery.getWindow(), "window");
- ArrayList ops = new ArrayList<>();
-
+ final List windowGroupProcessors = new ArrayList<>();
final List windowOutputColumns = new ArrayList<>(sourceRowSignature.getColumnNames());
+
final String outputNamePrefix = Calcites.findUnusedPrefixForDigits("w", sourceRowSignature.getColumnNames());
int outputNameCounter = 0;
- // Track prior partition columns and sort columns group-to-group, so we only insert sorts and repartitions if
- // we really need to.
- List priorPartitionColumns = null;
- LinkedHashSet priorSortColumns = new LinkedHashSet<>();
-
- final RelCollation priorCollation = partialQuery.getScan().getTraitSet().getTrait(RelCollationTraitDef.INSTANCE);
- if (priorCollation != null) {
- // Populate initial priorSortColumns using collation of the input to the window operation. Allows us to skip
- // the initial sort operator if the rows were already in the desired order.
- priorSortColumns = computeSortColumnsFromRelCollation(priorCollation, sourceRowSignature);
- }
-
- for (int i = 0; i < window.groups.size(); ++i) {
- final WindowGroup group = new WindowGroup(window, window.groups.get(i), sourceRowSignature);
-
- final LinkedHashSet sortColumns = new LinkedHashSet<>();
- for (String partitionColumn : group.getPartitionColumns()) {
- sortColumns.add(ColumnWithDirection.ascending(partitionColumn));
- }
- sortColumns.addAll(group.getOrdering());
-
- // Add sorting and partitioning if needed.
- if (!sortMatches(priorSortColumns, sortColumns)) {
- // Sort order needs to change. Resort and repartition.
- ops.add(new NaiveSortOperatorFactory(new ArrayList<>(sortColumns)));
- ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns()));
- priorSortColumns = sortColumns;
- priorPartitionColumns = group.getPartitionColumns();
- } else if (!group.getPartitionColumns().equals(priorPartitionColumns)) {
- // Sort order doesn't need to change, but partitioning does. Only repartition.
- ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns()));
- priorPartitionColumns = group.getPartitionColumns();
- }
+ for (Window.Group windowGroup : window.groups) {
+ final WindowGroup group = new WindowGroup(window, windowGroup, sourceRowSignature);
// Add aggregations.
final List aggregateCalls = group.getAggregateCalls();
@@ -184,7 +155,8 @@ public static Windowing fromCalciteStuff(
InputAccessor.buildFor(
window,
partialQuery.getSelectProject(),
- sourceRowSignature),
+ sourceRowSignature
+ ),
Collections.emptyList(),
aggName,
aggregateCall,
@@ -232,12 +204,14 @@ public static Windowing fromCalciteStuff(
throw new ISE("No processors from Window[%s], why was this code called?", window);
}
- ops.add(new WindowOperatorFactory(
+ windowGroupProcessors.add(new WindowComputationProcessor(group, new WindowOperatorFactory(
processors.size() == 1 ?
processors.get(0) : new ComposingProcessor(processors.toArray(new Processor[0]))
- ));
+ )));
}
+ List ops = computeWindowOperations(partialQuery, sourceRowSignature, windowGroupProcessors);
+
// Apply windowProject, if present.
if (partialQuery.getWindowProject() != null) {
// We know windowProject is a mapping due to the isMapping() check in DruidRules.
@@ -270,6 +244,119 @@ public static Windowing fromCalciteStuff(
}
}
+ /**
+ * Computes the list of operators that are to be applied in an optimised order
+ */
+ private static List computeWindowOperations(
+ final PartialDruidQuery partialQuery,
+ final RowSignature sourceRowSignature,
+ List windowGroupProcessors
+ )
+ {
+ final List ops = new ArrayList<>();
+ // Track prior partition columns and sort columns group-to-group, so we only insert sorts and repartitions if
+ // we really need to.
+ List priorPartitionColumns = null;
+ LinkedHashSet priorSortColumns = new LinkedHashSet<>();
+
+ final RelCollation priorCollation = partialQuery.getScan().getTraitSet().getTrait(RelCollationTraitDef.INSTANCE);
+ if (priorCollation != null) {
+ // Populate initial priorSortColumns using collation of the input to the window operation. Allows us to skip
+ // the initial sort operator if the rows were already in the desired order.
+ priorSortColumns = computeSortColumnsFromRelCollation(priorCollation, sourceRowSignature);
+ }
+
+ // sort the processors to optimise the order of window operators
+ // currently we are moving the empty groups to the front
+ windowGroupProcessors.sort(WindowComputationProcessor.MOVE_EMPTY_GROUPS_FIRST);
+
+ for (WindowComputationProcessor windowComputationProcessor : windowGroupProcessors) {
+ final WindowGroup group = windowComputationProcessor.getGroup();
+ final LinkedHashSet sortColumns = new LinkedHashSet<>();
+ for (String partitionColumn : group.getPartitionColumns()) {
+ sortColumns.add(ColumnWithDirection.ascending(partitionColumn));
+ }
+ sortColumns.addAll(group.getOrdering());
+
+ // Add sorting and partitioning if needed.
+ if (!sortMatches(priorSortColumns, sortColumns)) {
+ // Sort order needs to change. Resort and repartition.
+ ops.add(new NaiveSortOperatorFactory(new ArrayList<>(sortColumns)));
+ ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns()));
+ priorSortColumns = sortColumns;
+ priorPartitionColumns = group.getPartitionColumns();
+ } else if (!group.getPartitionColumns().equals(priorPartitionColumns)) {
+ // Sort order doesn't need to change, but partitioning does. Only repartition.
+ ops.add(new NaivePartitioningOperatorFactory(group.getPartitionColumns()));
+ priorPartitionColumns = group.getPartitionColumns();
+ }
+
+ ops.add(windowComputationProcessor.getProcessorOperatorFactory());
+ }
+
+ return ops;
+ }
+
+ private static class WindowComputationProcessor
+ {
+ private final WindowGroup group;
+ private final OperatorFactory processorOperatorFactory;
+
+ public WindowComputationProcessor(WindowGroup group, OperatorFactory processorOperatorFactory)
+ {
+ this.group = group;
+ this.processorOperatorFactory = processorOperatorFactory;
+ }
+
+ public WindowGroup getGroup()
+ {
+ return group;
+ }
+
+ public OperatorFactory getProcessorOperatorFactory()
+ {
+ return processorOperatorFactory;
+ }
+
+ /**
+ * Comparator to move the empty windows to the front
+ */
+ public static final Comparator MOVE_EMPTY_GROUPS_FIRST = (o1, o2) -> {
+ if (o1.getGroup().getPartitionColumns().isEmpty() && o2.getGroup().getPartitionColumns().isEmpty()) {
+ return 0;
+ }
+ if (o1.getGroup().getPartitionColumns().isEmpty()) {
+ return -1;
+ }
+ if (o2.getGroup().getPartitionColumns().isEmpty()) {
+ return 1;
+ }
+ return 0;
+ };
+
+ @Override
+ public boolean equals(Object o)
+ {
+ if (this == o) {
+ return true;
+ }
+ if (o == null || getClass() != o.getClass()) {
+ return false;
+ }
+ WindowComputationProcessor obj = (WindowComputationProcessor) o;
+ return Objects.equals(group, obj.group) && Objects.equals(
+ processorOperatorFactory,
+ obj.processorOperatorFactory
+ );
+ }
+
+ @Override
+ public int hashCode()
+ {
+ return Objects.hash(group, processorOperatorFactory);
+ }
+ }
+
private final RowSignature signature;
public Windowing(
diff --git a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
index 16706335515bb..f5e161b3a6aa4 100644
--- a/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
+++ b/sql/src/test/java/org/apache/druid/sql/calcite/CalciteWindowQueryTest.java
@@ -233,23 +233,49 @@ public void windowQueryTestWithCustomContextMaxSubqueryBytes(String filename) th
}
}
+ @Test
+ public void testEmptyWindowInSubquery()
+ {
+ testBuilder()
+ .sql(
+ "select c from (\n"
+ + " select channel, row_number() over () as c\n"
+ + " from wikipedia\n"
+ + " group by channel\n"
+ + ") LIMIT 5"
+ )
+ .queryContext(ImmutableMap.of(
+ PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
+ QueryContexts.ENABLE_DEBUG, true,
+ QueryContexts.WINDOWING_STRICT_VALIDATION, false
+ ))
+ .expectedResults(ImmutableList.of(
+ new Object[]{1L},
+ new Object[]{2L},
+ new Object[]{3L},
+ new Object[]{4L},
+ new Object[]{5L}
+ ))
+ .run();
+ }
+
@Test
public void testWindow()
{
testBuilder()
- .sql("SELECT\n" +
- "(rank() over (order by count(*) desc)),\n" +
- "(rank() over (order by count(*) desc))\n" +
- "FROM \"wikipedia\"")
- .queryContext(ImmutableMap.of(
- PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
- QueryContexts.ENABLE_DEBUG, true,
- QueryContexts.WINDOWING_STRICT_VALIDATION, false
- ))
- .expectedResults(ImmutableList.of(
- new Object[]{1L, 1L}
- ))
- .run();
+ .sql("SELECT\n" +
+ "(rank() over (order by count(*) desc)),\n" +
+ "(rank() over (order by count(*) desc))\n" +
+ "FROM \"wikipedia\"")
+ .queryContext(ImmutableMap.of(
+ PlannerContext.CTX_ENABLE_WINDOW_FNS, true,
+ QueryContexts.ENABLE_DEBUG, true,
+ QueryContexts.WINDOWING_STRICT_VALIDATION, false
+ ))
+ .expectedResults(ImmutableList.of(
+ new Object[]{1L, 1L}
+ ))
+ .run();
}
private WindowOperatorQuery getWindowOperatorQuery(List> queries)
diff --git a/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest b/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest
new file mode 100644
index 0000000000000..0c9d88b5041fd
--- /dev/null
+++ b/sql/src/test/resources/calcite/tests/window/WindowOpReorder.sqlTest
@@ -0,0 +1,51 @@
+type: "operatorValidation"
+
+sql: |
+ SELECT
+ m1,
+ m2,
+ SUM(m1) OVER(PARTITION BY m2) as sum1,
+ SUM(m2) OVER() as sum2
+ from druid.numfoo
+ GROUP BY m1,m2
+
+expectedOperators:
+ - type: "naivePartition"
+ partitionColumns: [ ]
+ - type: "window"
+ processor:
+ type: "framedAgg"
+ frame:
+ peerType: "ROWS"
+ lowUnbounded: true
+ lowOffset: 0
+ uppUnbounded: true
+ uppOffset: 0
+ orderBy: null
+ aggregations:
+ - { "type": "doubleSum", "name": "w1", "fieldName": "_d1" }
+ - type: "naiveSort"
+ columns:
+ - column: "_d1"
+ direction: "ASC"
+ - type: "naivePartition"
+ partitionColumns: [ "_d1" ]
+ - type: "window"
+ processor:
+ type: "framedAgg"
+ frame:
+ peerType: "ROWS"
+ lowUnbounded: true
+ lowOffset: 0
+ uppUnbounded: true
+ uppOffset: 0
+ orderBy: null
+ aggregations:
+ - { "type": "doubleSum", "name": "w0", "fieldName": "_d0" }
+expectedResults:
+ - [1.0, 1.0, 1.0, 21.0]
+ - [2.0, 2.0, 2.0, 21.0]
+ - [3.0, 3.0, 3.0, 21.0]
+ - [4.0, 4.0, 4.0, 21.0]
+ - [5.0, 5.0, 5.0, 21.0]
+ - [6.0, 6.0, 6.0, 21.0]
\ No newline at end of file
diff --git a/website/.spelling b/website/.spelling
index a14f233539c39..20be2b183b3bb 100644
--- a/website/.spelling
+++ b/website/.spelling
@@ -277,6 +277,7 @@ backpressure
base64
big-endian
bigint
+blkio
blobstore
Boolean
boolean
@@ -531,6 +532,7 @@ subqueries
subquery
subsecond
substring
+substrings
subtask
subtasks
supervisorTaskId
diff --git a/website/redirects.js b/website/redirects.js
index bcbe53cb51e35..e2bb4da7b68c0 100644
--- a/website/redirects.js
+++ b/website/redirects.js
@@ -148,8 +148,11 @@ const Redirects=[
"to": "/docs/latest/querying/scan-query"
},
{
- "from": "/docs/latest/development/extensions-core/namespaced-lookup.html",
- "to": "/docs/latest/development/extensions-core/lookups-cached-global"
+ "from": [
+ "/docs/latest/development/extensions-core/namespaced-lookup.html",
+ "/docs/latest/development/extensions-core/lookups-cached-global"
+ ],
+ "to": "/docs/latest/querying/lookups-cached-global"
},
{
"from": "/docs/latest/development/indexer.html",
diff --git a/website/sidebars.json b/website/sidebars.json
index 45631da72a619..1f2ea873fcbb4 100644
--- a/website/sidebars.json
+++ b/website/sidebars.json
@@ -185,7 +185,16 @@
"items": [
"querying/datasource",
"querying/joins",
- "querying/lookups",
+ {"type": "category",
+ "label": "Lookups",
+ "link": {
+ "type": "doc",
+ "id": "querying/lookups"
+ },
+ "items":[
+ "querying/lookups-cached-global",
+ "querying/kafka-extraction-namespace"
+ ]},
"querying/multi-value-dimensions",
"querying/arrays",
"querying/nested-columns",
@@ -354,8 +363,20 @@
"items":
[
"release-info/release-notes",
- "release-info/upgrade-notes"
+ "release-info/upgrade-notes",
+ {
+ "type": "category",
+ "label": "Migration guides",
+ "link": {
+ "type": "doc",
+ "id": "release-info/migration-guide"
+ },
+ "items": [
+ "release-info/migr-front-coded-dict",
+ "release-info/migr-subquery-limit"
+ ]
+ }
]},
"misc/papers-and-talks"
]
-}
\ No newline at end of file
+}