Removes FiniteFirehoseFactory and its implementations (#12852)

The FiniteFirehoseFactory and InputRowParser classes were deprecated in 0.17.0 (#8823) in favor of InputSource & InputFormat. This PR removes the FiniteFirehoseFactory and all its implementations along with classes solely used by them like Fetcher (Used by PrefetchableTextFilesFirehoseFactory). Refactors classes including tests using FiniteFirehoseFactory to use InputSource instead.
Removing InputRowParser may not be as trivial as many classes that aren't deprecated depends on it (with no alternatives), like EventReceiverFirehoseFactory. Hence FirehoseFactory, EventReceiverFirehoseFactory, and Firehose are marked deprecated.
This commit is contained in:
Tejaswini Bandlamudi 2023-03-02 18:07:17 +05:30 committed by GitHub
parent 1aae37f7d6
commit 7103cb4b9d
No known key found for this signature in database
GPG Key ID: 4AEE18F83AFDEB23
150 changed files with 1557 additions and 11313 deletions

View File

@ -95,7 +95,7 @@ jobs:
source ./integration-tests-ex/image/target/env.sh
docker tag $DRUID_IT_IMAGE_NAME $DRUID_IT_IMAGE_NAME-jdk${{ matrix.jdk }}
echo $DRUID_IT_IMAGE_NAME
docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ matrix.jdk }}.tar.gz
docker save "$DRUID_IT_IMAGE_NAME" | gzip > druid-container-jdk${{ inputs.build_jdk }}.tar.gz
- name: Load docker image
run: |

View File

@ -576,7 +576,7 @@ This deep storage is used to interface with Cassandra. Note that the `druid-cas
#### HDFS input source
You can set the following property to specify permissible protocols for
the [HDFS input source](../ingestion/native-batch-input-source.md#hdfs-input-source) and the [HDFS firehose](../ingestion/native-batch-firehose.md#hdfsfirehose).
the [HDFS input source](../ingestion/native-batch-input-source.md#hdfs-input-source).
|Property|Possible Values|Description|Default|
|--------|---------------|-----------|-------|
@ -586,7 +586,7 @@ the [HDFS input source](../ingestion/native-batch-input-source.md#hdfs-input-sou
#### HTTP input source
You can set the following property to specify permissible protocols for
the [HTTP input source](../ingestion/native-batch-input-source.md#http-input-source) and the [HTTP firehose](../ingestion/native-batch-firehose.md#httpfirehose).
the [HTTP input source](../ingestion/native-batch-input-source.md#http-input-source).
|Property|Possible Values|Description|Default|
|--------|---------------|-----------|-------|
@ -599,7 +599,6 @@ the [HTTP input source](../ingestion/native-batch-input-source.md#http-input-sou
You can use the following properties to specify permissible JDBC options for:
- [SQL input source](../ingestion/native-batch-input-source.md#sql-input-source)
- [SQL firehose](../ingestion/native-batch-firehose.md#sqlfirehose),
- [globally cached JDBC lookups](../development/extensions-core/lookups-cached-global.md#jdbc-lookup)
- [JDBC Data Fetcher for per-lookup caching](../development/extensions-core/druid-lookups.md#data-fetcher-layer).

View File

@ -40,8 +40,6 @@ The [Google Cloud Storage input source](../../ingestion/native-batch-input-sourc
to read objects directly from Google Cloud Storage. If you use the [Hadoop task](../../ingestion/hadoop.md),
you can read data from Google Cloud Storage by specifying the paths in your [`inputSpec`](../../ingestion/hadoop.md#inputspec).
Objects can also be read directly from Google Cloud Storage via the [StaticGoogleBlobStoreFirehose](../../ingestion/native-batch-firehose.md#staticgoogleblobstorefirehose)
### Deep Storage
Deep storage can be written to Google Cloud Storage either via this extension or the [druid-hdfs-storage extension](../extensions-core/hdfs.md).

View File

@ -125,7 +125,7 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC
|`druid.metadata.mysql.ssl.enabledSSLCipherSuites`|Overrides the existing cipher suites with these cipher suites.|none|no|
|`druid.metadata.mysql.ssl.enabledTLSProtocols`|Overrides the TLS protocols with these protocols.|none|no|
### MySQL InputSource and Firehose
### MySQL InputSource
```json
{
@ -133,22 +133,17 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC
"spec": {
"dataSchema": {
"dataSource": "some_datasource",
"parser": {
"parseSpec": {
"format": "timeAndDims",
"dimensionsSpec": {
"dimensionExclusions": [],
"dimensions": [
"dim1",
"dim2",
"dim3"
]
},
"timestampSpec": {
"format": "auto",
"column": "ts"
}
}
"dimensionsSpec": {
"dimensionExclusions": [],
"dimensions": [
"dim1",
"dim2",
"dim3"
]
},
"timestampSpec": {
"format": "auto",
"column": "ts"
},
"metricsSpec": [],
"granularitySpec": {
@ -167,7 +162,7 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"inputSource": {
"type": "sql",
"database": {
"type": "mysql",
@ -180,6 +175,9 @@ If using the MariaDB connector library, set `druid.metadata.mysql.driver.driverC
"sqls": [
"SELECT * FROM some_table"
]
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -87,7 +87,7 @@ In most cases, the configuration options map directly to the [postgres JDBC conn
### PostgreSQL Firehose
The PostgreSQL extension provides an implementation of an [SqlFirehose](../../ingestion/native-batch-firehose.md) which can be used to ingest data into Druid from a PostgreSQL database.
The PostgreSQL extension provides an implementation of an [SQL input source](../../ingestion/native-batch-input-source.md) which can be used to ingest data into Druid from a PostgreSQL database.
```json
{
@ -95,22 +95,17 @@ The PostgreSQL extension provides an implementation of an [SqlFirehose](../../in
"spec": {
"dataSchema": {
"dataSource": "some_datasource",
"parser": {
"parseSpec": {
"format": "timeAndDims",
"dimensionsSpec": {
"dimensionExclusions": [],
"dimensions": [
"dim1",
"dim2",
"dim3"
]
},
"timestampSpec": {
"format": "auto",
"column": "ts"
}
}
"dimensionsSpec": {
"dimensionExclusions": [],
"dimensions": [
"dim1",
"dim2",
"dim3"
]
},
"timestampSpec": {
"format": "auto",
"column": "ts"
},
"metricsSpec": [],
"granularitySpec": {
@ -129,7 +124,7 @@ The PostgreSQL extension provides an implementation of an [SqlFirehose](../../in
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"inputSource": {
"type": "sql",
"database": {
"type": "postgresql",
@ -142,6 +137,9 @@ The PostgreSQL extension provides an implementation of an [SqlFirehose](../../in
"sqls": [
"SELECT * FROM some_table"
]
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -263,7 +263,7 @@ The size-based split hint spec affects all splittable input sources except for t
#### Segments Split Hint Spec
The segments split hint spec is used only for [`DruidInputSource`](./native-batch-input-source.md) and legacy `IngestSegmentFirehose`.
The segments split hint spec is used only for [`DruidInputSource`](./native-batch-input-source.md).
|property|description|default|required?|
|--------|-----------|-------|---------|
@ -588,7 +588,7 @@ An example of the result is
"filter": "lineitem.tbl.5"
},
"inputFormat": {
"format": "tsv",
"type": "tsv",
"delimiter": "|",
"columns": [
"l_orderkey",

View File

@ -1,46 +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.firehose.aliyun;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.initialization.DruidModule;
import java.util.List;
public class OssFirehoseDruidModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new SimpleModule().registerSubtypes(new NamedType(StaticOssFirehoseFactory.class, "static-aliyun-oss"))
);
}
@Override
public void configure(Binder binder)
{
}
}

View File

@ -1,243 +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.firehose.aliyun;
import com.aliyun.oss.OSS;
import com.aliyun.oss.OSSException;
import com.aliyun.oss.model.GetObjectRequest;
import com.aliyun.oss.model.OSSObject;
import com.aliyun.oss.model.OSSObjectSummary;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.storage.aliyun.OssStorageDruidModule;
import org.apache.druid.storage.aliyun.OssUtils;
import org.apache.druid.utils.CompressionUtils;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
/**
* Builds firehoses that read from a predefined list of aliyun OSS objects and then dry up.
*/
public class StaticOssFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
{
private static final Logger log = new Logger(StaticOssFirehoseFactory.class);
private final OSS client;
private final List<URI> uris;
private final List<URI> prefixes;
@JsonCreator
public StaticOssFirehoseFactory(
@JacksonInject OSS client,
@JsonProperty("uris") List<URI> uris,
@JsonProperty("prefixes") List<URI> prefixes,
@JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
@JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
@JsonProperty("fetchTimeout") Long fetchTimeout,
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
)
{
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
this.client = Preconditions.checkNotNull(client, "client");
this.uris = uris == null ? new ArrayList<>() : uris;
this.prefixes = prefixes == null ? new ArrayList<>() : prefixes;
if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) {
throw new IAE("uris and prefixes cannot be used together");
}
if (this.uris.isEmpty() && this.prefixes.isEmpty()) {
throw new IAE("uris or prefixes must be specified");
}
for (final URI inputURI : this.uris) {
Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()),
"input uri scheme == %s (%s)",
OssStorageDruidModule.SCHEME,
inputURI);
}
for (final URI inputURI : this.prefixes) {
Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()),
"input uri scheme == %s (%s)",
OssStorageDruidModule.SCHEME,
inputURI);
}
}
@JsonProperty
public List<URI> getUris()
{
return uris;
}
@JsonProperty("prefixes")
public List<URI> getPrefixes()
{
return prefixes;
}
@Override
protected Collection<URI> initObjects()
{
if (!uris.isEmpty()) {
return uris;
} else {
final List<OSSObjectSummary> objects = new ArrayList<>();
for (final URI prefix : prefixes) {
final Iterator<OSSObjectSummary> objectSummaryIterator = OssUtils.objectSummaryIterator(
client,
Collections.singletonList(prefix),
OssUtils.MAX_LISTING_LENGTH
);
objectSummaryIterator.forEachRemaining(objects::add);
}
return objects.stream().map(OssUtils::summaryToUri).collect(Collectors.toList());
}
}
@Override
protected InputStream openObjectStream(URI object) throws IOException
{
try {
// Get data of the given object and open an input stream
final String bucket = object.getAuthority();
final String key = OssUtils.extractKey(object);
final OSSObject ossObject = client.getObject(bucket, key);
if (ossObject == null) {
throw new ISE("Failed to get an Aliyun OSS object for bucket[%s] and key[%s]", bucket, key);
}
return ossObject.getObjectContent();
}
catch (OSSException e) {
throw new IOException(e);
}
}
@Override
protected InputStream openObjectStream(URI object, long start) throws IOException
{
final String bucket = object.getAuthority();
final String key = OssUtils.extractKey(object);
final GetObjectRequest request = new GetObjectRequest(bucket, key);
try {
final OSSObject ossObject = client.getObject(request);
if (ossObject == null) {
throw new ISE(
"Failed to get an Aliyun OSS object for bucket[%s], key[%s], and start[%d]",
bucket,
key,
start
);
}
InputStream is = ossObject.getObjectContent();
is.skip(start);
return is;
}
catch (OSSException e) {
throw new IOException(e);
}
}
@Override
protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
{
return CompressionUtils.decompress(stream, OssUtils.extractKey(object));
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
StaticOssFirehoseFactory that = (StaticOssFirehoseFactory) o;
return Objects.equals(uris, that.uris) &&
Objects.equals(prefixes, that.prefixes) &&
getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
getFetchTimeout() == that.getFetchTimeout() &&
getMaxFetchRetry() == that.getMaxFetchRetry();
}
@Override
public int hashCode()
{
return Objects.hash(
uris,
prefixes,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
@Override
protected Predicate<Throwable> getRetryCondition()
{
return OssUtils.RETRYABLE;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, URI> withSplit(InputSplit<URI> split)
{
return new StaticOssFirehoseFactory(
client,
Collections.singletonList(split.get()),
null,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -14,5 +14,4 @@
# limitations under the License.
org.apache.druid.storage.aliyun.OssStorageDruidModule
org.apache.druid.firehose.aliyun.OssFirehoseDruidModule
org.apache.druid.data.input.aliyun.OssInputSourceDruidModule

View File

@ -143,10 +143,6 @@
<artifactId>rackspace-cloudfiles-uk</artifactId>
<version>${jclouds.version}</version>
</dependency>
<dependency>
<groupId>com.fasterxml.jackson.module</groupId>
<artifactId>jackson-module-guice</artifactId>
</dependency>
<!-- Tests -->
<dependency>

View File

@ -1,101 +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.firehose.cloudfiles;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import javax.validation.constraints.NotNull;
import java.util.Objects;
public class CloudFilesBlob
{
@JsonProperty
@NotNull
private String container;
@JsonProperty
@NotNull
private String path;
@JsonProperty
@NotNull
private String region;
@JsonCreator
public CloudFilesBlob(
@JsonProperty("container") String container,
@JsonProperty("path") String path,
@JsonProperty("region") String region
)
{
this.container = container;
this.path = path;
this.region = region;
}
public String getContainer()
{
return container;
}
public String getPath()
{
return path;
}
public String getRegion()
{
return region;
}
@Override
public String toString()
{
return "CloudFilesBlob{"
+ "container=" + container
+ ",path=" + path
+ ",region=" + region
+ "}";
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final CloudFilesBlob that = (CloudFilesBlob) o;
return Objects.equals(container, that.container) &&
Objects.equals(path, that.path) &&
Objects.equals(region, that.region);
}
@Override
public int hashCode()
{
return Objects.hash(container, path, region);
}
}

View File

@ -1,48 +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.firehose.cloudfiles;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.initialization.DruidModule;
import java.util.List;
public class CloudFilesFirehoseDruidModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new SimpleModule().registerSubtypes(
new NamedType(StaticCloudFilesFirehoseFactory.class, "staticcloudfiles")));
}
@Override
public void configure(Binder arg0)
{
}
}

View File

@ -1,162 +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.firehose.cloudfiles;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Predicate;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.storage.cloudfiles.CloudFilesByteSource;
import org.apache.druid.storage.cloudfiles.CloudFilesObjectApiProxy;
import org.apache.druid.storage.cloudfiles.CloudFilesUtils;
import org.apache.druid.utils.CompressionUtils;
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
public class StaticCloudFilesFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<CloudFilesBlob>
{
private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class);
private final CloudFilesApi cloudFilesApi;
private final List<CloudFilesBlob> blobs;
@JsonCreator
public StaticCloudFilesFirehoseFactory(
@JacksonInject CloudFilesApi cloudFilesApi,
@JsonProperty("blobs") List<CloudFilesBlob> blobs,
@JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
@JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
@JsonProperty("fetchTimeout") Long fetchTimeout,
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
)
{
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
this.cloudFilesApi = cloudFilesApi;
this.blobs = blobs;
}
@JsonProperty
public List<CloudFilesBlob> getBlobs()
{
return blobs;
}
@Override
protected Collection<CloudFilesBlob> initObjects()
{
return blobs;
}
@Override
protected InputStream openObjectStream(CloudFilesBlob object) throws IOException
{
return openObjectStream(object, 0);
}
@Override
protected InputStream openObjectStream(CloudFilesBlob object, long start) throws IOException
{
return createCloudFilesByteSource(object).openStream(start);
}
private CloudFilesByteSource createCloudFilesByteSource(CloudFilesBlob object)
{
final String region = object.getRegion();
final String container = object.getContainer();
final String path = object.getPath();
log.info("Retrieving file from region[%s], container[%s] and path [%s]",
region, container, path
);
CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
return new CloudFilesByteSource(objectApi, path);
}
@Override
protected InputStream wrapObjectStream(CloudFilesBlob object, InputStream stream) throws IOException
{
return CompressionUtils.decompress(stream, object.getPath());
}
@Override
public boolean equals(Object o)
{
if (o == this) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final StaticCloudFilesFirehoseFactory that = (StaticCloudFilesFirehoseFactory) o;
return Objects.equals(blobs, that.blobs) &&
getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
getFetchTimeout() == that.getFetchTimeout() &&
getMaxFetchRetry() == that.getMaxFetchRetry();
}
@Override
public int hashCode()
{
return Objects.hash(
blobs,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
@Override
protected Predicate<Throwable> getRetryCondition()
{
return CloudFilesUtils.CLOUDFILESRETRY;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, CloudFilesBlob> withSplit(InputSplit<CloudFilesBlob> split)
{
return new StaticCloudFilesFirehoseFactory(
cloudFilesApi,
Collections.singletonList(split.get()),
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -16,4 +16,3 @@
# under the License.
org.apache.druid.storage.cloudfiles.CloudFilesStorageDruidModule
org.apache.druid.firehose.cloudfiles.CloudFilesFirehoseDruidModule

View File

@ -1,105 +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.firehose.cloudfiles;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.module.guice.ObjectMapperModule;
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.Provides;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.easymock.EasyMock;
import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.List;
public class StaticCloudFilesFirehoseFactoryTest
{
private static final CloudFilesApi API = EasyMock.niceMock(CloudFilesApi.class);
@Test
public void testSerde() throws IOException
{
final ObjectMapper mapper = createObjectMapper(new TestModule());
final List<CloudFilesBlob> blobs = ImmutableList.of(
new CloudFilesBlob("container", "foo", "bar"),
new CloudFilesBlob("container", "foo", "bar2")
);
final StaticCloudFilesFirehoseFactory factory = new StaticCloudFilesFirehoseFactory(
API,
blobs,
2048L,
1024L,
512L,
100L,
5
);
final StaticCloudFilesFirehoseFactory outputFact = mapper.readValue(
mapper.writeValueAsString(factory),
StaticCloudFilesFirehoseFactory.class
);
Assert.assertEquals(factory, outputFact);
}
private static ObjectMapper createObjectMapper(DruidModule baseModule)
{
final ObjectMapper baseMapper = new DefaultObjectMapper();
baseModule.getJacksonModules().forEach(baseMapper::registerModule);
final Injector injector = Guice.createInjector(
new ObjectMapperModule(),
baseModule
);
return injector.getInstance(ObjectMapper.class);
}
private static class TestModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(new SimpleModule());
}
@Override
public void configure(Binder binder)
{
}
@Provides
public CloudFilesApi getRestS3Service()
{
return API;
}
}
}

View File

@ -1,93 +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.firehose.azure;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import javax.validation.constraints.NotNull;
import java.util.Objects;
/**
* Represents an Azure based blob. Used with {@link StaticAzureBlobStoreFirehoseFactory}.
*
* @deprecated as of version 0.18.0 because support for firehose has been discontinued. Please use
* {@link org.apache.druid.data.input.azure.AzureEntity} with {@link org.apache.druid.data.input.azure.AzureInputSource}
* instead.
*/
@Deprecated
public class AzureBlob
{
@JsonProperty
@NotNull
private String container;
@JsonProperty
@NotNull
private String path;
@JsonCreator
public AzureBlob(@JsonProperty("container") String container, @JsonProperty("path") String path)
{
this.container = container;
this.path = path;
}
public String getContainer()
{
return container;
}
public String getPath()
{
return path;
}
@Override
public String toString()
{
return "AzureBlob{"
+ "container=" + container
+ ",path=" + path
+ "}";
}
@Override
public boolean equals(Object o)
{
if (o == this) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final AzureBlob that = (AzureBlob) o;
return Objects.equals(container, that.container) &&
Objects.equals(path, that.path);
}
@Override
public int hashCode()
{
return Objects.hash(container, path);
}
}

View File

@ -1,167 +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.firehose.azure;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.storage.azure.AzureByteSource;
import org.apache.druid.storage.azure.AzureStorage;
import org.apache.druid.storage.azure.AzureUtils;
import org.apache.druid.utils.CompressionUtils;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
/**
* This class is heavily inspired by the StaticS3FirehoseFactory class in the org.apache.druid.firehose.s3 package
*
* @deprecated as of version 0.18.0 because support for firehose has been discontinued. Please use
* {@link org.apache.druid.data.input.azure.AzureInputSource} instead.
*/
@Deprecated
public class StaticAzureBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<AzureBlob>
{
private final AzureStorage azureStorage;
private final List<AzureBlob> blobs;
@JsonCreator
public StaticAzureBlobStoreFirehoseFactory(
@JacksonInject AzureStorage azureStorage,
@JsonProperty("blobs") List<AzureBlob> blobs,
@JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
@JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
@JsonProperty("fetchTimeout") Long fetchTimeout,
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
)
{
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
this.blobs = blobs;
this.azureStorage = azureStorage;
}
@JsonProperty
public List<AzureBlob> getBlobs()
{
return blobs;
}
@Override
protected Collection<AzureBlob> initObjects()
{
return blobs;
}
@Override
protected InputStream openObjectStream(AzureBlob object) throws IOException
{
return makeByteSource(azureStorage, object).openStream();
}
@Override
protected InputStream openObjectStream(AzureBlob object, long start) throws IOException
{
// BlobInputStream.skip() moves the next read offset instead of skipping first 'start' bytes.
final InputStream in = openObjectStream(object);
final long skip = in.skip(start);
Preconditions.checkState(skip == start, "start offset was [%s] but [%s] bytes were skipped", start, skip);
return in;
}
@Override
protected InputStream wrapObjectStream(AzureBlob object, InputStream stream) throws IOException
{
return CompressionUtils.decompress(stream, object.getPath());
}
private static AzureByteSource makeByteSource(AzureStorage azureStorage, AzureBlob object)
{
final String container = object.getContainer();
final String path = StringUtils.maybeRemoveLeadingSlash(object.getPath());
return new AzureByteSource(azureStorage, container, path);
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final StaticAzureBlobStoreFirehoseFactory that = (StaticAzureBlobStoreFirehoseFactory) o;
return Objects.equals(blobs, that.blobs) &&
getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
getFetchTimeout() == that.getFetchTimeout() &&
getMaxFetchRetry() == that.getMaxFetchRetry();
}
@Override
public int hashCode()
{
return Objects.hash(
blobs,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
@Override
protected Predicate<Throwable> getRetryCondition()
{
return AzureUtils.AZURE_RETRY;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, AzureBlob> withSplit(InputSplit<AzureBlob> split)
{
return new StaticAzureBlobStoreFirehoseFactory(
azureStorage,
Collections.singletonList(split.get()),
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -33,7 +33,6 @@ import com.microsoft.azure.storage.CloudStorageAccount;
import com.microsoft.azure.storage.blob.CloudBlobClient;
import org.apache.druid.data.input.azure.AzureEntityFactory;
import org.apache.druid.data.input.azure.AzureInputSource;
import org.apache.druid.firehose.azure.StaticAzureBlobStoreFirehoseFactory;
import org.apache.druid.guice.Binders;
import org.apache.druid.guice.JsonConfigProvider;
import org.apache.druid.guice.LazySingleton;
@ -84,7 +83,6 @@ public class AzureStorageDruidModule implements DruidModule
}
},
new SimpleModule().registerSubtypes(
new NamedType(StaticAzureBlobStoreFirehoseFactory.class, "static-azure-blobstore"),
new NamedType(AzureInputSource.class, SCHEME)
)
);

View File

@ -1,104 +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.firehose.azure;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.fasterxml.jackson.module.guice.ObjectMapperModule;
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.Provides;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.storage.azure.AzureStorage;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.List;
public class StaticAzureBlobStoreFirehoseFactoryTest
{
private static final AzureStorage STORAGE = new AzureStorage(null);
@Test
public void testSerde() throws IOException
{
final ObjectMapper mapper = createObjectMapper(new TestModule());
final List<AzureBlob> blobs = ImmutableList.of(
new AzureBlob("foo", "bar"),
new AzureBlob("foo", "bar2")
);
final StaticAzureBlobStoreFirehoseFactory factory = new StaticAzureBlobStoreFirehoseFactory(
STORAGE,
blobs,
2048L,
1024L,
512L,
100L,
5
);
final StaticAzureBlobStoreFirehoseFactory outputFact = mapper.readValue(
mapper.writeValueAsString(factory),
StaticAzureBlobStoreFirehoseFactory.class
);
Assert.assertEquals(factory, outputFact);
}
private static ObjectMapper createObjectMapper(DruidModule baseModule)
{
final ObjectMapper baseMapper = new DefaultObjectMapper();
baseModule.getJacksonModules().forEach(baseMapper::registerModule);
final Injector injector = Guice.createInjector(
new ObjectMapperModule(),
baseModule
);
return injector.getInstance(ObjectMapper.class);
}
private static class TestModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(new SimpleModule());
}
@Override
public void configure(Binder binder)
{
}
@Provides
public AzureStorage getRestS3Service()
{
return STORAGE;
}
}
}

View File

@ -1,81 +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.firehose.google;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import java.util.Objects;
public class GoogleBlob
{
private final String bucket;
private final String path;
@JsonCreator
public GoogleBlob(@JsonProperty("bucket") String bucket, @JsonProperty("path") String path)
{
this.bucket = bucket;
this.path = path;
}
@JsonProperty
public String getBucket()
{
return bucket;
}
@JsonProperty
public String getPath()
{
return path;
}
@Override
public String toString()
{
return "GoogleBlob {"
+ "bucket=" + bucket
+ ",path=" + path
+ "}";
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final GoogleBlob that = (GoogleBlob) o;
return Objects.equals(bucket, that.bucket) &&
Objects.equals(path, that.path);
}
@Override
public int hashCode()
{
return Objects.hash(bucket, path);
}
}

View File

@ -1,155 +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.firehose.google;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Predicate;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.storage.google.GoogleByteSource;
import org.apache.druid.storage.google.GoogleStorage;
import org.apache.druid.storage.google.GoogleUtils;
import org.apache.druid.utils.CompressionUtils;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.Collections;
import java.util.List;
import java.util.Objects;
public class StaticGoogleBlobStoreFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<GoogleBlob>
{
private final GoogleStorage storage;
private final List<GoogleBlob> blobs;
@JsonCreator
public StaticGoogleBlobStoreFirehoseFactory(
@JacksonInject GoogleStorage storage,
@JsonProperty("blobs") List<GoogleBlob> blobs,
@JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
@JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
@JsonProperty("fetchTimeout") Long fetchTimeout,
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
)
{
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
this.storage = storage;
this.blobs = blobs;
}
@JsonProperty
public List<GoogleBlob> getBlobs()
{
return blobs;
}
@Override
protected Collection<GoogleBlob> initObjects()
{
return blobs;
}
@Override
protected InputStream openObjectStream(GoogleBlob object) throws IOException
{
return openObjectStream(object, 0);
}
@Override
protected InputStream openObjectStream(GoogleBlob object, long start) throws IOException
{
return createGoogleByteSource(object).openStream(start);
}
private GoogleByteSource createGoogleByteSource(GoogleBlob object)
{
final String bucket = object.getBucket();
final String path = StringUtils.maybeRemoveLeadingSlash(object.getPath());
return new GoogleByteSource(storage, bucket, path);
}
@Override
protected InputStream wrapObjectStream(GoogleBlob object, InputStream stream) throws IOException
{
return CompressionUtils.decompress(stream, object.getPath());
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
final StaticGoogleBlobStoreFirehoseFactory that = (StaticGoogleBlobStoreFirehoseFactory) o;
return Objects.equals(blobs, that.blobs) &&
getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
getFetchTimeout() == that.getFetchTimeout() &&
getMaxFetchRetry() == that.getMaxFetchRetry();
}
@Override
public int hashCode()
{
return Objects.hash(
blobs,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
@Override
protected Predicate<Throwable> getRetryCondition()
{
return GoogleUtils::isRetryable;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, GoogleBlob> withSplit(InputSplit<GoogleBlob> split)
{
return new StaticGoogleBlobStoreFirehoseFactory(
storage,
Collections.singletonList(split.get()),
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -34,7 +34,6 @@ import com.google.inject.Provides;
import com.google.inject.multibindings.MapBinder;
import org.apache.druid.data.SearchableVersionedDataFinder;
import org.apache.druid.data.input.google.GoogleCloudStorageInputSource;
import org.apache.druid.firehose.google.StaticGoogleBlobStoreFirehoseFactory;
import org.apache.druid.guice.Binders;
import org.apache.druid.guice.JsonConfigProvider;
import org.apache.druid.guice.LazySingleton;
@ -77,7 +76,6 @@ public class GoogleStorageDruidModule implements DruidModule
}
},
new SimpleModule().registerSubtypes(
new NamedType(StaticGoogleBlobStoreFirehoseFactory.class, "static-google-blobstore"),
new NamedType(GoogleCloudStorageInputSource.class, SCHEME)
)
);

View File

@ -1,63 +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.firehose.google;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import org.apache.druid.data.input.google.GoogleCloudStorageInputSourceTest;
import org.apache.druid.storage.google.GoogleStorage;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.List;
public class StaticGoogleBlobStoreFirehoseFactoryTest
{
private static final GoogleStorage STORAGE = new GoogleStorage(null);
@Test
public void testSerde() throws IOException
{
final ObjectMapper mapper = GoogleCloudStorageInputSourceTest.createGoogleObjectMapper();
final List<GoogleBlob> blobs = ImmutableList.of(
new GoogleBlob("foo", "bar"),
new GoogleBlob("foo", "bar2")
);
final StaticGoogleBlobStoreFirehoseFactory factory = new StaticGoogleBlobStoreFirehoseFactory(
STORAGE,
blobs,
2048L,
1024L,
512L,
100L,
5
);
final StaticGoogleBlobStoreFirehoseFactory outputFact = mapper.readValue(
mapper.writeValueAsString(factory),
StaticGoogleBlobStoreFirehoseFactory.class
);
Assert.assertEquals(factory, outputFact);
}
}

View File

@ -1,127 +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.firehose.hdfs;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Predicate;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import org.apache.druid.guice.Hdfs;
import org.apache.druid.inputsource.hdfs.HdfsInputSource;
import org.apache.druid.inputsource.hdfs.HdfsInputSourceConfig;
import org.apache.druid.storage.hdfs.HdfsDataSegmentPuller;
import org.apache.druid.utils.CompressionUtils;
import org.apache.hadoop.conf.Configuration;
import org.apache.hadoop.fs.FSDataInputStream;
import org.apache.hadoop.fs.Path;
import java.io.IOException;
import java.io.InputStream;
import java.util.Collection;
import java.util.List;
public class HdfsFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<Path>
{
private final List<String> inputPaths;
private final Configuration conf;
private final HdfsInputSourceConfig inputSourceConfig;
@JsonCreator
public HdfsFirehoseFactory(
@JsonProperty("paths") Object inputPaths,
@JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
@JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
@JsonProperty("fetchTimeout") Long fetchTimeout,
@JsonProperty("maxFetchRetry") Integer maxFetchRetry,
@JacksonInject @Hdfs Configuration conf,
@JacksonInject HdfsInputSourceConfig inputSourceConfig
)
{
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
this.inputPaths = HdfsInputSource.coerceInputPathsToList(inputPaths, "paths");
this.conf = conf;
this.inputSourceConfig = inputSourceConfig;
this.inputPaths.forEach(p -> HdfsInputSource.verifyProtocol(conf, inputSourceConfig, p));
}
@JsonProperty("paths")
public List<String> getInputPaths()
{
return inputPaths;
}
@Override
protected Collection<Path> initObjects() throws IOException
{
return HdfsInputSource.getPaths(inputPaths, conf);
}
@Override
protected InputStream openObjectStream(Path path) throws IOException
{
return path.getFileSystem(conf).open(path);
}
@Override
protected InputStream openObjectStream(Path path, long start) throws IOException
{
final FSDataInputStream in = path.getFileSystem(conf).open(path);
in.seek(start);
return in;
}
@Override
protected InputStream wrapObjectStream(Path path, InputStream stream) throws IOException
{
return CompressionUtils.decompress(stream, path.getName());
}
@Override
protected Predicate<Throwable> getRetryCondition()
{
return HdfsDataSegmentPuller.RETRY_PREDICATE;
}
@Override
public boolean isSplittable()
{
return true;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, Path> withSplit(InputSplit<Path> split)
{
return new HdfsFirehoseFactory(
split.get().toString(),
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry(),
conf,
inputSourceConfig
);
}
}

View File

@ -26,7 +26,6 @@ import com.google.inject.Binder;
import com.google.inject.Inject;
import com.google.inject.multibindings.MapBinder;
import org.apache.druid.data.SearchableVersionedDataFinder;
import org.apache.druid.firehose.hdfs.HdfsFirehoseFactory;
import org.apache.druid.guice.Binders;
import org.apache.druid.guice.Hdfs;
import org.apache.druid.guice.JsonConfigProvider;
@ -66,7 +65,6 @@ public class HdfsStorageDruidModule implements DruidModule
return Collections.singletonList(
new SimpleModule().registerSubtypes(
new NamedType(HdfsLoadSpec.class, HdfsStorageDruidModule.SCHEME),
new NamedType(HdfsFirehoseFactory.class, HdfsStorageDruidModule.SCHEME),
new NamedType(HdfsInputSource.class, HdfsStorageDruidModule.SCHEME)
)
);

View File

@ -1,218 +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.firehose.hdfs;
import com.fasterxml.jackson.databind.InjectableValues.Std;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.inputsource.hdfs.HdfsInputSourceConfig;
import org.apache.druid.storage.hdfs.HdfsStorageDruidModule;
import org.apache.hadoop.conf.Configuration;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.Collections;
public class HdfsFirehoseFactoryTest
{
private static final HdfsInputSourceConfig DEFAULT_INPUT_SOURCE_CONFIG = new HdfsInputSourceConfig(null);
private static final Configuration DEFAULT_CONFIGURATION = new Configuration();
@BeforeClass
public static void setup()
{
DEFAULT_CONFIGURATION.set("fs.default.name", "hdfs://localhost:7020");
}
@Rule
public ExpectedException expectedException = ExpectedException.none();
@Test
public void testArrayPaths() throws IOException
{
final HdfsFirehoseFactory firehoseFactory = new HdfsFirehoseFactory(
Collections.singletonList("/foo/bar"),
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
final ObjectMapper mapper = createMapper();
final HdfsFirehoseFactory firehoseFactory2 = (HdfsFirehoseFactory)
mapper.readValue(mapper.writeValueAsString(firehoseFactory), FirehoseFactory.class);
Assert.assertEquals(
firehoseFactory.getInputPaths(),
firehoseFactory2.getInputPaths()
);
}
@Test
public void testStringPaths() throws IOException
{
final HdfsFirehoseFactory firehoseFactory = new HdfsFirehoseFactory(
"/foo/bar",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
final ObjectMapper mapper = createMapper();
final HdfsFirehoseFactory firehoseFactory2 = (HdfsFirehoseFactory)
mapper.readValue(mapper.writeValueAsString(firehoseFactory), FirehoseFactory.class);
Assert.assertEquals(
firehoseFactory.getInputPaths(),
firehoseFactory2.getInputPaths()
);
}
@Test
public void testConstructorAllowsOnlyDefaultProtocol()
{
new HdfsFirehoseFactory(
"hdfs://localhost:7020/foo/bar",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("Only [hdfs] protocols are allowed");
new HdfsFirehoseFactory(
"file:/foo/bar",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
}
@Test
public void testConstructorAllowsOnlyCustomProtocol()
{
final Configuration conf = new Configuration();
conf.set("fs.ftp.impl", "org.apache.hadoop.fs.ftp.FTPFileSystem");
new HdfsFirehoseFactory(
"ftp://localhost:21/foo/bar",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
new HdfsInputSourceConfig(ImmutableSet.of("ftp"))
);
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("Only [druid] protocols are allowed");
new HdfsFirehoseFactory(
"hdfs://localhost:7020/foo/bar",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
new HdfsInputSourceConfig(ImmutableSet.of("druid"))
);
}
@Test
public void testConstructorWithDefaultHdfs()
{
new HdfsFirehoseFactory(
"/foo/bar*",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
new HdfsFirehoseFactory(
"foo/bar*",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
new HdfsFirehoseFactory(
"hdfs:///foo/bar*",
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
new HdfsFirehoseFactory(
"hdfs://localhost:10020/foo/bar*", // different hdfs
null,
null,
null,
null,
null,
DEFAULT_CONFIGURATION,
DEFAULT_INPUT_SOURCE_CONFIG
);
}
private static ObjectMapper createMapper()
{
final ObjectMapper mapper = new ObjectMapper();
new HdfsStorageDruidModule().getJacksonModules().forEach(mapper::registerModule);
mapper.setInjectableValues(
new Std()
.addValue(Configuration.class, DEFAULT_CONFIGURATION)
.addValue(HdfsInputSourceConfig.class, DEFAULT_INPUT_SOURCE_CONFIG)
);
return mapper;
}
}

View File

@ -24,11 +24,15 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.curator.test.TestingCluster;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.client.indexing.SamplerSpec;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.FloatDimensionSchema;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorIOConfig;
import org.apache.druid.indexing.kafka.supervisor.KafkaSupervisorSpec;
@ -37,6 +41,7 @@ import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
import org.apache.druid.indexing.overlord.sampler.SamplerConfig;
import org.apache.druid.indexing.overlord.sampler.SamplerException;
import org.apache.druid.indexing.overlord.sampler.SamplerTestUtils;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
@ -56,9 +61,11 @@ import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.ExpectedException;
import java.io.IOException;
import java.util.Arrays;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
{
@ -167,6 +174,87 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
OBJECT_MAPPER
);
runSamplerAndCompareResponse(samplerSpec, true);
}
@Test
public void testWithInputRowParser() throws IOException
{
insertData(generateRecords(TOPIC));
ObjectMapper objectMapper = new DefaultObjectMapper();
TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null);
DimensionsSpec dimensionsSpec = new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
);
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
DataSchema dataSchema = new DataSchema(
"test_ds",
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class),
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
objectMapper
);
KafkaSupervisorSpec supervisorSpec = new KafkaSupervisorSpec(
null,
dataSchema,
null,
new KafkaSupervisorIOConfig(
TOPIC,
null,
null,
null,
null,
kafkaServer.consumerProperties(),
null,
null,
null,
null,
true,
null,
null,
null,
null,
null,
null
),
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null
);
KafkaSamplerSpec samplerSpec = new KafkaSamplerSpec(
supervisorSpec,
new SamplerConfig(5, null, null, null),
new InputSourceSampler(new DefaultObjectMapper()),
OBJECT_MAPPER
);
runSamplerAndCompareResponse(samplerSpec, false);
}
private static void runSamplerAndCompareResponse(SamplerSpec samplerSpec, boolean useInputFormat)
{
SamplerResponse response = samplerSpec.sample();
Assert.assertEquals(5, response.getNumRowsRead());
@ -258,7 +346,7 @@ public class KafkaSamplerSpecTest extends InitializedNullHandlingTest
null,
null,
true,
"Unable to parse row [unparseable] into JSON"
"Unable to parse row [unparseable]" + (useInputFormat ? " into JSON" : "")
), it.next());
Assert.assertFalse(it.hasNext());

View File

@ -24,14 +24,18 @@ import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.client.indexing.SamplerSpec;
import org.apache.druid.common.aws.AWSCredentialsConfig;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.FloatDimensionSchema;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorIOConfig;
import org.apache.druid.indexing.kinesis.supervisor.KinesisSupervisorSpec;
@ -54,10 +58,12 @@ import org.easymock.EasyMockSupport;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class KinesisSamplerSpecTest extends EasyMockSupport
{
@ -112,23 +118,8 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
}
@Test(timeout = 10_000L)
public void testSample() throws Exception
public void testSample() throws InterruptedException
{
EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).once();
recordSupplier.assign(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID)));
EasyMock.expectLastCall().once();
recordSupplier.seekToEarliest(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID)));
EasyMock.expectLastCall().once();
EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(STREAM)).once();
recordSupplier.close();
EasyMock.expectLastCall().once();
replayAll();
KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec(
null,
DATA_SCHEMA,
@ -176,6 +167,104 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
null
);
runSamplerAndCompareResponse(samplerSpec, true);
}
@Test
public void testSampleWithInputRowParser() throws IOException, InterruptedException
{
ObjectMapper objectMapper = new DefaultObjectMapper();
TimestampSpec timestampSpec = new TimestampSpec("timestamp", "iso", null);
DimensionsSpec dimensionsSpec = new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
);
InputRowParser parser = new StringInputRowParser(new JSONParseSpec(timestampSpec, dimensionsSpec, JSONPathSpec.DEFAULT, null, null), "UTF8");
DataSchema dataSchema = new DataSchema(
"test_ds",
objectMapper.readValue(objectMapper.writeValueAsBytes(parser), Map.class),
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
objectMapper
);
KinesisSupervisorSpec supervisorSpec = new KinesisSupervisorSpec(
null,
dataSchema,
null,
new KinesisSupervisorIOConfig(
STREAM,
null,
null,
null,
null,
null,
null,
null,
null,
true,
null,
null,
null,
null,
null,
null,
null,
null,
null,
false
),
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null,
null
);
KinesisSamplerSpec samplerSpec = new TestableKinesisSamplerSpec(
supervisorSpec,
new SamplerConfig(5, null, null, null),
new InputSourceSampler(new DefaultObjectMapper()),
null
);
runSamplerAndCompareResponse(samplerSpec, false);
}
private void runSamplerAndCompareResponse(SamplerSpec samplerSpec, boolean useInputFormat) throws InterruptedException
{
EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).once();
recordSupplier.assign(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID)));
EasyMock.expectLastCall().once();
recordSupplier.seekToEarliest(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID)));
EasyMock.expectLastCall().once();
EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(STREAM)).once();
recordSupplier.close();
EasyMock.expectLastCall().once();
replayAll();
SamplerResponse response = samplerSpec.sample();
verifyAll();
@ -269,7 +358,7 @@ public class KinesisSamplerSpecTest extends EasyMockSupport
null,
null,
true,
"Unable to parse row [unparseable] into JSON"
"Unable to parse row [unparseable]" + (useInputFormat ? " into JSON" : "")
), it.next());
Assert.assertFalse(it.hasNext());

View File

@ -1,48 +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.firehose.s3;
import com.fasterxml.jackson.databind.Module;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.initialization.DruidModule;
import java.util.List;
/**
*/
public class S3FirehoseDruidModule implements DruidModule
{
@Override
public List<? extends Module> getJacksonModules()
{
return ImmutableList.of(
new SimpleModule().registerSubtypes(new NamedType(StaticS3FirehoseFactory.class, "static-s3"))
);
}
@Override
public void configure(Binder binder)
{
}
}

View File

@ -1,236 +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.firehose.s3;
import com.amazonaws.services.s3.model.AmazonS3Exception;
import com.amazonaws.services.s3.model.GetObjectRequest;
import com.amazonaws.services.s3.model.S3Object;
import com.amazonaws.services.s3.model.S3ObjectSummary;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import com.google.common.base.Predicate;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.storage.s3.S3Utils;
import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
import org.apache.druid.utils.CompressionUtils;
import java.io.IOException;
import java.io.InputStream;
import java.net.URI;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Objects;
import java.util.stream.Collectors;
/**
* Builds firehoses that read from a predefined list of S3 objects and then dry up.
*/
public class StaticS3FirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
{
private static final Logger log = new Logger(StaticS3FirehoseFactory.class);
private static final int MAX_LISTING_LENGTH = 1024;
private final ServerSideEncryptingAmazonS3 s3Client;
private final List<URI> uris;
private final List<URI> prefixes;
@JsonCreator
public StaticS3FirehoseFactory(
@JacksonInject ServerSideEncryptingAmazonS3 s3Client,
@JsonProperty("uris") List<URI> uris,
@JsonProperty("prefixes") List<URI> prefixes,
@JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
@JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
@JsonProperty("fetchTimeout") Long fetchTimeout,
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
)
{
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
this.s3Client = Preconditions.checkNotNull(s3Client, "s3Client");
this.uris = uris == null ? new ArrayList<>() : uris;
this.prefixes = prefixes == null ? new ArrayList<>() : prefixes;
if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) {
throw new IAE("uris and prefixes cannot be used together");
}
if (this.uris.isEmpty() && this.prefixes.isEmpty()) {
throw new IAE("uris or prefixes must be specified");
}
for (final URI inputURI : this.uris) {
Preconditions.checkArgument("s3".equals(inputURI.getScheme()), "input uri scheme == s3 (%s)", inputURI);
}
for (final URI inputURI : this.prefixes) {
Preconditions.checkArgument("s3".equals(inputURI.getScheme()), "input uri scheme == s3 (%s)", inputURI);
}
}
@JsonProperty
public List<URI> getUris()
{
return uris;
}
@JsonProperty("prefixes")
public List<URI> getPrefixes()
{
return prefixes;
}
@Override
protected Collection<URI> initObjects()
{
if (!uris.isEmpty()) {
return uris;
} else {
final List<S3ObjectSummary> objects = new ArrayList<>();
for (final URI prefix : prefixes) {
final Iterator<S3ObjectSummary> objectSummaryIterator = S3Utils.objectSummaryIterator(
s3Client,
Collections.singletonList(prefix),
MAX_LISTING_LENGTH
);
objectSummaryIterator.forEachRemaining(objects::add);
}
return objects.stream().map(S3Utils::summaryToUri).collect(Collectors.toList());
}
}
@Override
protected InputStream openObjectStream(URI object) throws IOException
{
try {
// Get data of the given object and open an input stream
final String bucket = object.getAuthority();
final String key = S3Utils.extractS3Key(object);
final S3Object s3Object = s3Client.getObject(bucket, key);
if (s3Object == null) {
throw new ISE("Failed to get an s3 object for bucket[%s] and key[%s]", bucket, key);
}
return s3Object.getObjectContent();
}
catch (AmazonS3Exception e) {
throw new IOException(e);
}
}
@Override
protected InputStream openObjectStream(URI object, long start) throws IOException
{
final String bucket = object.getAuthority();
final String key = S3Utils.extractS3Key(object);
final GetObjectRequest request = new GetObjectRequest(bucket, key);
request.setRange(start);
try {
final S3Object s3Object = s3Client.getObject(request);
if (s3Object == null) {
throw new ISE(
"Failed to get an s3 object for bucket[%s], key[%s], and start[%d]",
bucket,
key,
start
);
}
return s3Object.getObjectContent();
}
catch (AmazonS3Exception e) {
throw new IOException(e);
}
}
@Override
protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
{
return CompressionUtils.decompress(stream, S3Utils.extractS3Key(object));
}
@Override
public boolean equals(Object o)
{
if (this == o) {
return true;
}
if (o == null || getClass() != o.getClass()) {
return false;
}
StaticS3FirehoseFactory that = (StaticS3FirehoseFactory) o;
return Objects.equals(uris, that.uris) &&
Objects.equals(prefixes, that.prefixes) &&
getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
getFetchTimeout() == that.getFetchTimeout() &&
getMaxFetchRetry() == that.getMaxFetchRetry();
}
@Override
public int hashCode()
{
return Objects.hash(
uris,
prefixes,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
@Override
protected Predicate<Throwable> getRetryCondition()
{
return S3Utils.S3RETRY;
}
@Override
public FiniteFirehoseFactory<StringInputRowParser, URI> withSplit(InputSplit<URI> split)
{
return new StaticS3FirehoseFactory(
s3Client,
Collections.singletonList(split.get()),
null,
getMaxCacheCapacityBytes(),
getMaxFetchCapacityBytes(),
getPrefetchTriggerBytes(),
getFetchTimeout(),
getMaxFetchRetry()
);
}
}

View File

@ -15,5 +15,4 @@
org.apache.druid.storage.s3.output.S3StorageConnectorModule
org.apache.druid.storage.s3.S3StorageDruidModule
org.apache.druid.firehose.s3.S3FirehoseDruidModule
org.apache.druid.data.input.s3.S3InputSourceDruidModule

View File

@ -1,115 +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.firehose.s3;
import com.amazonaws.services.s3.AmazonS3Client;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.s3.S3InputSourceTest;
import org.apache.druid.storage.s3.NoopServerSideEncryption;
import org.apache.druid.storage.s3.ServerSideEncryptingAmazonS3;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Test;
import java.io.IOException;
import java.net.URI;
import java.util.Arrays;
import java.util.Comparator;
import java.util.List;
import java.util.stream.Collectors;
/**
*/
public class StaticS3FirehoseFactoryTest
{
private static final AmazonS3Client S3_CLIENT = EasyMock.createNiceMock(AmazonS3Client.class);
private static final ServerSideEncryptingAmazonS3 SERVICE = new ServerSideEncryptingAmazonS3(
S3_CLIENT,
new NoopServerSideEncryption()
);
@Test
public void testSerde() throws Exception
{
final ObjectMapper mapper = S3InputSourceTest.createS3ObjectMapper();
final List<URI> uris = Arrays.asList(
new URI("s3://foo/bar/file.gz"),
new URI("s3://bar/foo/file2.gz")
);
final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory(
SERVICE,
uris,
null,
2048L,
1024L,
512L,
100L,
5
);
final StaticS3FirehoseFactory outputFact = mapper.readValue(
mapper.writeValueAsString(factory),
StaticS3FirehoseFactory.class
);
Assert.assertEquals(factory, outputFact);
}
@Test
public void testWithSplit() throws IOException
{
final List<URI> uris = Arrays.asList(
URI.create("s3://foo/bar/file.gz"),
URI.create("s3://bar/foo/file2.gz")
);
uris.sort(Comparator.comparing(URI::toString));
final StaticS3FirehoseFactory factory = new StaticS3FirehoseFactory(
SERVICE,
uris,
null,
2048L,
1024L,
512L,
100L,
5
);
final List<FiniteFirehoseFactory<StringInputRowParser, URI>> subFactories = factory
.getSplits(null)
.map(factory::withSplit)
.sorted(Comparator.comparing(eachFactory -> {
final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) eachFactory;
return staticS3FirehoseFactory.getUris().toString();
}))
.collect(Collectors.toList());
Assert.assertEquals(uris.size(), subFactories.size());
for (int i = 0; i < uris.size(); i++) {
final StaticS3FirehoseFactory staticS3FirehoseFactory = (StaticS3FirehoseFactory) subFactories.get(i);
final List<URI> subFactoryUris = staticS3FirehoseFactory.getUris();
Assert.assertEquals(1, subFactoryUris.size());
Assert.assertEquals(uris.get(i), subFactoryUris.get(0));
}
}
}

View File

@ -24,7 +24,6 @@ import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.collect.ImmutableList;
import com.google.inject.Binder;
import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.initialization.DruidModule;
import org.apache.druid.segment.realtime.firehose.EventReceiverFirehoseFactory;
@ -38,8 +37,7 @@ public class IndexingServiceFirehoseModule implements DruidModule
return ImmutableList.<Module>of(
new SimpleModule("IndexingServiceFirehoseModule")
.registerSubtypes(
new NamedType(EventReceiverFirehoseFactory.class, "receiver"),
new NamedType(IngestSegmentFirehoseFactory.class, "ingestSegment")
new NamedType(EventReceiverFirehoseFactory.class, "receiver")
)
);
}

View File

@ -1,140 +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.indexing.common;
import com.google.common.collect.BiMap;
import com.google.common.collect.HashBiMap;
import com.google.common.collect.Lists;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.partition.PartitionChunk;
import javax.annotation.Nullable;
import javax.validation.constraints.NotNull;
import java.util.List;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
* @deprecated only used by {@link org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory}
*/
@Deprecated
public class ReingestionTimelineUtils
{
/**
* @param timelineSegments A list of timeline objects, such as that returned by VersionedIntervalTimeline.lookup().
* @param excludeDimensions Dimensions to be excluded
* @return A list of all the unique dimension column names present in the segments within timelineSegments
*/
public static List<String> getUniqueDimensions(
List<TimelineObjectHolder<String, DataSegment>> timelineSegments,
@Nullable Set<String> excludeDimensions
)
{
final BiMap<String, Integer> uniqueDims = HashBiMap.create();
// Here, we try to retain the order of dimensions as they were specified since the order of dimensions may be
// optimized for performance.
// Dimensions are extracted from the recent segments to olders because recent segments are likely to be queried more
// frequently, and thus the performance should be optimized for recent ones rather than old ones.
// timelineSegments are sorted in order of interval
int index = 0;
for (TimelineObjectHolder<String, DataSegment> timelineHolder : Lists.reverse(timelineSegments)) {
for (PartitionChunk<DataSegment> chunk : timelineHolder.getObject()) {
for (String dimension : chunk.getObject().getDimensions()) {
if (!uniqueDims.containsKey(dimension) &&
(excludeDimensions == null || !excludeDimensions.contains(dimension))) {
uniqueDims.put(dimension, index++);
}
}
}
}
final BiMap<Integer, String> orderedDims = uniqueDims.inverse();
return IntStream.range(0, orderedDims.size())
.mapToObj(orderedDims::get)
.collect(Collectors.toList());
}
/**
* @param timelineSegments A list of timeline objects, such as that returned by VersionedIntervalTimeline.lookup().
* @return A list of all the unique metric column names present in the segments within timelineSegments
*/
public static List<String> getUniqueMetrics(List<TimelineObjectHolder<String, DataSegment>> timelineSegments)
{
final BiMap<String, Integer> uniqueMetrics = HashBiMap.create();
// Here, we try to retain the order of metrics as they were specified. Metrics are extracted from the recent
// segments to olders.
// timelineSegments are sorted in order of interval
int[] index = {0};
for (TimelineObjectHolder<String, DataSegment> timelineHolder : Lists.reverse(timelineSegments)) {
for (PartitionChunk<DataSegment> chunk : timelineHolder.getObject()) {
for (String metric : chunk.getObject().getMetrics()) {
uniqueMetrics.computeIfAbsent(
metric,
k -> {
return index[0]++;
}
);
}
}
}
final BiMap<Integer, String> orderedMetrics = uniqueMetrics.inverse();
return IntStream.range(0, orderedMetrics.size())
.mapToObj(orderedMetrics::get)
.collect(Collectors.toList());
}
/**
* Utility function to get dimensions that should be ingested. The preferred order is
* - Explicit dimensions if they are provided.
* - Custom dimensions are provided in the inputSpec.
* - Calculate dimensions from the timeline but exclude any dimension exclusions.
*
* @param explicitDimensions sent as part of the re-ingestion InputSource.
* @param dimensionsSpec from the provided ingestion spec.
* @param timeLineSegments for the datasource that is being read.
* @return
*/
public static List<String> getDimensionsToReingest(
@Nullable List<String> explicitDimensions,
@NotNull DimensionsSpec dimensionsSpec,
@NotNull List<TimelineObjectHolder<String, DataSegment>> timeLineSegments)
{
final List<String> dims;
if (explicitDimensions != null) {
dims = explicitDimensions;
} else if (dimensionsSpec.hasCustomDimensions()) {
dims = dimensionsSpec.getDimensionNames();
} else {
dims = ReingestionTimelineUtils.getUniqueDimensions(
timeLineSegments,
dimensionsSpec.getDimensionExclusions()
);
}
return dims;
}
}

View File

@ -25,7 +25,6 @@ import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.errorprone.annotations.concurrent.GuardedBy;
import org.apache.druid.client.indexing.ClientCompactionTaskTransformSpec;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputSource;
@ -44,8 +43,6 @@ import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
import org.apache.druid.indexing.common.task.IndexTask.IndexTuningConfig;
import org.apache.druid.indexing.common.task.batch.MaxAllowedLocksExceededException;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.indexing.firehose.WindowedSegmentId;
import org.apache.druid.indexing.input.InputRowSchemas;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.java.util.common.IAE;
@ -85,7 +82,6 @@ import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Collection;
import java.util.Collections;
import java.util.HashMap;
import java.util.Iterator;
@ -570,13 +566,9 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
}
/**
* If the given firehoseFactory is {@link IngestSegmentFirehoseFactory}, then it finds the segments to lock
* from the firehoseFactory. This is because those segments will be read by this task no matter what segments would be
* filtered by intervalsToRead, so they need to be locked.
* <p>
* However, firehoseFactory is not IngestSegmentFirehoseFactory, it means this task will overwrite some segments
* with data read from some input source outside of Druid. As a result, only the segments falling in intervalsToRead
* should be locked.
* This task will overwrite some segments with data read from input source outside of Druid.
* As a result, only the segments falling in intervalsToRead should be locked.
* <p>
* The order of segments within the returned list is unspecified, but each segment is guaranteed to appear in the list
* only once.
@ -584,48 +576,14 @@ public abstract class AbstractBatchIndexTask extends AbstractTask
protected static List<DataSegment> findInputSegments(
String dataSource,
TaskActionClient actionClient,
List<Interval> intervalsToRead,
FirehoseFactory firehoseFactory
List<Interval> intervalsToRead
) throws IOException
{
if (firehoseFactory instanceof IngestSegmentFirehoseFactory) {
// intervalsToRead is ignored here.
final List<WindowedSegmentId> inputSegments = ((IngestSegmentFirehoseFactory) firehoseFactory).getSegments();
if (inputSegments == null) {
final Interval inputInterval = Preconditions.checkNotNull(
((IngestSegmentFirehoseFactory) firehoseFactory).getInterval(),
"input interval"
);
return ImmutableList.copyOf(
actionClient.submit(
new RetrieveUsedSegmentsAction(dataSource, inputInterval, null, Segments.ONLY_VISIBLE)
)
);
} else {
final List<String> inputSegmentIds =
inputSegments.stream().map(WindowedSegmentId::getSegmentId).collect(Collectors.toList());
final Collection<DataSegment> dataSegmentsInIntervals = actionClient.submit(
new RetrieveUsedSegmentsAction(
dataSource,
null,
inputSegments.stream()
.flatMap(windowedSegmentId -> windowedSegmentId.getIntervals().stream())
.collect(Collectors.toSet()),
Segments.ONLY_VISIBLE
)
);
return dataSegmentsInIntervals.stream()
.filter(segment -> inputSegmentIds.contains(segment.getId().toString()))
.collect(Collectors.toList());
}
} else {
return ImmutableList.copyOf(
actionClient.submit(
new RetrieveUsedSegmentsAction(dataSource, null, intervalsToRead, Segments.ONLY_VISIBLE)
)
);
}
return ImmutableList.copyOf(
actionClient.submit(
new RetrieveUsedSegmentsAction(dataSource, null, intervalsToRead, Segments.ONLY_VISIBLE)
)
);
}
/**

View File

@ -112,6 +112,7 @@ import java.util.concurrent.ExecutionException;
import java.util.concurrent.TimeUnit;
import java.util.concurrent.TimeoutException;
@Deprecated
public class AppenderatorDriverRealtimeIndexTask extends AbstractTask implements ChatHandler
{
private static final String CTX_KEY_LOOKUP_TIER = "lookupTier";

View File

@ -32,14 +32,11 @@ import com.google.common.collect.ImmutableList;
import com.google.common.hash.HashFunction;
import com.google.common.hash.Hashing;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.Rows;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.hll.HyperLogLogCollector;
import org.apache.druid.indexer.Checks;
import org.apache.druid.indexer.IngestionState;
@ -269,8 +266,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
return findInputSegments(
getDataSource(),
taskActionClient,
intervals,
ingestionSchema.ioConfig.firehoseFactory
intervals
);
}
@ -486,9 +482,7 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
.inputIntervals()
.isEmpty();
final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
final InputSource inputSource = ingestionSchema.getIOConfig().getInputSource();
final File tmpDir = toolbox.getIndexingTmpDir();
@ -1195,16 +1189,9 @@ public class IndexTask extends AbstractBatchIndexTask implements ChatHandler
return inputFormat;
}
public InputSource getNonNullInputSource(@Nullable InputRowParser inputRowParser)
public InputSource getNonNullInputSource()
{
if (inputSource == null) {
return new FirehoseFactoryToInputSourceAdaptor(
(FiniteFirehoseFactory) firehoseFactory,
inputRowParser
);
} else {
return inputSource;
}
return Preconditions.checkNotNull(inputSource, "inputSource");
}
public InputFormat getNonNullInputFormat()

View File

@ -78,6 +78,7 @@ import java.util.TimerTask;
import java.util.concurrent.CountDownLatch;
import java.util.concurrent.ThreadLocalRandom;
@Deprecated
public class RealtimeIndexTask extends AbstractTask
{
public static final String CTX_KEY_LOOKUP_TIER = "lookupTier";

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexing.common.task.batch.parallel;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.SplittableInputSource;
@ -59,9 +58,7 @@ abstract class InputSourceSplitParallelIndexTaskRunner<T extends Task, R extends
context
);
this.ingestionSchema = ingestionSchema;
this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource();
}
@Override
@ -86,13 +83,8 @@ abstract class InputSourceSplitParallelIndexTaskRunner<T extends Task, R extends
{
final FirehoseFactory firehoseFactory;
final InputSource inputSource;
if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) {
firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split);
inputSource = null;
} else {
firehoseFactory = null;
inputSource = baseInputSource.withSplit(split);
}
firehoseFactory = null;
inputSource = baseInputSource.withSplit(split);
final ParallelIndexIngestionSpec subTaskIngestionSpec = new ParallelIndexIngestionSpec(
ingestionSchema.getDataSchema(),
new ParallelIndexIOConfig(

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.collect.ImmutableList;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.indexer.Checks;
import org.apache.druid.indexer.Property;
import org.apache.druid.java.util.common.IAE;
@ -45,9 +44,7 @@ public class ParallelIndexIngestionSpec extends IngestionSpec<ParallelIndexIOCon
super(dataSchema, ioConfig, tuningConfig);
if (dataSchema.getParserMap() != null && ioConfig.getInputSource() != null) {
if (!(ioConfig.getInputSource() instanceof FirehoseFactoryToInputSourceAdaptor)) {
throw new IAE("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
}
throw new IAE("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
}
if (ioConfig.getInputSource() != null && ioConfig.getInputSource().needsFormat()) {
Checks.checkOneNotNullOrEmpty(

View File

@ -30,7 +30,6 @@ import org.apache.datasketches.hll.HllSketch;
import org.apache.datasketches.hll.Union;
import org.apache.datasketches.memory.Memory;
import org.apache.druid.common.guava.FutureUtils;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.indexer.IngestionState;
@ -122,7 +121,7 @@ import java.util.stream.Collectors;
/**
* ParallelIndexSupervisorTask is capable of running multiple subTasks for parallel indexing. This is
* applicable if the input {@link FiniteFirehoseFactory} is splittable. While this task is running, it can submit
* applicable if the input {@link InputSource} is splittable. While this task is running, it can submit
* multiple child tasks to overlords. This task succeeds only when all its child tasks succeed; otherwise it fails.
*
* @see ParallelIndexTaskRunner
@ -241,9 +240,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
checkPartitionsSpecForForceGuaranteedRollup(ingestionSchema.getTuningConfig().getGivenOrDefaultPartitionsSpec());
}
this.baseInputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
this.baseInputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
this.missingIntervalsInOverwriteMode = (getIngestionMode()
!= IngestionMode.APPEND)
&& ingestionSchema.getDataSchema()
@ -426,8 +423,7 @@ public class ParallelIndexSupervisorTask extends AbstractBatchIndexTask implemen
return findInputSegments(
getDataSource(),
taskActionClient,
intervals,
ingestionSchema.getIOConfig().getFirehoseFactory()
intervals
);
}

View File

@ -156,9 +156,7 @@ public class PartialDimensionCardinalityTask extends PerfectRollupWorkerTask
HashedPartitionsSpec partitionsSpec = (HashedPartitionsSpec) tuningConfig.getPartitionsSpec();
Preconditions.checkNotNull(partitionsSpec, "partitionsSpec required in tuningConfig");
InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
InputFormat inputFormat = inputSource.needsFormat()
? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema)
: null;

View File

@ -203,9 +203,7 @@ public class PartialDimensionDistributionTask extends PerfectRollupWorkerTask
);
boolean isAssumeGrouped = partitionsSpec.isAssumeGrouped();
InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
InputFormat inputFormat = inputSource.needsFormat()
? ParallelIndexSupervisorTask.getInputFormat(ingestionSchema)
: null;

View File

@ -109,9 +109,7 @@ abstract class PartialSegmentGenerateTask<T extends GeneratedPartitionsReport> e
@Override
public final TaskStatus runTask(TaskToolbox toolbox) throws Exception
{
final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,

View File

@ -22,7 +22,6 @@ package org.apache.druid.indexing.common.task.batch.parallel;
import com.google.common.annotations.VisibleForTesting;
import org.apache.commons.lang3.mutable.MutableObject;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.impl.SplittableInputSource;
@ -118,9 +117,7 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner
context
);
this.ingestionSchema = ingestionSchema;
this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
this.baseInputSource = (SplittableInputSource) ingestionSchema.getIOConfig().getNonNullInputSource();
}
@VisibleForTesting
@ -171,13 +168,9 @@ public class SinglePhaseParallelIndexTaskRunner extends ParallelIndexPhaseRunner
{
final FirehoseFactory firehoseFactory;
final InputSource inputSource;
if (baseInputSource instanceof FirehoseFactoryToInputSourceAdaptor) {
firehoseFactory = ((FirehoseFactoryToInputSourceAdaptor) baseInputSource).getFirehoseFactory().withSplit(split);
inputSource = null;
} else {
firehoseFactory = null;
inputSource = baseInputSource.withSplit(split);
}
firehoseFactory = null;
inputSource = baseInputSource.withSplit(split);
final Map<String, Object> subtaskContext = new HashMap<>(getContext());
return new SinglePhaseSubTaskSpec(
getBaseSubtaskSpecName() + "_" + getAndIncrementNextSpecId(),

View File

@ -247,9 +247,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
ingestionSchema.getTuningConfig().getMaxSavedParseExceptions()
);
final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource(
ingestionSchema.getDataSchema().getParser()
);
final InputSource inputSource = ingestionSchema.getIOConfig().getNonNullInputSource();
final ParallelIndexSupervisorTaskClient taskClient = toolbox.getSupervisorTaskClientProvider().build(
supervisorTaskId,
@ -307,8 +305,7 @@ public class SinglePhaseSubTask extends AbstractBatchSubtask implements ChatHand
return findInputSegments(
getDataSource(),
taskActionClient,
intervals,
ingestionSchema.getIOConfig().getFirehoseFactory()
intervals
);
}

View File

@ -1,330 +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.indexing.firehose;
import com.fasterxml.jackson.annotation.JacksonInject;
import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Function;
import com.google.common.base.Preconditions;
import com.google.common.collect.Iterables;
import com.google.common.collect.Lists;
import com.google.common.collect.Maps;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.SegmentsSplitHintSpec;
import org.apache.druid.data.input.SplitHintSpec;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.indexing.common.ReingestionTimelineUtils;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.input.DruidInputSource;
import org.apache.druid.java.util.common.HumanReadableBytes;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.parsers.ParseException;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.query.filter.DimFilter;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.loading.SegmentCacheManager;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.realtime.firehose.IngestSegmentFirehose;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.joda.time.Interval;
import javax.annotation.Nullable;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.Map;
import java.util.stream.Stream;
/**
* @deprecated use {@link DruidInputSource} instead
*/
@Deprecated
public class IngestSegmentFirehoseFactory implements FiniteFirehoseFactory<InputRowParser, List<WindowedSegmentId>>
{
private static final EmittingLogger log = new EmittingLogger(IngestSegmentFirehoseFactory.class);
private final String dataSource;
// Exactly one of interval and segmentIds should be non-null. Typically 'interval' is specified directly
// by the user creating this firehose and 'segmentIds' is used for sub-tasks if it is split for parallel
// batch ingestion.
@Nullable
private final Interval interval;
@Nullable
private final List<WindowedSegmentId> segmentIds;
private final DimFilter dimFilter;
private final List<String> dimensions;
private final List<String> metrics;
@Nullable
private final Long maxInputSegmentBytesPerTask;
private final IndexIO indexIO;
private final CoordinatorClient coordinatorClient;
private final SegmentCacheManagerFactory segmentCacheManagerFactory;
private final RetryPolicyFactory retryPolicyFactory;
private List<InputSplit<List<WindowedSegmentId>>> splits;
@JsonCreator
public IngestSegmentFirehoseFactory(
@JsonProperty("dataSource") final String dataSource,
@JsonProperty("interval") @Nullable Interval interval,
// Specifying "segments" is intended only for when this FirehoseFactory has split itself,
// not for direct end user use.
@JsonProperty("segments") @Nullable List<WindowedSegmentId> segmentIds,
@JsonProperty("filter") DimFilter dimFilter,
@JsonProperty("dimensions") List<String> dimensions,
@JsonProperty("metrics") List<String> metrics,
@JsonProperty("maxInputSegmentBytesPerTask") @Deprecated @Nullable Long maxInputSegmentBytesPerTask,
@JacksonInject IndexIO indexIO,
@JacksonInject CoordinatorClient coordinatorClient,
@JacksonInject SegmentCacheManagerFactory segmentCacheManagerFactory,
@JacksonInject RetryPolicyFactory retryPolicyFactory
)
{
Preconditions.checkNotNull(dataSource, "dataSource");
if ((interval == null && segmentIds == null) || (interval != null && segmentIds != null)) {
throw new IAE("Specify exactly one of 'interval' and 'segments'");
}
this.dataSource = dataSource;
this.interval = interval;
this.segmentIds = segmentIds;
this.dimFilter = dimFilter;
this.dimensions = dimensions;
this.metrics = metrics;
this.maxInputSegmentBytesPerTask = maxInputSegmentBytesPerTask;
this.indexIO = Preconditions.checkNotNull(indexIO, "null IndexIO");
this.coordinatorClient = Preconditions.checkNotNull(coordinatorClient, "null CoordinatorClient");
this.segmentCacheManagerFactory = Preconditions.checkNotNull(segmentCacheManagerFactory, "null segmentCacheManagerFactory");
this.retryPolicyFactory = Preconditions.checkNotNull(retryPolicyFactory, "null RetryPolicyFactory");
}
@Override
public FiniteFirehoseFactory<InputRowParser, List<WindowedSegmentId>> withSplit(InputSplit<List<WindowedSegmentId>> split)
{
return new IngestSegmentFirehoseFactory(
dataSource,
null,
split.get(),
dimFilter,
dimensions,
metrics,
maxInputSegmentBytesPerTask,
indexIO,
coordinatorClient,
segmentCacheManagerFactory,
retryPolicyFactory
);
}
@JsonProperty
public String getDataSource()
{
return dataSource;
}
@JsonProperty
@Nullable
public Interval getInterval()
{
return interval;
}
@JsonProperty
@Nullable
public List<WindowedSegmentId> getSegments()
{
return segmentIds;
}
@JsonProperty("filter")
public DimFilter getDimensionsFilter()
{
return dimFilter;
}
@JsonProperty
public List<String> getDimensions()
{
return dimensions;
}
@JsonProperty
public List<String> getMetrics()
{
return metrics;
}
@Nullable
@JsonProperty
public Long getMaxInputSegmentBytesPerTask()
{
return maxInputSegmentBytesPerTask;
}
@Override
public Firehose connect(InputRowParser inputRowParser, File temporaryDirectory) throws ParseException
{
log.debug(
"Connecting firehose: dataSource[%s], interval[%s], segmentIds[%s]",
dataSource,
interval,
segmentIds
);
final List<TimelineObjectHolder<String, DataSegment>> timeLineSegments = getTimeline();
// Download all segments locally.
// Note: this requires enough local storage space to fit all of the segments, even though
// IngestSegmentFirehose iterates over the segments in series. We may want to change this
// to download files lazily, perhaps sharing code with PrefetchableTextFilesFirehoseFactory.
final SegmentCacheManager segmentCacheManager = segmentCacheManagerFactory.manufacturate(temporaryDirectory);
Map<DataSegment, File> segmentFileMap = Maps.newLinkedHashMap();
for (TimelineObjectHolder<String, DataSegment> holder : timeLineSegments) {
for (PartitionChunk<DataSegment> chunk : holder.getObject()) {
final DataSegment segment = chunk.getObject();
segmentFileMap.computeIfAbsent(segment, k -> {
try {
return segmentCacheManager.getSegmentFiles(segment);
}
catch (SegmentLoadingException e) {
throw new RuntimeException(e);
}
});
}
}
final List<String> dims = ReingestionTimelineUtils.getDimensionsToReingest(
dimensions,
inputRowParser.getParseSpec().getDimensionsSpec(),
timeLineSegments
);
final List<String> metricsList = metrics == null
? ReingestionTimelineUtils.getUniqueMetrics(timeLineSegments)
: metrics;
final List<WindowedStorageAdapter> adapters = Lists.newArrayList(
Iterables.concat(
Iterables.transform(
timeLineSegments,
new Function<TimelineObjectHolder<String, DataSegment>, Iterable<WindowedStorageAdapter>>() {
@Override
public Iterable<WindowedStorageAdapter> apply(final TimelineObjectHolder<String, DataSegment> holder)
{
return
Iterables.transform(
holder.getObject(),
new Function<PartitionChunk<DataSegment>, WindowedStorageAdapter>() {
@Override
public WindowedStorageAdapter apply(final PartitionChunk<DataSegment> input)
{
final DataSegment segment = input.getObject();
try {
return new WindowedStorageAdapter(
new QueryableIndexStorageAdapter(
indexIO.loadIndex(
Preconditions.checkNotNull(
segmentFileMap.get(segment),
"File for segment %s", segment.getId()
)
)
),
holder.getInterval()
);
}
catch (IOException e) {
throw new RuntimeException(e);
}
}
}
);
}
}
)
)
);
final TransformSpec transformSpec = TransformSpec.fromInputRowParser(inputRowParser);
return new IngestSegmentFirehose(adapters, transformSpec, dims, metricsList, dimFilter);
}
private List<TimelineObjectHolder<String, DataSegment>> getTimeline()
{
if (interval == null) {
return DruidInputSource.getTimelineForSegmentIds(coordinatorClient, dataSource, segmentIds);
} else {
return DruidInputSource.getTimelineForInterval(coordinatorClient, retryPolicyFactory, dataSource, interval);
}
}
private void initializeSplitsIfNeeded(@Nullable SplitHintSpec splitHintSpec)
{
if (splits != null) {
return;
}
splits = Lists.newArrayList(
DruidInputSource.createSplits(
coordinatorClient,
retryPolicyFactory,
dataSource,
interval,
splitHintSpec == null
? new SegmentsSplitHintSpec(
maxInputSegmentBytesPerTask == null
? null
: new HumanReadableBytes(maxInputSegmentBytesPerTask),
null
)
: splitHintSpec
)
);
}
@Override
public boolean isSplittable()
{
// Specifying 'segments' to this factory instead of 'interval' is intended primarily for internal use by
// parallel batch injection: we don't need to support splitting a list of segments.
return interval != null;
}
@Override
public Stream<InputSplit<List<WindowedSegmentId>>> getSplits(@Nullable SplitHintSpec splitHintSpec)
{
initializeSplitsIfNeeded(splitHintSpec);
return splits.stream();
}
@Override
public int getNumSplits(@Nullable SplitHintSpec splitHintSpec)
{
initializeSplitsIfNeeded(splitHintSpec);
return splits.size();
}
}

View File

@ -25,13 +25,9 @@ import com.fasterxml.jackson.annotation.JsonProperty;
import com.google.common.base.Preconditions;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.client.indexing.SamplerSpec;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.segment.indexing.DataSchema;
import javax.annotation.Nullable;
@ -61,28 +57,17 @@ public class IndexTaskSamplerSpec implements SamplerSpec
Preconditions.checkNotNull(ingestionSpec.getIOConfig(), "[spec.ioConfig] is required");
if (ingestionSpec.getIOConfig().getInputSource() != null) {
this.inputSource = ingestionSpec.getIOConfig().getInputSource();
if (ingestionSpec.getIOConfig().getInputSource().needsFormat()) {
this.inputFormat = Preconditions.checkNotNull(
ingestionSpec.getIOConfig().getInputFormat(),
"[spec.ioConfig.inputFormat] is required"
);
} else {
this.inputFormat = null;
}
this.inputSource = Preconditions.checkNotNull(
ingestionSpec.getIOConfig().getInputSource(),
"[spec.ioConfig.inputSource] is required"
);
if (inputSource.needsFormat()) {
this.inputFormat = Preconditions.checkNotNull(
ingestionSpec.getIOConfig().getInputFormat(),
"[spec.ioConfig.inputFormat] is required"
);
} else {
final FirehoseFactory firehoseFactory = Preconditions.checkNotNull(
ingestionSpec.getIOConfig().getFirehoseFactory(),
"[spec.ioConfig.firehose] is required"
);
if (!(firehoseFactory instanceof FiniteFirehoseFactory)) {
throw new IAE("firehose should be an instance of FiniteFirehoseFactory");
}
this.inputSource = new FirehoseFactoryToInputSourceAdaptor(
(FiniteFirehoseFactory) firehoseFactory,
ingestionSpec.getDataSchema().getParser()
);
this.inputFormat = null;
}

View File

@ -23,19 +23,20 @@ import com.google.common.base.Preconditions;
import com.google.common.base.Throwables;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.client.indexing.SamplerSpec;
import org.apache.druid.data.input.ByteBufferInputRowParser;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.AbstractInputSource;
import org.apache.druid.data.input.InputEntity;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputRowListPlusRawValues;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.InputSourceReader;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.InputStats;
import org.apache.druid.data.input.SplitHintSpec;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.SplittableInputSource;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
import org.apache.druid.indexing.overlord.sampler.SamplerConfig;
@ -88,10 +89,7 @@ public abstract class SeekableStreamSamplerSpec<PartitionIdType, SequenceOffsetT
final InputSource inputSource;
final InputFormat inputFormat;
if (dataSchema.getParser() != null) {
inputSource = new FirehoseFactoryToInputSourceAdaptor(
new SeekableStreamSamplerFirehoseFactory(),
dataSchema.getParser()
);
inputSource = new SeekableStreamSamplerInputSource(dataSchema.getParser());
inputFormat = null;
} else {
RecordSupplier<PartitionIdType, SequenceOffsetType, RecordType> recordSupplier;
@ -120,18 +118,18 @@ public abstract class SeekableStreamSamplerSpec<PartitionIdType, SequenceOffsetT
protected abstract RecordSupplier<PartitionIdType, SequenceOffsetType, RecordType> createRecordSupplier();
private class SeekableStreamSamplerFirehoseFactory implements FiniteFirehoseFactory<ByteBufferInputRowParser, Object>
private class SeekableStreamSamplerInputSource extends AbstractInputSource implements SplittableInputSource
{
@Override
public Firehose connect(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory)
private final InputRowParser parser;
public SeekableStreamSamplerInputSource(InputRowParser parser)
{
throw new UnsupportedOperationException();
this.parser = parser;
}
@Override
public Firehose connectForSampler(ByteBufferInputRowParser parser, @Nullable File temporaryDirectory)
public InputRowParser getParser()
{
return new SeekableStreamSamplerFirehose(parser);
return parser;
}
@Override
@ -141,30 +139,42 @@ public abstract class SeekableStreamSamplerSpec<PartitionIdType, SequenceOffsetT
}
@Override
public Stream<InputSplit<Object>> getSplits(@Nullable SplitHintSpec splitHintSpec)
public Stream<InputSplit> createSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec)
{
throw new UnsupportedOperationException();
}
@Override
public int getNumSplits(@Nullable SplitHintSpec splitHintSpec)
public int estimateNumSplits(InputFormat inputFormat, @Nullable SplitHintSpec splitHintSpec)
{
throw new UnsupportedOperationException();
}
@Override
public FiniteFirehoseFactory withSplit(InputSplit split)
public SplittableInputSource withSplit(InputSplit split)
{
throw new UnsupportedOperationException();
}
@Override
public boolean needsFormat()
{
return false;
}
@Override
protected InputSourceReader fixedFormatReader(InputRowSchema inputRowSchema, @Nullable File temporaryDirectory)
{
return new SeekableStreamSamplerInputSourceReader(parser);
}
}
private class SeekableStreamSamplerFirehose implements Firehose
private class SeekableStreamSamplerInputSourceReader implements InputSourceReader
{
private final InputRowParser parser;
private final CloseableIterator<InputEntity> entityIterator;
protected SeekableStreamSamplerFirehose(InputRowParser parser)
public SeekableStreamSamplerInputSourceReader(InputRowParser parser)
{
this.parser = parser;
if (parser instanceof StringInputRowParser) {
@ -181,47 +191,81 @@ public abstract class SeekableStreamSamplerSpec<PartitionIdType, SequenceOffsetT
}
@Override
public boolean hasMore()
public CloseableIterator<InputRow> read()
{
return entityIterator.hasNext();
}
return new CloseableIterator<InputRow>()
{
@Override
public InputRow nextRow()
{
throw new UnsupportedOperationException();
}
@Override
public InputRowListPlusRawValues nextRowWithRaw()
{
final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer();
final Map<String, Object> rawColumns;
try {
if (parser instanceof StringInputRowParser) {
rawColumns = ((StringInputRowParser) parser).buildStringKeyMap(bb);
} else {
rawColumns = null;
@Override
public boolean hasNext()
{
return entityIterator.hasNext();
}
}
catch (ParseException e) {
return InputRowListPlusRawValues.of(null, e);
}
try {
final List<InputRow> rows = parser.parseBatch(bb);
return InputRowListPlusRawValues.of(rows.isEmpty() ? null : rows, rawColumns);
}
catch (ParseException e) {
return InputRowListPlusRawValues.of(rawColumns, e);
}
@Override
public InputRow next()
{
throw new UnsupportedOperationException();
}
@Override
public void close() throws IOException
{
entityIterator.close();
}
};
}
@Override
public void close() throws IOException
public CloseableIterator<InputRow> read(InputStats inputStats)
{
entityIterator.close();
return null;
}
@Override
public CloseableIterator<InputRowListPlusRawValues> sample()
{
return new CloseableIterator<InputRowListPlusRawValues>()
{
@Override
public boolean hasNext()
{
return entityIterator.hasNext();
}
@Override
public InputRowListPlusRawValues next()
{
final ByteBuffer bb = ((ByteEntity) entityIterator.next()).getBuffer();
final Map<String, Object> rawColumns;
try {
if (parser instanceof StringInputRowParser) {
rawColumns = ((StringInputRowParser) parser).buildStringKeyMap(bb);
} else {
rawColumns = null;
}
}
catch (ParseException e) {
return InputRowListPlusRawValues.of(null, e);
}
try {
bb.position(0);
final List<InputRow> rows = parser.parseBatch(bb);
return InputRowListPlusRawValues.of(rows.isEmpty() ? null : rows, rawColumns);
}
catch (ParseException e) {
return InputRowListPlusRawValues.of(rawColumns, e);
}
}
@Override
public void close() throws IOException
{
entityIterator.close();
}
};
}
}
}

View File

@ -24,26 +24,27 @@ import com.fasterxml.jackson.annotation.JsonCreator;
import com.fasterxml.jackson.annotation.JsonProperty;
import com.fasterxml.jackson.annotation.JsonTypeName;
import com.fasterxml.jackson.databind.ObjectMapper;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.task.RealtimeIndexTask;
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
import org.apache.druid.indexing.common.task.IndexTask;
import org.apache.druid.indexing.common.task.TaskResource;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import java.io.File;
/**
*/
@JsonTypeName("test_realtime")
public class TestRealtimeTask extends RealtimeIndexTask
@JsonTypeName("test_index")
public class TestIndexTask extends IndexTask
{
private final TaskStatus status;
@JsonCreator
public TestRealtimeTask(
public TestIndexTask(
@JsonProperty("id") String id,
@JsonProperty("resource") TaskResource taskResource,
@JsonProperty("dataSource") String dataSource,
@ -54,13 +55,42 @@ public class TestRealtimeTask extends RealtimeIndexTask
super(
id,
taskResource,
new FireDepartment(
new IndexIngestionSpec(
new DataSchema(dataSource, null, new AggregatorFactory[]{}, null, null, mapper),
new RealtimeIOConfig(
new LocalFirehoseFactory(new File("lol"), "rofl", null),
(schema, config, metrics) -> null
new IndexTask.IndexIOConfig(
null,
new LocalInputSource(new File("lol"), "rofl"),
new JsonInputFormat(null, null, null, null, null),
false,
false
),
null
new IndexTask.IndexTuningConfig(
null,
null,
null,
10,
null,
null,
null,
null,
null,
null,
new DynamicPartitionsSpec(10000, null),
new IndexSpec(),
null,
3,
false,
null,
null,
null,
null,
null,
null,
null,
null,
null
)
),
null
);

View File

@ -32,7 +32,6 @@ import org.apache.druid.client.indexing.NoopOverlordClient;
import org.apache.druid.data.input.impl.CSVParseSpec;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
@ -48,7 +47,6 @@ import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskConfig;
import org.apache.druid.indexing.common.task.CompactionTask.Builder;
import org.apache.druid.indexing.common.task.batch.parallel.ParallelIndexTuningConfig;
import org.apache.druid.indexing.firehose.IngestSegmentFirehoseFactory;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.ISE;
@ -71,7 +69,6 @@ import org.apache.druid.segment.DimensionSelector;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.QueryableIndexStorageAdapter;
import org.apache.druid.segment.VirtualColumns;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.join.NoopJoinableFactory;
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
@ -1489,96 +1486,6 @@ public class CompactionTaskRunTest extends IngestionTestBase
Assert.assertEquals(TaskState.FAILED, compactionResult.lhs.getStatusCode());
}
/**
* Run a regular index task that's equivalent to the compaction task in {@link #testRunWithDynamicPartitioning()},
* using {@link IngestSegmentFirehoseFactory}.
* <p>
* This is not entirely CompactionTask related, but it's similar conceptually and it requires
* similar setup to what this test suite already has.
* <p>
* It could be moved to a separate test class if needed.
*/
@Test
public void testRunRegularIndexTaskWithIngestSegmentFirehose() throws Exception
{
runIndexTask();
IndexTask indexTask = new IndexTask(
null,
null,
new IndexTask.IndexIngestionSpec(
new DataSchema(
"test",
getObjectMapper().convertValue(
new StringInputRowParser(
DEFAULT_PARSE_SPEC,
null
),
Map.class
),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
new UniformGranularitySpec(
Granularities.HOUR,
Granularities.MINUTE,
null
),
null,
getObjectMapper()
),
new IndexTask.IndexIOConfig(
new IngestSegmentFirehoseFactory(
DATA_SOURCE,
Intervals.of("2014-01-01/2014-01-02"),
null,
null,
null,
null,
null,
getIndexIO(),
coordinatorClient,
segmentCacheManagerFactory,
RETRY_POLICY_FACTORY
),
false,
false
),
IndexTaskTest.createTuningConfig(5000000, null, null, Long.MAX_VALUE, null, false, true)
),
null
);
// This is a regular index so we need to explicitly add this context to store the CompactionState
indexTask.addToContext(Tasks.STORE_COMPACTION_STATE_KEY, true);
final Pair<TaskStatus, List<DataSegment>> resultPair = runTask(indexTask);
Assert.assertTrue(resultPair.lhs.isSuccess());
final List<DataSegment> segments = resultPair.rhs;
Assert.assertEquals(3, segments.size());
for (int i = 0; i < 3; i++) {
Assert.assertEquals(
Intervals.of("2014-01-01T0%d:00:00/2014-01-01T0%d:00:00", i, i + 1),
segments.get(i).getInterval()
);
Assert.assertEquals(
getDefaultCompactionState(Granularities.HOUR, Granularities.MINUTE, ImmutableList.of()),
segments.get(i).getLastCompactionState()
);
if (lockGranularity == LockGranularity.SEGMENT) {
Assert.assertEquals(
new NumberedOverwriteShardSpec(32768, 0, 2, (short) 1, (short) 1),
segments.get(i).getShardSpec()
);
} else {
Assert.assertEquals(new NumberedShardSpec(0, 1), segments.get(i).getShardSpec());
}
}
}
private Pair<TaskStatus, List<DataSegment>> runIndexTask() throws Exception
{
return runIndexTask(null, null, false);

View File

@ -20,11 +20,8 @@
package org.apache.druid.indexing.common.task;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.NoopFirehoseFactory;
import org.apache.druid.data.input.impl.NoopInputFormat;
import org.apache.druid.data.input.impl.NoopInputSource;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.common.task.IndexTask.IndexIOConfig;
import org.apache.druid.indexing.common.task.IndexTask.IndexIngestionSpec;
import org.apache.druid.java.util.common.granularity.Granularities;
@ -91,56 +88,4 @@ public class IndexIngestionSpecTest
null
);
}
@Test
public void testFirehoseAndInputSource()
{
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage(
"At most one of [Property{name='firehose', value=NoopFirehoseFactory{}}, Property{name='inputSource'"
);
final IndexIngestionSpec spec = new IndexIngestionSpec(
new DataSchema(
"dataSource",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[0],
new ArbitraryGranularitySpec(Granularities.NONE, null),
null
),
new IndexIOConfig(
new NoopFirehoseFactory(),
new NoopInputSource(),
null,
null,
null
),
null
);
}
@Test
public void testFirehoseAndInputFormat()
{
expectedException.expect(IllegalArgumentException.class);
expectedException.expectMessage("Cannot use firehose and inputFormat together.");
final IndexIngestionSpec spec = new IndexIngestionSpec(
new DataSchema(
"dataSource",
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
new AggregatorFactory[0],
new ArbitraryGranularitySpec(Granularities.NONE, null),
null
),
new IndexIOConfig(
new NoopFirehoseFactory(),
null,
new NoopInputFormat(),
null,
null
),
null
);
}
}

View File

@ -38,7 +38,6 @@ import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
@ -87,7 +86,6 @@ import org.apache.druid.segment.loading.SegmentLoaderConfig;
import org.apache.druid.segment.loading.SegmentLocalCacheManager;
import org.apache.druid.segment.loading.StorageLocationConfig;
import org.apache.druid.segment.realtime.appenderator.AppenderatorsManager;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.segment.realtime.firehose.WindowedStorageAdapter;
import org.apache.druid.segment.realtime.plumber.NoopSegmentHandoffNotifierFactory;
import org.apache.druid.segment.transform.ExpressionTransform;
@ -956,7 +954,9 @@ public class IndexTaskTest extends IngestionTestBase
ingestionSpec = createIngestionSpec(
jsonMapper,
tmpDir,
new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0),
timestampSpec,
DimensionsSpec.EMPTY,
new CsvInputFormat(columns, null, null, true, 0),
null,
null,
tuningConfig,
@ -967,9 +967,7 @@ public class IndexTaskTest extends IngestionTestBase
ingestionSpec = createIngestionSpec(
jsonMapper,
tmpDir,
timestampSpec,
DimensionsSpec.EMPTY,
new CsvInputFormat(columns, null, null, true, 0),
new CSVParseSpec(timestampSpec, DimensionsSpec.EMPTY, null, columns, true, 0),
null,
null,
tuningConfig,
@ -1484,12 +1482,6 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = ImmutableList.of(
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1} (Path: %s, Record: 1, Line: 2)",
tmpFile.toURI()
)
);
} else {
indexIngestionSpec = createIngestionSpec(
jsonMapper,
@ -1501,11 +1493,14 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = ImmutableList.of(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1}"
);
}
expectedMessages = ImmutableList.of(
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, d=a, val=1} (Path: %s, Record: 1, Line: 2)",
tmpFile.toURI()
)
);
IndexTask indexTask = new IndexTask(
null,
null,
@ -1625,13 +1620,12 @@ public class IndexTaskTest extends IngestionTestBase
IngestionStatsAndErrorsTaskReportData reportData = getTaskReportData();
final int processedBytes = useInputFormatApi ? 657 : 0;
Map<String, Object> expectedMetrics = ImmutableMap.of(
RowIngestionMeters.DETERMINE_PARTITIONS,
ImmutableMap.of(
RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
RowIngestionMeters.PROCESSED, 4,
RowIngestionMeters.PROCESSED_BYTES, processedBytes,
RowIngestionMeters.PROCESSED_BYTES, 657,
RowIngestionMeters.UNPARSEABLE, 4,
RowIngestionMeters.THROWN_AWAY, 1
),
@ -1639,7 +1633,7 @@ public class IndexTaskTest extends IngestionTestBase
ImmutableMap.of(
RowIngestionMeters.PROCESSED_WITH_ERROR, 3,
RowIngestionMeters.PROCESSED, 1,
RowIngestionMeters.PROCESSED_BYTES, processedBytes,
RowIngestionMeters.PROCESSED_BYTES, 657,
RowIngestionMeters.UNPARSEABLE, 4,
RowIngestionMeters.THROWN_AWAY, 1
)
@ -1650,36 +1644,24 @@ public class IndexTaskTest extends IngestionTestBase
ParseExceptionReport.forPhase(reportData, RowIngestionMeters.BUILD_SEGMENTS);
List<String> expectedMessages;
if (useInputFormatApi) {
expectedMessages = Arrays.asList(
StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
StringUtils.format(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
tmpFile.toURI()
),
StringUtils.format(
"Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
tmpFile.toURI()
),
"Unable to parse value[notnumber] for field[val]",
"could not convert value [notnumber] to float",
"could not convert value [notnumber] to long",
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
tmpFile.toURI()
)
);
} else {
expectedMessages = Arrays.asList(
"Unable to parse row [this is not JSON]",
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
"Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]",
"Unable to parse value[notnumber] for field[val]",
"could not convert value [notnumber] to float",
"could not convert value [notnumber] to long",
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
);
}
expectedMessages = Arrays.asList(
StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
StringUtils.format(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
tmpFile.toURI()
),
StringUtils.format(
"Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
tmpFile.toURI()
),
"Unable to parse value[notnumber] for field[val]",
"could not convert value [notnumber] to float",
"could not convert value [notnumber] to long",
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
tmpFile.toURI()
)
);
Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages());
@ -1697,30 +1679,21 @@ public class IndexTaskTest extends IngestionTestBase
parseExceptionReport =
ParseExceptionReport.forPhase(reportData, RowIngestionMeters.DETERMINE_PARTITIONS);
if (useInputFormatApi) {
expectedMessages = Arrays.asList(
StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
StringUtils.format(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
tmpFile.toURI()
),
StringUtils.format(
"Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
tmpFile.toURI()
),
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
tmpFile.toURI()
)
);
} else {
expectedMessages = Arrays.asList(
"Unable to parse row [this is not JSON]",
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
"Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}]",
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
);
}
expectedMessages = Arrays.asList(
StringUtils.format("Unable to parse row [this is not JSON] (Path: %s, Record: 6, Line: 9)", tmpFile.toURI()),
StringUtils.format(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 6, Line: 8)",
tmpFile.toURI()
),
StringUtils.format(
"Unable to parse row [{\"time\":9.0x,\"dim\":\"a\",\"dimLong\":2,\"dimFloat\":3.0,\"val\":1}] (Path: %s, Record: 5, Line: 6)",
tmpFile.toURI()
),
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 1)",
tmpFile.toURI()
)
);
Assert.assertEquals(expectedMessages, parseExceptionReport.getErrorMessages());
@ -1802,20 +1775,6 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = Arrays.asList(
StringUtils.format(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)",
tmpFile.toURI()
),
StringUtils.format(
"Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)",
tmpFile.toURI()
),
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)",
tmpFile.toURI()
)
);
} else {
ingestionSpec = createIngestionSpec(
jsonMapper,
@ -1827,13 +1786,22 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = Arrays.asList(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
"Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}",
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
);
}
expectedMessages = Arrays.asList(
StringUtils.format(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)",
tmpFile.toURI()
),
StringUtils.format(
"Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)",
tmpFile.toURI()
),
StringUtils.format(
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)",
tmpFile.toURI()
)
);
IndexTask indexTask = new IndexTask(
null,
null,
@ -1860,9 +1828,9 @@ public class IndexTaskTest extends IngestionTestBase
ImmutableMap.of(
RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
RowIngestionMeters.PROCESSED, 1,
RowIngestionMeters.PROCESSED_BYTES, useInputFormatApi ? 182 : 0,
RowIngestionMeters.PROCESSED_BYTES, 182,
RowIngestionMeters.UNPARSEABLE, 3,
RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2
RowIngestionMeters.THROWN_AWAY, 1
)
);
@ -1949,11 +1917,6 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = Arrays.asList(
StringUtils.format("Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", tmpFile.toURI()),
StringUtils.format("Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", tmpFile.toURI()),
StringUtils.format("Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", tmpFile.toURI())
);
} else {
ingestionSpec = createIngestionSpec(
jsonMapper,
@ -1965,13 +1928,13 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = Arrays.asList(
"Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1}",
"Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1}",
"Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1}"
);
}
expectedMessages = Arrays.asList(
StringUtils.format("Timestamp[99999999999-01-01T00:00:10Z] is unparseable! Event: {time=99999999999-01-01T00:00:10Z, dim=b, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 3, Line: 6)", tmpFile.toURI()),
StringUtils.format("Timestamp[9.0] is unparseable! Event: {time=9.0, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 2, Line: 4)", tmpFile.toURI()),
StringUtils.format("Timestamp[unparseable] is unparseable! Event: {time=unparseable, dim=a, dimLong=2, dimFloat=3.0, val=1} (Path: %s, Record: 1, Line: 2)", tmpFile.toURI())
);
IndexTask indexTask = new IndexTask(
null,
null,
@ -1990,9 +1953,9 @@ public class IndexTaskTest extends IngestionTestBase
ImmutableMap.of(
RowIngestionMeters.PROCESSED_WITH_ERROR, 0,
RowIngestionMeters.PROCESSED, 1,
RowIngestionMeters.PROCESSED_BYTES, useInputFormatApi ? 182 : 0,
RowIngestionMeters.PROCESSED_BYTES, 182,
RowIngestionMeters.UNPARSEABLE, 3,
RowIngestionMeters.THROWN_AWAY, useInputFormatApi ? 1 : 2
RowIngestionMeters.THROWN_AWAY, 1
),
RowIngestionMeters.BUILD_SEGMENTS,
ImmutableMap.of(
@ -2135,12 +2098,6 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = ImmutableList.of(
StringUtils.format(
"Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1} (Path: %s, Record: 1, Line: 2)",
tmpFile.toURI()
)
);
} else {
ingestionSpec = createIngestionSpec(
jsonMapper,
@ -2152,11 +2109,14 @@ public class IndexTaskTest extends IngestionTestBase
false,
false
);
expectedMessages = ImmutableList.of(
"Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1}"
);
}
expectedMessages = ImmutableList.of(
StringUtils.format(
"Timestamp[null] is unparseable! Event: {column_1=2014-01-01T00:00:10Z, column_2=a, column_3=1} (Path: %s, Record: 1, Line: 2)",
tmpFile.toURI()
)
);
IndexTask indexTask = new IndexTask(
null,
null,
@ -2916,16 +2876,12 @@ public class IndexTaskTest extends IngestionTestBase
tuningConfig
);
} else {
parseSpec = parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC;
return new IndexIngestionSpec(
new DataSchema(
DATASOURCE,
objectMapper.convertValue(
new StringInputRowParser(
parseSpec != null ? parseSpec : DEFAULT_PARSE_SPEC,
null
),
Map.class
),
parseSpec.getTimestampSpec(),
parseSpec.getDimensionsSpec(),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
@ -2935,14 +2891,13 @@ public class IndexTaskTest extends IngestionTestBase
Collections.singletonList(Intervals.of("2014/2015"))
),
transformSpec,
null,
objectMapper
),
new IndexIOConfig(
new LocalFirehoseFactory(
baseDir,
"druid*",
null
),
null,
new LocalInputSource(baseDir, "druid*"),
createInputFormatFromParseSpec(parseSpec),
appendToExisting,
dropExisting
),

View File

@ -23,6 +23,18 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.base.Optional;
import com.google.common.util.concurrent.Futures;
import com.google.common.util.concurrent.ListenableFuture;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.CSVParseSpec;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DelimitedInputFormat;
import org.apache.druid.data.input.impl.DelimitedParseSpec;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.RegexInputFormat;
import org.apache.druid.data.input.impl.RegexParseSpec;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.SingleFileTaskReportFileWriter;
@ -47,6 +59,7 @@ import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.indexing.overlord.autoscaling.ScalingStats;
import org.apache.druid.java.util.common.ISE;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.RE;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.emitter.EmittingLogger;
import org.apache.druid.metadata.EntryExistsException;
@ -222,6 +235,50 @@ public abstract class IngestionTestBase extends InitializedNullHandlingTest
return testUtils.getIndexMergerV9Factory();
}
/**
* Converts ParseSpec to InputFormat for indexing tests. To be used until {@link FirehoseFactory}
* & {@link InputRowParser} is deprecated and removed.
*
* @param parseSpec
* @return
*/
public static InputFormat createInputFormatFromParseSpec(ParseSpec parseSpec)
{
if (parseSpec instanceof JSONParseSpec) {
JSONParseSpec jsonParseSpec = (JSONParseSpec) parseSpec;
return new JsonInputFormat(jsonParseSpec.getFlattenSpec(), jsonParseSpec.getFeatureSpec(), jsonParseSpec.getKeepNullColumns(), null, null);
} else if (parseSpec instanceof CSVParseSpec) {
CSVParseSpec csvParseSpec = (CSVParseSpec) parseSpec;
boolean getColumnsFromHeader = csvParseSpec.isHasHeaderRow() && csvParseSpec.getSkipHeaderRows() == 0;
return new CsvInputFormat(
csvParseSpec.getColumns(),
csvParseSpec.getListDelimiter(),
getColumnsFromHeader ? null : true,
getColumnsFromHeader ? true : null,
csvParseSpec.getSkipHeaderRows()
);
} else if (parseSpec instanceof DelimitedParseSpec) {
DelimitedParseSpec delimitedParseSpec = (DelimitedParseSpec) parseSpec;
boolean getColumnsFromHeader = delimitedParseSpec.isHasHeaderRow() && delimitedParseSpec.getSkipHeaderRows() == 0;
return new DelimitedInputFormat(
delimitedParseSpec.getColumns(),
delimitedParseSpec.getListDelimiter(),
delimitedParseSpec.getDelimiter(),
getColumnsFromHeader ? null : true,
getColumnsFromHeader ? true : null,
delimitedParseSpec.getSkipHeaderRows()
);
} else if (parseSpec instanceof RegexParseSpec) {
RegexParseSpec regexParseSpec = (RegexParseSpec) parseSpec;
return new RegexInputFormat(
regexParseSpec.getPattern(),
regexParseSpec.getListDelimiter(),
regexParseSpec.getColumns());
} else {
throw new RE(StringUtils.format("Unsupported ParseSpec format %s", parseSpec.toString()));
}
}
public class TestLocalTaskActionClientFactory implements TaskActionClientFactory
{
@Override

View File

@ -24,7 +24,10 @@ import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.NoopInputFormat;
import org.apache.druid.data.input.impl.TimestampSpec;
@ -43,13 +46,11 @@ import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.incremental.RowIngestionMetersFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.indexing.RealtimeTuningConfig;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.server.security.AuthTestUtils;
import org.apache.druid.timeline.partition.NoneShardSpec;
import org.hamcrest.CoreMatchers;
@ -64,7 +65,6 @@ import java.io.File;
public class TaskSerdeTest
{
private final ObjectMapper jsonMapper;
private final RowIngestionMetersFactory rowIngestionMetersFactory;
private final IndexSpec indexSpec = new IndexSpec();
@Rule
@ -74,14 +74,14 @@ public class TaskSerdeTest
{
TestUtils testUtils = new TestUtils();
jsonMapper = testUtils.getTestObjectMapper();
rowIngestionMetersFactory = testUtils.getRowIngestionMetersFactory();
for (final Module jacksonModule : new FirehoseModule().getJacksonModules()) {
jsonMapper.registerModule(jacksonModule);
}
jsonMapper.registerSubtypes(
new NamedType(ParallelIndexTuningConfig.class, "index_parallel"),
new NamedType(IndexTuningConfig.class, "index")
new NamedType(IndexTuningConfig.class, "index"),
new NamedType(MockFirehoseFactory.class, "mock")
);
}
@ -402,7 +402,7 @@ public class TaskSerdeTest
jsonMapper
),
new RealtimeIOConfig(
new LocalFirehoseFactory(new File("lol"), "rofl", null),
new MockFirehoseFactory(),
(schema, config, metrics) -> null
),
@ -588,6 +588,15 @@ public class TaskSerdeTest
Assert.assertEquals("blah", task2.getClasspathPrefix());
}
private static class MockFirehoseFactory implements FirehoseFactory
{
@Override
public Firehose connect(InputRowParser parser, File temporaryDirectory)
{
return null;
}
}
@Test
public void testHadoopIndexTaskWithContextSerde() throws Exception
{

View File

@ -25,7 +25,6 @@ import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
@ -59,7 +58,6 @@ import org.apache.druid.segment.loading.SegmentLoader;
import org.apache.druid.segment.loading.SegmentLoadingException;
import org.apache.druid.segment.loading.SegmentLocalCacheLoader;
import org.apache.druid.segment.loading.TombstoneLoadSpec;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.timeline.DataSegment;
import org.joda.time.Interval;
import org.junit.Assert;
@ -241,24 +239,22 @@ abstract class AbstractMultiPhaseParallelIndexingTest extends AbstractParallelIn
tuningConfig
);
} else {
Preconditions.checkArgument(inputFormat == null);
Preconditions.checkArgument(inputFormat == null && parseSpec != null);
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDirectory, filter, null),
null,
new LocalInputSource(inputDirectory, filter),
createInputFormatFromParseSpec(parseSpec),
appendToExisting,
dropExisting
);
//noinspection unchecked
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
getObjectMapper().convertValue(
new StringInputRowParser(parseSpec, null),
Map.class
),
DATASOURCE,
parseSpec.getTimestampSpec(),
parseSpec.getDimensionsSpec(),
DEFAULT_METRICS_SPEC,
granularitySpec,
null,
getObjectMapper()
null
),
ioConfig,
tuningConfig

View File

@ -903,8 +903,10 @@ public class AbstractParallelIndexSupervisorTaskTest extends IngestionTestBase
Map<String, Object> expectedPayload = (Map<String, Object>) expectedReports.get("payload");
Map<String, Object> actualPayload = (Map<String, Object>) actualReports.get("payload");
Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
Assert.assertEquals(expectedPayload.get("rowStats"), actualPayload.get("rowStats"));
Assert.assertEquals(expectedPayload.get("ingestionState"), actualPayload.get("ingestionState"));
Map<String, Object> expectedTotals = (Map<String, Object>) expectedPayload.get("totals");
Map<String, Object> actualTotals = (Map<String, Object>) actualReports.get("totals");
Assert.assertEquals(expectedTotals, actualTotals);
List<ParseExceptionReport> expectedParseExceptionReports =
(List<ParseExceptionReport>) ((Map<String, Object>)

View File

@ -26,7 +26,6 @@ import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.ParseSpec;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
@ -47,7 +46,6 @@ import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.joda.time.Interval;
import org.junit.Assert;
import org.junit.Before;
@ -267,23 +265,23 @@ public class HashPartitionTaskKillTest extends AbstractMultiPhaseParallelIndexin
} else {
Preconditions.checkArgument(inputFormat == null);
ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDir, filter, null),
appendToExisting
null,
new LocalInputSource(inputDir, filter),
createInputFormatFromParseSpec(parseSpec),
appendToExisting,
null
);
//noinspection unchecked
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
getObjectMapper().convertValue(
new StringInputRowParser(parseSpec, null),
Map.class
),
parseSpec.getTimestampSpec(),
parseSpec.getDimensionsSpec(),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
granularitySpec,
null,
getObjectMapper()
null
),
ioConfig,
tuningConfig

View File

@ -20,7 +20,6 @@
package org.apache.druid.indexing.common.task.batch.parallel;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.jsontype.NamedType;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.impl.CsvInputFormat;
import org.apache.druid.data.input.impl.DimensionsSpec;
@ -38,7 +37,6 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.hamcrest.CoreMatchers;
import org.joda.time.Interval;
import org.junit.Assert;
@ -67,9 +65,6 @@ public class ParallelIndexSupervisorTaskSerdeTest
{
TestUtils testUtils = new TestUtils();
ObjectMapper objectMapper = testUtils.getTestObjectMapper();
objectMapper.registerSubtypes(
new NamedType(LocalFirehoseFactory.class, "local")
);
return objectMapper;
}

View File

@ -27,11 +27,15 @@ import org.apache.commons.codec.Charsets;
import org.apache.druid.data.input.InputSource;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexer.partitions.HashedPartitionsSpec;
import org.apache.druid.indexer.partitions.PartitionsSpec;
import org.apache.druid.indexer.partitions.SingleDimensionPartitionsSpec;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.IAE;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.http.client.response.StringFullResponseHolder;
import org.apache.druid.rpc.HttpResponseException;
@ -291,6 +295,83 @@ public class ParallelIndexSupervisorTaskTest
null
);
}
@Test
public void testFailToConstructWhenBothInputSourceAndParserAreSet()
{
final ObjectMapper mapper = new DefaultObjectMapper();
final ParallelIndexIOConfig ioConfig = new ParallelIndexIOConfig(
null,
new InlineInputSource("test"),
null,
false,
null
);
final ParallelIndexTuningConfig tuningConfig = new ParallelIndexTuningConfig(
null,
null,
null,
10,
1000L,
null,
null,
null,
null,
new HashedPartitionsSpec(null, 10, null),
new IndexSpec(
new RoaringBitmapSerdeFactory(true),
CompressionStrategy.UNCOMPRESSED,
CompressionStrategy.LZF,
LongEncodingStrategy.LONGS
),
new IndexSpec(),
1,
true,
true,
10000L,
OffHeapMemorySegmentWriteOutMediumFactory.instance(),
null,
10,
100,
20L,
new Duration(3600),
128,
null,
null,
false,
null,
null,
null,
null,
null
);
expectedException.expect(IAE.class);
expectedException.expectMessage("Cannot use parser and inputSource together. Try using inputFormat instead of parser.");
new ParallelIndexIngestionSpec(
new DataSchema(
"datasource",
mapper.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec(null, null, null),
DimensionsSpec.EMPTY,
null,
null,
null
)
),
Map.class
),
null,
null,
null,
mapper
),
ioConfig,
tuningConfig
);
}
}
public static class StaticUtilsTest

View File

@ -27,7 +27,6 @@ import org.apache.druid.data.input.impl.DimensionSchema;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LocalInputSource;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexing.common.LockGranularity;
import org.apache.druid.indexing.common.TaskToolbox;
@ -47,7 +46,6 @@ import org.apache.druid.segment.incremental.ParseExceptionReport;
import org.apache.druid.segment.incremental.RowIngestionMetersTotals;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.Partitions;
import org.apache.druid.timeline.SegmentTimeline;
@ -928,27 +926,22 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
ingestionSpec = new ParallelIndexIngestionSpec(
new DataSchema(
"dataSource",
getObjectMapper().convertValue(
new StringInputRowParser(
DEFAULT_PARSE_SPEC,
null
),
Map.class
),
new AggregatorFactory[]{
new LongSumAggregatorFactory("val", "val")
},
DEFAULT_TIMESTAMP_SPEC,
DEFAULT_DIMENSIONS_SPEC,
DEFAULT_METRICS_SPEC,
new UniformGranularitySpec(
segmentGranularity,
Granularities.MINUTE,
interval == null ? null : Collections.singletonList(interval)
),
null,
getObjectMapper()
null
),
new ParallelIndexIOConfig(
new LocalFirehoseFactory(inputDir, inputSourceFilter, null),
appendToExisting
null,
new LocalInputSource(inputDir, inputSourceFilter),
createInputFormatFromParseSpec(DEFAULT_PARSE_SPEC),
appendToExisting,
null
),
tuningConfig
);
@ -966,10 +959,10 @@ public class SinglePhaseParallelIndexingTest extends AbstractParallelIndexSuperv
private String getErrorMessageForUnparseableTimestamp()
{
return useInputFormatApi ? StringUtils.format(
return StringUtils.format(
"Timestamp[2017unparseable] is unparseable! Event: {ts=2017unparseable} (Path: %s, Record: 5, Line: 5)",
new File(inputDir, "test_0").toURI()
) : "Timestamp[2017unparseable] is unparseable! Event: {ts=2017unparseable}";
);
}
private static class SettableSplittableLocalInputSource extends LocalInputSource

View File

@ -1,615 +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.indexing.firehose;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.fasterxml.jackson.databind.introspect.AnnotationIntrospectorPair;
import com.fasterxml.jackson.databind.module.SimpleModule;
import com.google.common.base.Preconditions;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.inject.Binder;
import com.google.inject.Module;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.FirehoseFactory;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.MapInputRowParser;
import org.apache.druid.data.input.impl.TimeAndDimsParseSpec;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.guice.GuiceAnnotationIntrospector;
import org.apache.druid.guice.GuiceInjectableValues;
import org.apache.druid.guice.GuiceInjectors;
import org.apache.druid.indexing.common.ReingestionTimelineUtils;
import org.apache.druid.indexing.common.RetryPolicyConfig;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.config.TaskStorageConfig;
import org.apache.druid.indexing.common.task.NoopTask;
import org.apache.druid.indexing.common.task.Task;
import org.apache.druid.indexing.overlord.HeapMemoryTaskStorage;
import org.apache.druid.indexing.overlord.Segments;
import org.apache.druid.indexing.overlord.TaskLockbox;
import org.apache.druid.indexing.overlord.TaskStorage;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.logger.Logger;
import org.apache.druid.java.util.emitter.service.ServiceEmitter;
import org.apache.druid.math.expr.ExprMacroTable;
import org.apache.druid.metadata.IndexerSQLMetadataStorageCoordinator;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.TestHelper;
import org.apache.druid.segment.column.ColumnHolder;
import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.segment.loading.LocalDataSegmentPuller;
import org.apache.druid.segment.loading.LocalLoadSpec;
import org.apache.druid.segment.realtime.firehose.CombiningFirehoseFactory;
import org.apache.druid.segment.transform.ExpressionTransform;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.server.metrics.NoopServiceEmitter;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.TimelineObjectHolder;
import org.apache.druid.timeline.partition.NumberedPartitionChunk;
import org.apache.druid.timeline.partition.NumberedShardSpec;
import org.apache.druid.timeline.partition.PartitionChunk;
import org.apache.druid.timeline.partition.PartitionHolder;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.Before;
import org.junit.BeforeClass;
import org.junit.Rule;
import org.junit.Test;
import org.junit.rules.TemporaryFolder;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.nio.file.Paths;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.stream.Collectors;
import java.util.stream.IntStream;
/**
*
*/
@RunWith(Parameterized.class)
public class IngestSegmentFirehoseFactoryTest
{
private static final ObjectMapper MAPPER;
private static final IndexMergerV9 INDEX_MERGER_V9;
private static final IndexIO INDEX_IO;
private static final TaskStorage TASK_STORAGE;
private static final IndexerSQLMetadataStorageCoordinator MDC;
private static final TaskLockbox TASK_LOCKBOX;
private static final Task TASK;
@Rule
public TemporaryFolder temporaryFolder = new TemporaryFolder();
static {
TestUtils testUtils = new TestUtils();
MAPPER = setupInjectablesInObjectMapper(TestHelper.makeJsonMapper());
INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9();
INDEX_IO = testUtils.getTestIndexIO();
TASK_STORAGE = new HeapMemoryTaskStorage(
new TaskStorageConfig(null)
{
}
);
MDC = new IndexerSQLMetadataStorageCoordinator(null, null, null)
{
private final Set<DataSegment> published = new HashSet<>();
@Override
public List<DataSegment> retrieveUsedSegmentsForIntervals(
String dataSource,
List<Interval> interval,
Segments visibility
)
{
return ImmutableList.copyOf(SEGMENT_SET);
}
@Override
public List<DataSegment> retrieveUnusedSegmentsForInterval(String dataSource, Interval interval)
{
return ImmutableList.of();
}
@Override
public Set<DataSegment> announceHistoricalSegments(Set<DataSegment> segments)
{
Set<DataSegment> added = new HashSet<>();
for (final DataSegment segment : segments) {
if (published.add(segment)) {
added.add(segment);
}
}
return ImmutableSet.copyOf(added);
}
@Override
public void deleteSegments(Set<DataSegment> segments)
{
// do nothing
}
};
TASK_LOCKBOX = new TaskLockbox(TASK_STORAGE, MDC);
TASK = NoopTask.create();
TASK_LOCKBOX.add(TASK);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> constructorFeeder() throws IOException
{
final IndexSpec indexSpec = new IndexSpec();
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(JodaUtils.MIN_INSTANT)
.withDimensionsSpec(ROW_PARSER)
.withMetrics(
new LongSumAggregatorFactory(METRIC_LONG_NAME, DIM_LONG_NAME),
new DoubleSumAggregatorFactory(METRIC_FLOAT_NAME, DIM_FLOAT_NAME)
)
.build();
final IncrementalIndex index = new OnheapIncrementalIndex.Builder()
.setIndexSchema(schema)
.setMaxRowCount(MAX_ROWS * MAX_SHARD_NUMBER)
.build();
for (Integer i = 0; i < MAX_ROWS; ++i) {
index.add(ROW_PARSER.parseBatch(buildRow(i.longValue())).get(0));
}
FileUtils.mkdirp(PERSIST_DIR);
INDEX_MERGER_V9.persist(index, PERSIST_DIR, indexSpec, null);
final CoordinatorClient cc = new CoordinatorClient(null, null)
{
@Override
public Collection<DataSegment> fetchUsedSegmentsInDataSourceForIntervals(
String dataSource,
List<Interval> intervals
)
{
return ImmutableSet.copyOf(SEGMENT_SET);
}
};
SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class);
EasyMock.replay(notifierFactory);
final SegmentCacheManagerFactory slf = new SegmentCacheManagerFactory(MAPPER);
final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
Collection<Object[]> values = new ArrayList<>();
for (InputRowParser parser : Arrays.<InputRowParser>asList(
ROW_PARSER,
new MapInputRowParser(
new JSONParseSpec(
new TimestampSpec(TIME_COLUMN, "auto", null),
DimensionsSpec.builder()
.setDimensionExclusions(ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME))
.build(),
null,
null,
null
)
)
)) {
for (List<String> dim_names : Arrays.<List<String>>asList(null, ImmutableList.of(DIM_NAME))) {
for (List<String> metric_names : Arrays.<List<String>>asList(
null,
ImmutableList.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME)
)) {
for (Boolean wrapInCombining : Arrays.asList(false, true)) {
final IngestSegmentFirehoseFactory isfFactory = new IngestSegmentFirehoseFactory(
TASK.getDataSource(),
Intervals.ETERNITY,
null,
new SelectorDimFilter(DIM_NAME, DIM_VALUE, null),
dim_names,
metric_names,
null,
INDEX_IO,
cc,
slf,
retryPolicyFactory
);
final FirehoseFactory factory = wrapInCombining
? new CombiningFirehoseFactory(ImmutableList.of(isfFactory))
: isfFactory;
values.add(
new Object[]{
StringUtils.format(
"DimNames[%s]MetricNames[%s]ParserDimNames[%s]WrapInCombining[%s]",
dim_names == null ? "null" : "dims",
metric_names == null ? "null" : "metrics",
parser == ROW_PARSER ? "dims" : "null",
wrapInCombining
),
factory,
parser
}
);
}
}
}
}
return values;
}
public static ObjectMapper setupInjectablesInObjectMapper(ObjectMapper objectMapper)
{
objectMapper.registerModule(
new SimpleModule("testModule").registerSubtypes(LocalLoadSpec.class)
);
final GuiceAnnotationIntrospector guiceIntrospector = new GuiceAnnotationIntrospector();
objectMapper.setAnnotationIntrospectors(
new AnnotationIntrospectorPair(
guiceIntrospector,
objectMapper.getSerializationConfig().getAnnotationIntrospector()
),
new AnnotationIntrospectorPair(
guiceIntrospector,
objectMapper.getDeserializationConfig().getAnnotationIntrospector()
)
);
objectMapper.setInjectableValues(
new GuiceInjectableValues(
GuiceInjectors.makeStartupInjectorWithModules(
ImmutableList.of(
new Module()
{
@Override
public void configure(Binder binder)
{
binder.bind(LocalDataSegmentPuller.class);
binder.bind(ExprMacroTable.class).toInstance(TestExprMacroTable.INSTANCE);
}
}
)
)
)
);
return objectMapper;
}
public IngestSegmentFirehoseFactoryTest(
String testName,
FirehoseFactory factory,
InputRowParser rowParser
)
{
this.factory = factory;
// Must decorate the parser, since IngestSegmentFirehoseFactory will undecorate it.
this.rowParser = TransformSpec.NONE.decorate(rowParser);
}
private static final Logger log = new Logger(IngestSegmentFirehoseFactoryTest.class);
private static final String DATA_SOURCE_NAME = "testDataSource";
private static final String DATA_SOURCE_VERSION = "version";
private static final Integer BINARY_VERSION = -1;
private static final String DIM_NAME = "testDimName";
private static final String DIM_VALUE = "testDimValue";
private static final String DIM_LONG_NAME = "testDimLongName";
private static final String DIM_FLOAT_NAME = "testDimFloatName";
private static final String METRIC_LONG_NAME = "testLongMetric";
private static final String METRIC_FLOAT_NAME = "testFloatMetric";
private static final Long METRIC_LONG_VALUE = 1L;
private static final Float METRIC_FLOAT_VALUE = 1.0f;
private static final String TIME_COLUMN = "ts";
private static final Integer MAX_SHARD_NUMBER = 10;
private static final Integer MAX_ROWS = 10;
private static final File TMP_DIR = FileUtils.createTempDir();
private static final File PERSIST_DIR = Paths.get(TMP_DIR.getAbsolutePath(), "indexTestMerger").toFile();
private static final List<DataSegment> SEGMENT_SET = new ArrayList<>(MAX_SHARD_NUMBER);
private final FirehoseFactory<InputRowParser> factory;
private final InputRowParser rowParser;
private File tempDir;
private static final InputRowParser<Map<String, Object>> ROW_PARSER = new MapInputRowParser(
new TimeAndDimsParseSpec(
new TimestampSpec(TIME_COLUMN, "auto", null),
DimensionsSpec.builder()
.setDimensions(DimensionsSpec.getDefaultSchemas(ImmutableList.of(DIM_NAME)))
.setDimensionExclusions(ImmutableList.of(DIM_FLOAT_NAME, DIM_LONG_NAME))
.build()
)
);
private static Map<String, Object> buildRow(Long ts)
{
return ImmutableMap.of(
TIME_COLUMN, ts,
DIM_NAME, DIM_VALUE,
DIM_FLOAT_NAME, METRIC_FLOAT_VALUE,
DIM_LONG_NAME, METRIC_LONG_VALUE
);
}
private static DataSegment buildSegment(Integer shardNumber)
{
Preconditions.checkArgument(shardNumber < MAX_SHARD_NUMBER);
Preconditions.checkArgument(shardNumber >= 0);
return new DataSegment(
DATA_SOURCE_NAME,
Intervals.ETERNITY,
DATA_SOURCE_VERSION,
ImmutableMap.of(
"type", "local",
"path", PERSIST_DIR.getAbsolutePath()
),
ImmutableList.of(DIM_NAME),
ImmutableList.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME),
new NumberedShardSpec(
shardNumber,
MAX_SHARD_NUMBER
),
BINARY_VERSION,
0L
);
}
@BeforeClass
public static void setUpStatic()
{
for (int i = 0; i < MAX_SHARD_NUMBER; ++i) {
SEGMENT_SET.add(buildSegment(i));
}
}
@AfterClass
public static void tearDownStatic()
{
recursivelyDelete(TMP_DIR);
}
private static void recursivelyDelete(final File dir)
{
if (dir != null) {
if (dir.isDirectory()) {
final File[] files = dir.listFiles();
if (files != null) {
for (File file : files) {
recursivelyDelete(file);
}
}
} else {
if (!dir.delete()) {
log.warn("Could not delete file at [%s]", dir.getAbsolutePath());
}
}
}
}
@Before
public void setup() throws IOException
{
tempDir = temporaryFolder.newFolder();
}
@After
public void teardown()
{
tempDir.delete();
}
@Test
public void sanityTest()
{
if (factory instanceof CombiningFirehoseFactory) {
// This method tests IngestSegmentFirehoseFactory-specific methods.
return;
}
final IngestSegmentFirehoseFactory isfFactory = (IngestSegmentFirehoseFactory) factory;
Assert.assertEquals(TASK.getDataSource(), isfFactory.getDataSource());
if (isfFactory.getDimensions() != null) {
Assert.assertArrayEquals(new String[]{DIM_NAME}, isfFactory.getDimensions().toArray());
}
Assert.assertEquals(Intervals.ETERNITY, isfFactory.getInterval());
if (isfFactory.getMetrics() != null) {
Assert.assertEquals(
ImmutableSet.of(METRIC_LONG_NAME, METRIC_FLOAT_NAME),
ImmutableSet.copyOf(isfFactory.getMetrics())
);
}
}
@Test
public void simpleFirehoseReadingTest() throws IOException
{
Assert.assertEquals(MAX_SHARD_NUMBER.longValue(), SEGMENT_SET.size());
Integer rowcount = 0;
try (final Firehose firehose = factory.connect(rowParser, TMP_DIR)) {
while (firehose.hasMore()) {
InputRow row = firehose.nextRow();
Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray());
Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray());
Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME));
Assert.assertEquals(
METRIC_FLOAT_VALUE,
row.getMetric(METRIC_FLOAT_NAME).floatValue(),
METRIC_FLOAT_VALUE * 0.0001
);
++rowcount;
}
}
Assert.assertEquals((int) MAX_SHARD_NUMBER * MAX_ROWS, (int) rowcount);
}
@Test
public void testTransformSpec() throws IOException
{
Assert.assertEquals(MAX_SHARD_NUMBER.longValue(), SEGMENT_SET.size());
Integer rowcount = 0;
final TransformSpec transformSpec = new TransformSpec(
new SelectorDimFilter(ColumnHolder.TIME_COLUMN_NAME, "1", null),
ImmutableList.of(
new ExpressionTransform(METRIC_FLOAT_NAME, METRIC_FLOAT_NAME + " * 10", ExprMacroTable.nil())
)
);
int skipped = 0;
try (final Firehose firehose =
factory.connect(transformSpec.decorate(rowParser), TMP_DIR)) {
while (firehose.hasMore()) {
InputRow row = firehose.nextRow();
if (row == null) {
skipped++;
continue;
}
Assert.assertArrayEquals(new String[]{DIM_NAME}, row.getDimensions().toArray());
Assert.assertArrayEquals(new String[]{DIM_VALUE}, row.getDimension(DIM_NAME).toArray());
Assert.assertEquals(METRIC_LONG_VALUE.longValue(), row.getMetric(METRIC_LONG_NAME).longValue());
Assert.assertEquals(
METRIC_FLOAT_VALUE * 10,
row.getMetric(METRIC_FLOAT_NAME).floatValue(),
METRIC_FLOAT_VALUE * 0.0001
);
++rowcount;
}
}
Assert.assertEquals(90, skipped);
Assert.assertEquals((int) MAX_ROWS, (int) rowcount);
}
@Test
public void testGetUniqueDimensionsAndMetrics()
{
final int numSegmentsPerPartitionChunk = 5;
final int numPartitionChunksPerTimelineObject = 10;
final int numSegments = numSegmentsPerPartitionChunk * numPartitionChunksPerTimelineObject;
final Interval interval = Intervals.of("2017-01-01/2017-01-02");
final String version = "1";
final List<TimelineObjectHolder<String, DataSegment>> timelineSegments = new ArrayList<>();
for (int i = 0; i < numPartitionChunksPerTimelineObject; i++) {
final List<PartitionChunk<DataSegment>> chunks = new ArrayList<>();
for (int j = 0; j < numSegmentsPerPartitionChunk; j++) {
final List<String> dims = IntStream.range(i, i + numSegmentsPerPartitionChunk)
.mapToObj(suffix -> "dim" + suffix)
.collect(Collectors.toList());
final List<String> metrics = IntStream.range(i, i + numSegmentsPerPartitionChunk)
.mapToObj(suffix -> "met" + suffix)
.collect(Collectors.toList());
final DataSegment segment = new DataSegment(
"ds",
interval,
version,
ImmutableMap.of(),
dims,
metrics,
new NumberedShardSpec(numPartitionChunksPerTimelineObject, i),
1,
1
);
final PartitionChunk<DataSegment> partitionChunk = new NumberedPartitionChunk<>(
i,
numPartitionChunksPerTimelineObject,
segment
);
chunks.add(partitionChunk);
}
final TimelineObjectHolder<String, DataSegment> timelineHolder = new TimelineObjectHolder<>(
interval,
version,
new PartitionHolder<>(chunks)
);
timelineSegments.add(timelineHolder);
}
final String[] expectedDims = new String[]{
"dim9",
"dim10",
"dim11",
"dim12",
"dim13",
"dim8",
"dim7",
"dim6",
"dim5",
"dim4",
"dim3",
"dim2",
"dim1",
"dim0"
};
final String[] expectedMetrics = new String[]{
"met9",
"met10",
"met11",
"met12",
"met13",
"met8",
"met7",
"met6",
"met5",
"met4",
"met3",
"met2",
"met1",
"met0"
};
Assert.assertEquals(
Arrays.asList(expectedDims),
ReingestionTimelineUtils.getUniqueDimensions(timelineSegments, null)
);
Assert.assertEquals(
Arrays.asList(expectedMetrics),
ReingestionTimelineUtils.getUniqueMetrics(timelineSegments)
);
}
private static ServiceEmitter newMockEmitter()
{
return new NoopServiceEmitter();
}
}

View File

@ -1,453 +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.indexing.firehose;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import com.google.common.collect.Iterables;
import org.apache.druid.client.coordinator.CoordinatorClient;
import org.apache.druid.data.input.FiniteFirehoseFactory;
import org.apache.druid.data.input.Firehose;
import org.apache.druid.data.input.InputRow;
import org.apache.druid.data.input.InputSplit;
import org.apache.druid.data.input.MapBasedInputRow;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.MapInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.common.RetryPolicyConfig;
import org.apache.druid.indexing.common.RetryPolicyFactory;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.java.util.common.DateTimes;
import org.apache.druid.java.util.common.FileUtils;
import org.apache.druid.java.util.common.Intervals;
import org.apache.druid.java.util.common.JodaUtils;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.filter.TrueDimFilter;
import org.apache.druid.segment.IndexIO;
import org.apache.druid.segment.IndexMergerV9;
import org.apache.druid.segment.IndexSpec;
import org.apache.druid.segment.handoff.SegmentHandoffNotifierFactory;
import org.apache.druid.segment.incremental.IncrementalIndex;
import org.apache.druid.segment.incremental.IncrementalIndexSchema;
import org.apache.druid.segment.incremental.IndexSizeExceededException;
import org.apache.druid.segment.incremental.OnheapIncrementalIndex;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.timeline.DataSegment;
import org.apache.druid.timeline.SegmentId;
import org.apache.druid.timeline.partition.LinearShardSpec;
import org.easymock.EasyMock;
import org.joda.time.Interval;
import org.junit.After;
import org.junit.Assert;
import org.junit.Test;
import org.junit.runner.RunWith;
import org.junit.runners.Parameterized;
import java.io.File;
import java.io.IOException;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.Collection;
import java.util.HashSet;
import java.util.List;
import java.util.Map;
import java.util.Set;
import java.util.UUID;
import java.util.stream.Collectors;
@RunWith(Parameterized.class)
public class IngestSegmentFirehoseFactoryTimelineTest
{
private static final String DATA_SOURCE = "foo";
private static final String TIME_COLUMN = "t";
private static final String[] DIMENSIONS = new String[]{"d1"};
private static final String[] METRICS = new String[]{"m1"};
// Must decorate the parser, since IngestSegmentFirehoseFactory will undecorate it.
private static final InputRowParser<Map<String, Object>> ROW_PARSER = TransformSpec.NONE.decorate(
new MapInputRowParser(
new JSONParseSpec(
new TimestampSpec(TIME_COLUMN, "auto", null),
new DimensionsSpec(
DimensionsSpec.getDefaultSchemas(Arrays.asList(DIMENSIONS))
),
null,
null,
null
)
)
);
private final IngestSegmentFirehoseFactory factory;
private final File tmpDir;
private final int expectedCount;
private final long expectedSum;
private final int segmentCount;
private static final ObjectMapper MAPPER;
private static final IndexIO INDEX_IO;
private static final IndexMergerV9 INDEX_MERGER_V9;
static {
TestUtils testUtils = new TestUtils();
MAPPER = IngestSegmentFirehoseFactoryTest.setupInjectablesInObjectMapper(testUtils.getTestObjectMapper());
INDEX_IO = testUtils.getTestIndexIO();
INDEX_MERGER_V9 = testUtils.getTestIndexMergerV9();
}
public IngestSegmentFirehoseFactoryTimelineTest(
String name,
IngestSegmentFirehoseFactory factory,
File tmpDir,
int expectedCount,
long expectedSum,
int segmentCount
)
{
this.factory = factory;
this.tmpDir = tmpDir;
this.expectedCount = expectedCount;
this.expectedSum = expectedSum;
this.segmentCount = segmentCount;
}
@Test
public void test() throws Exception
{
// Junit 4.12 doesn't have a good way to run tearDown after multiple tests in a Parameterized
// class run. (Junit 4.13 adds @AfterParam but isn't released yet.) Fake it by just running
// "tests" in series inside one @Test.
testSimple();
testSplit();
}
private void testSimple() throws Exception
{
int count = 0;
long sum = 0;
try (final Firehose firehose = factory.connect(ROW_PARSER, tmpDir)) {
while (firehose.hasMore()) {
final InputRow row = firehose.nextRow();
count++;
sum += row.getMetric(METRICS[0]).longValue();
}
}
Assert.assertEquals("count", expectedCount, count);
Assert.assertEquals("sum", expectedSum, sum);
}
private void testSplit() throws Exception
{
Assert.assertTrue(factory.isSplittable());
final int numSplits = factory.getNumSplits(null);
// We set maxInputSegmentBytesPerSplit to 2 so each segment should become a byte.
Assert.assertEquals(segmentCount, numSplits);
final List<InputSplit<List<WindowedSegmentId>>> splits =
factory.getSplits(null).collect(Collectors.toList());
Assert.assertEquals(numSplits, splits.size());
int count = 0;
long sum = 0;
for (InputSplit<List<WindowedSegmentId>> split : splits) {
final FiniteFirehoseFactory<InputRowParser, List<WindowedSegmentId>> splitFactory =
factory.withSplit(split);
try (final Firehose firehose = splitFactory.connect(ROW_PARSER, tmpDir)) {
while (firehose.hasMore()) {
final InputRow row = firehose.nextRow();
count++;
sum += row.getMetric(METRICS[0]).longValue();
}
}
}
Assert.assertEquals("count", expectedCount, count);
Assert.assertEquals("sum", expectedSum, sum);
}
@After
public void tearDown() throws Exception
{
FileUtils.deleteDirectory(tmpDir);
}
private static TestCase tc(
String intervalString,
int expectedCount,
long expectedSum,
DataSegmentMaker... segmentMakers
)
{
final File tmpDir = FileUtils.createTempDir();
final Set<DataSegment> segments = new HashSet<>();
for (DataSegmentMaker segmentMaker : segmentMakers) {
segments.add(segmentMaker.make(tmpDir));
}
return new TestCase(
tmpDir,
Intervals.of(intervalString),
expectedCount,
expectedSum,
segments
);
}
private static DataSegmentMaker ds(
String intervalString,
String version,
int partitionNum,
InputRow... rows
)
{
return new DataSegmentMaker(Intervals.of(intervalString), version, partitionNum, Arrays.asList(rows));
}
private static InputRow ir(String timeString, long metricValue)
{
return new MapBasedInputRow(
DateTimes.of(timeString).getMillis(),
Arrays.asList(DIMENSIONS),
ImmutableMap.of(
TIME_COLUMN, DateTimes.of(timeString).toString(),
DIMENSIONS[0], "bar",
METRICS[0], metricValue
)
);
}
private static Map<String, Object> persist(File tmpDir, InputRow... rows)
{
final File persistDir = new File(tmpDir, UUID.randomUUID().toString());
final IncrementalIndexSchema schema = new IncrementalIndexSchema.Builder()
.withMinTimestamp(JodaUtils.MIN_INSTANT)
.withDimensionsSpec(ROW_PARSER)
.withMetrics(new LongSumAggregatorFactory(METRICS[0], METRICS[0]))
.build();
final IncrementalIndex index = new OnheapIncrementalIndex.Builder()
.setIndexSchema(schema)
.setMaxRowCount(rows.length)
.build();
for (InputRow row : rows) {
try {
index.add(row);
}
catch (IndexSizeExceededException e) {
throw new RuntimeException(e);
}
}
try {
INDEX_MERGER_V9.persist(index, persistDir, new IndexSpec(), null);
}
catch (IOException e) {
throw new RuntimeException(e);
}
return ImmutableMap.of(
"type", "local",
"path", persistDir.getAbsolutePath()
);
}
@Parameterized.Parameters(name = "{0}")
public static Collection<Object[]> constructorFeeder()
{
final List<TestCase> testCases = ImmutableList.of(
tc(
"2000/2000T02", 3, 7,
ds("2000/2000T01", "v1", 0, ir("2000", 1), ir("2000T00:01", 2)),
ds("2000T01/2000T02", "v1", 0, ir("2000T01", 4))
) /* Adjacent segments */,
tc(
"2000/2000T02", 3, 7,
ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4))
) /* 1H segment overlaid on top of 2H segment */,
tc(
"2000/2000-01-02", 4, 23,
ds("2000/2000-01-02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8), ir("2000T02", 16)),
ds("2000T01/2000T02", "v2", 0, ir("2000T01:01", 4))
) /* 1H segment overlaid on top of 1D segment */,
tc(
"2000/2000T02", 4, 15,
ds("2000/2000T02", "v1", 0, ir("2000", 1), ir("2000T00:01", 2), ir("2000T01", 8)),
ds("2000/2000T02", "v1", 1, ir("2000T01:01", 4))
) /* Segment set with two segments for the same interval */,
tc(
"2000T01/2000T02", 1, 2,
ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4))
) /* Segment wider than desired interval */,
tc(
"2000T02/2000T04", 2, 12,
ds("2000/2000T03", "v1", 0, ir("2000", 1), ir("2000T01", 2), ir("2000T02", 4)),
ds("2000T03/2000T04", "v1", 0, ir("2000T03", 8))
) /* Segment intersecting desired interval */
);
final List<Object[]> constructors = new ArrayList<>();
for (final TestCase testCase : testCases) {
SegmentHandoffNotifierFactory notifierFactory = EasyMock.createNiceMock(SegmentHandoffNotifierFactory.class);
EasyMock.replay(notifierFactory);
final SegmentCacheManagerFactory slf = new SegmentCacheManagerFactory(MAPPER);
final RetryPolicyFactory retryPolicyFactory = new RetryPolicyFactory(new RetryPolicyConfig());
final CoordinatorClient cc = new CoordinatorClient(null, null)
{
@Override
public Collection<DataSegment> fetchUsedSegmentsInDataSourceForIntervals(
String dataSource,
List<Interval> intervals
)
{
// Expect the interval we asked for
if (intervals.equals(ImmutableList.of(testCase.interval))) {
return ImmutableSet.copyOf(testCase.segments);
} else {
throw new IllegalArgumentException("BAD");
}
}
@Override
public DataSegment fetchUsedSegment(String dataSource, String segmentId)
{
return testCase.segments
.stream()
.filter(s -> s.getId().toString().equals(segmentId))
.findAny()
.get(); // throwing if not found is exactly what the real code does
}
};
final IngestSegmentFirehoseFactory factory = new IngestSegmentFirehoseFactory(
DATA_SOURCE,
testCase.interval,
null,
TrueDimFilter.instance(),
Arrays.asList(DIMENSIONS),
Arrays.asList(METRICS),
// Split as much as possible
1L,
INDEX_IO,
cc,
slf,
retryPolicyFactory
);
constructors.add(
new Object[]{
testCase.toString(),
factory,
testCase.tmpDir,
testCase.expectedCount,
testCase.expectedSum,
testCase.segments.size()
}
);
}
return constructors;
}
private static class TestCase
{
final File tmpDir;
final Interval interval;
final int expectedCount;
final long expectedSum;
final Set<DataSegment> segments;
public TestCase(
File tmpDir,
Interval interval,
int expectedCount,
long expectedSum,
Set<DataSegment> segments
)
{
this.tmpDir = tmpDir;
this.interval = interval;
this.expectedCount = expectedCount;
this.expectedSum = expectedSum;
this.segments = segments;
}
@Override
public String toString()
{
final List<SegmentId> segmentIds = new ArrayList<>();
for (DataSegment segment : segments) {
segmentIds.add(segment.getId());
}
return "TestCase{" +
"interval=" + interval +
", expectedCount=" + expectedCount +
", expectedSum=" + expectedSum +
", segments=" + segmentIds +
'}';
}
}
private static class DataSegmentMaker
{
final Interval interval;
final String version;
final int partitionNum;
final List<InputRow> rows;
public DataSegmentMaker(
Interval interval,
String version,
int partitionNum,
List<InputRow> rows
)
{
this.interval = interval;
this.version = version;
this.partitionNum = partitionNum;
this.rows = rows;
}
public DataSegment make(File tmpDir)
{
final Map<String, Object> loadSpec = persist(tmpDir, Iterables.toArray(rows, InputRow.class));
return new DataSegment(
DATA_SOURCE,
interval,
version,
loadSpec,
Arrays.asList(DIMENSIONS),
Arrays.asList(METRICS),
new LinearShardSpec(partitionNum),
-1,
2L
);
}
}
}

View File

@ -37,7 +37,7 @@ import org.apache.druid.indexer.TaskLocation;
import org.apache.druid.indexer.TaskState;
import org.apache.druid.indexer.TaskStatus;
import org.apache.druid.indexing.common.IndexingServiceCondition;
import org.apache.druid.indexing.common.TestRealtimeTask;
import org.apache.druid.indexing.common.TestIndexTask;
import org.apache.druid.indexing.common.TestTasks;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.task.Task;
@ -251,7 +251,7 @@ public class RemoteTaskRunnerTest
{
doSetup();
TestRealtimeTask task1 = new TestRealtimeTask(
TestIndexTask task1 = new TestIndexTask(
"rt1",
new TaskResource("rt1", 1),
"foo",
@ -262,7 +262,7 @@ public class RemoteTaskRunnerTest
Assert.assertTrue(taskAnnounced(task1.getId()));
mockWorkerRunningTask(task1);
TestRealtimeTask task2 = new TestRealtimeTask(
TestIndexTask task2 = new TestIndexTask(
"rt2",
new TaskResource("rt1", 1),
"foo",
@ -271,7 +271,7 @@ public class RemoteTaskRunnerTest
);
remoteTaskRunner.run(task2);
TestRealtimeTask task3 = new TestRealtimeTask(
TestIndexTask task3 = new TestIndexTask(
"rt3",
new TaskResource("rt2", 1),
"foo",
@ -314,7 +314,7 @@ public class RemoteTaskRunnerTest
{
doSetup();
TestRealtimeTask task1 = new TestRealtimeTask(
TestIndexTask task1 = new TestIndexTask(
"rt1",
new TaskResource("rt1", 1),
"foo",
@ -325,7 +325,7 @@ public class RemoteTaskRunnerTest
Assert.assertTrue(taskAnnounced(task1.getId()));
mockWorkerRunningTask(task1);
TestRealtimeTask task2 = new TestRealtimeTask(
TestIndexTask task2 = new TestIndexTask(
"rt2",
new TaskResource("rt2", 3),
"foo",
@ -334,7 +334,7 @@ public class RemoteTaskRunnerTest
);
remoteTaskRunner.run(task2);
TestRealtimeTask task3 = new TestRealtimeTask(
TestIndexTask task3 = new TestIndexTask(
"rt3",
new TaskResource("rt3", 2),
"foo",
@ -406,7 +406,7 @@ public class RemoteTaskRunnerTest
makeRemoteTaskRunner(rtrConfig);
TestRealtimeTask task1 = new TestRealtimeTask(
TestIndexTask task1 = new TestIndexTask(
"first",
new TaskResource("first", 1),
"foo",
@ -417,7 +417,7 @@ public class RemoteTaskRunnerTest
Assert.assertTrue(taskAnnounced(task1.getId()));
mockWorkerRunningTask(task1);
TestRealtimeTask task = new TestRealtimeTask(
TestIndexTask task = new TestIndexTask(
"second",
new TaskResource("task", 2),
"foo",
@ -426,7 +426,7 @@ public class RemoteTaskRunnerTest
);
remoteTaskRunner.run(task);
TestRealtimeTask task2 = new TestRealtimeTask(
TestIndexTask task2 = new TestIndexTask(
"second",
new TaskResource("second", 2),
"foo",
@ -457,7 +457,7 @@ public class RemoteTaskRunnerTest
public void testRunWithTaskComplete() throws Exception
{
doSetup();
TestRealtimeTask task1 = new TestRealtimeTask(
TestIndexTask task1 = new TestIndexTask(
"testTask",
new TaskResource("testTask", 2),
"foo",
@ -817,11 +817,11 @@ public class RemoteTaskRunnerTest
makeRemoteTaskRunner(rtrConfig);
TestRealtimeTask task1 = new TestRealtimeTask(
"realtime1",
new TaskResource("realtime1", 1),
TestIndexTask task1 = new TestIndexTask(
"test_index1",
new TaskResource("test_index1", 1),
"foo",
TaskStatus.success("realtime1"),
TaskStatus.success("test_index1"),
jsonMapper
);
Future<TaskStatus> taskFuture1 = remoteTaskRunner.run(task1);
@ -835,11 +835,11 @@ public class RemoteTaskRunnerTest
remoteTaskRunner.findWorkerRunningTask(task1.getId()).getContinuouslyFailedTasksCount()
);
TestRealtimeTask task2 = new TestRealtimeTask(
"realtime2",
new TaskResource("realtime2", 1),
TestIndexTask task2 = new TestIndexTask(
"test_index2",
new TaskResource("test_index2", 1),
"foo",
TaskStatus.running("realtime2"),
TaskStatus.running("test_index2"),
jsonMapper
);
Future<TaskStatus> taskFuture2 = remoteTaskRunner.run(task2);
@ -870,11 +870,11 @@ public class RemoteTaskRunnerTest
remoteTaskRunner.findWorkerRunningTask(task2.getId()).getContinuouslyFailedTasksCount()
);
TestRealtimeTask task3 = new TestRealtimeTask(
"realtime3",
new TaskResource("realtime3", 1),
TestIndexTask task3 = new TestIndexTask(
"test_index3",
new TaskResource("test_index3", 1),
"foo",
TaskStatus.running("realtime3"),
TaskStatus.running("test_index3"),
jsonMapper
);
Future<TaskStatus> taskFuture3 = remoteTaskRunner.run(task3);
@ -909,7 +909,7 @@ public class RemoteTaskRunnerTest
for (int i = 1; i < 13; i++) {
String taskId = StringUtils.format("rt-%d", i);
TestRealtimeTask task = new TestRealtimeTask(
TestIndexTask task = new TestIndexTask(
taskId,
new TaskResource(taskId, 1),
"foo",
@ -964,7 +964,7 @@ public class RemoteTaskRunnerTest
for (int i = 1; i < 13; i++) {
String taskId = StringUtils.format("rt-%d", i);
TestRealtimeTask task = new TestRealtimeTask(
TestIndexTask task = new TestIndexTask(
taskId,
new TaskResource(taskId, 1),
"foo",
@ -1009,14 +1009,14 @@ public class RemoteTaskRunnerTest
makeRemoteTaskRunner(rtrConfig);
TestRealtimeTask task1 = new TestRealtimeTask(
"realtime1", new TaskResource("realtime1", 1), "foo", TaskStatus.success("realtime1"), jsonMapper
TestIndexTask task1 = new TestIndexTask(
"test_index1", new TaskResource("test_index1", 1), "foo", TaskStatus.success("test_index1"), jsonMapper
);
TestRealtimeTask task2 = new TestRealtimeTask(
"realtime2", new TaskResource("realtime2", 1), "foo", TaskStatus.success("realtime2"), jsonMapper
TestIndexTask task2 = new TestIndexTask(
"test_index2", new TaskResource("test_index2", 1), "foo", TaskStatus.success("test_index2"), jsonMapper
);
TestRealtimeTask task3 = new TestRealtimeTask(
"realtime3", new TaskResource("realtime3", 1), "foo", TaskStatus.success("realtime3"), jsonMapper
TestIndexTask task3 = new TestIndexTask(
"test_index3", new TaskResource("test_index3", 1), "foo", TaskStatus.success("test_index3"), jsonMapper
);
Future<TaskStatus> taskFuture1 = remoteTaskRunner.run(task1);

View File

@ -26,7 +26,6 @@ import com.google.common.collect.ImmutableSet;
import net.thisptr.jackson.jq.internal.misc.Lists;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.client.indexing.SamplerResponse.SamplerResponseRow;
import org.apache.druid.data.input.FirehoseFactoryToInputSourceAdaptor;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.InputRowSchema;
import org.apache.druid.data.input.InputSource;
@ -39,11 +38,9 @@ import org.apache.druid.data.input.impl.InlineInputSource;
import org.apache.druid.data.input.impl.InputRowParser;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.JsonInputFormat;
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.guice.NestedDataModule;
import org.apache.druid.indexing.seekablestream.RecordSupplierInputSource;
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
import org.apache.druid.indexing.seekablestream.common.OrderedSequenceNumber;
@ -60,13 +57,9 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
import org.apache.druid.query.expression.TestExprMacroTable;
import org.apache.druid.query.filter.SelectorDimFilter;
import org.apache.druid.segment.column.ColumnType;
import org.apache.druid.segment.column.RowSignature;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.GranularitySpec;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.apache.druid.segment.nested.StructuredData;
import org.apache.druid.segment.realtime.firehose.InlineFirehoseFactory;
import org.apache.druid.segment.transform.ExpressionTransform;
import org.apache.druid.segment.transform.TransformSpec;
import org.apache.druid.testing.InitializedNullHandlingTest;
@ -132,9 +125,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
@Parameterized.Parameters(name = "parserType = {0}, useInputFormatApi={1}")
public static Iterable<Object[]> constructorFeeder()
{
NestedDataModule.registerHandlersAndSerde();
OBJECT_MAPPER.registerModules(new SamplerModule().getJacksonModules());
OBJECT_MAPPER.registerModules(NestedDataModule.getJacksonModulesList());
return ImmutableList.of(
new Object[]{ParserType.STR_JSON, false},
new Object[]{ParserType.STR_JSON, true},
@ -182,14 +173,12 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
@Test
public void testNoDataSchema()
{
final InputSource inputSource = createInputSource(getTestRows(), null);
final InputSource inputSource = createInputSource(getTestRows());
final SamplerResponse response = inputSourceSampler.sample(inputSource, createInputFormat(), null, null);
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(0, response.getNumRowsIndexed());
Assert.assertEquals(6, response.getData().size());
Assert.assertEquals(ImmutableList.of(), response.getLogicalDimensions());
Assert.assertEquals(RowSignature.builder().addTimeColumn().build(), response.getLogicalSegmentSchema());
List<SamplerResponseRow> data = response.getData();
@ -252,7 +241,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
@Test
public void testNoDataSchemaNumRows()
{
final InputSource inputSource = createInputSource(getTestRows(), null);
final InputSource inputSource = createInputSource(getTestRows());
final SamplerResponse response = inputSourceSampler.sample(
inputSource,
createInputFormat(),
@ -263,8 +252,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(3, response.getNumRowsRead());
Assert.assertEquals(0, response.getNumRowsIndexed());
Assert.assertEquals(3, response.getData().size());
Assert.assertEquals(ImmutableList.of(), response.getLogicalDimensions());
Assert.assertEquals(RowSignature.builder().addTimeColumn().build(), response.getLogicalSegmentSchema());
List<SamplerResponseRow> data = response.getData();
@ -303,7 +290,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
final TimestampSpec timestampSpec = new TimestampSpec(null, null, DateTimes.of("1970"));
final DimensionsSpec dimensionsSpec = new DimensionsSpec(null);
final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -311,40 +298,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(6, response.getNumRowsIndexed());
Assert.assertEquals(6, response.getData().size());
Assert.assertEquals(
parserType == ParserType.STR_JSON
? ImmutableList.of(
new StringDimensionSchema("t"),
new StringDimensionSchema("dim1"),
new StringDimensionSchema("met1"),
new StringDimensionSchema("dim2")
)
: ImmutableList.of(
new StringDimensionSchema("t"),
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2"),
new StringDimensionSchema("met1")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
parserType == ParserType.STR_JSON
? RowSignature.builder()
.addTimeColumn()
.add("t", ColumnType.STRING)
.add("dim1", ColumnType.STRING)
.add("met1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.build()
: RowSignature.builder()
.addTimeColumn()
.add("t", ColumnType.STRING)
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.STRING)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -446,7 +399,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
final TimestampSpec timestampSpec = new TimestampSpec("t", null, null);
final DimensionsSpec dimensionsSpec = new DimensionsSpec(null);
final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -454,36 +407,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(6, response.getData().size());
Assert.assertEquals(
parserType == ParserType.STR_JSON
? ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("met1"),
new StringDimensionSchema("dim2")
)
: ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2"),
new StringDimensionSchema("met1")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
parserType == ParserType.STR_JSON
? RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("met1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.build()
: RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.STRING)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -568,137 +491,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
);
}
@Test
public void testWithTimestampSpecNestedDiscovery() throws IOException
{
final TimestampSpec timestampSpec = new TimestampSpec("t", null, null);
final DimensionsSpec dimensionsSpec = DimensionsSpec.builder()
.useSchemaDiscovery(true)
.build();
final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(6, response.getData().size());
Assert.assertEquals(
parserType == ParserType.STR_JSON
? ImmutableList.of(
new StringDimensionSchema("dim1"),
new LongDimensionSchema("met1"),
new StringDimensionSchema("dim2")
)
: ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2"),
new StringDimensionSchema("met1")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
parserType == ParserType.STR_JSON
? RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.add("dim2", ColumnType.STRING)
.build()
: RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.STRING)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
getRawColumns().get(0),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1555934400000L)
.put("dim2", StructuredData.wrap(null))
.put("dim1", StructuredData.wrap("foo"))
.put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 1L : "1"))
.build(),
null,
null
),
data.get(0)
);
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
getRawColumns().get(1),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1555934400000L)
.put("dim2", StructuredData.wrap(null))
.put("dim1", StructuredData.wrap("foo"))
.put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 2L : "2"))
.build(),
null,
null
),
data.get(1)
);
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
getRawColumns().get(2),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1555934460000L)
.put("dim2", StructuredData.wrap(null))
.put("dim1", StructuredData.wrap("foo"))
.put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 3L : "3"))
.build(),
null,
null
),
data.get(2)
);
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
getRawColumns().get(3),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1555934400000L)
.put("dim2", StructuredData.wrap(null))
.put("dim1", StructuredData.wrap("foo2"))
.put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 4L : "4"))
.build(),
null,
null
),
data.get(3)
);
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
getRawColumns().get(4),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1555934400000L)
.put("dim2", StructuredData.wrap("bar"))
.put("dim1", StructuredData.wrap("foo"))
.put("met1", StructuredData.wrap(parserType == ParserType.STR_JSON ? 5L : "5"))
.build(),
null,
null
),
data.get(4)
);
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
getRawColumns().get(5),
null,
true,
getUnparseableTimestampString()
),
data.get(5)
);
}
@Test
public void testWithDimensionSpec() throws IOException
{
@ -707,7 +499,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
ImmutableList.of(StringDimensionSchema.create("dim1"), StringDimensionSchema.create("met1"))
);
final DataSchema dataSchema = createDataSchema(timestampSpec, dimensionsSpec, null, null, null);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -715,21 +507,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(6, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("met1")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("met1", ColumnType.STRING)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -828,7 +605,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
null
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -836,22 +613,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(6, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -955,7 +716,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
null
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -963,22 +724,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(4, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -1054,7 +799,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
null
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -1062,20 +807,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(3, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -1139,7 +870,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
transformSpec
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -1147,22 +878,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(4, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -1244,7 +959,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
transformSpec
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -1252,20 +967,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(6, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(3, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1PlusBar")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1PlusBar", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -1326,7 +1027,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
transformSpec
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -1334,22 +1035,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(5, response.getNumRowsRead());
Assert.assertEquals(4, response.getNumRowsIndexed());
Assert.assertEquals(3, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -1423,14 +1108,13 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
//
Map<String, Object> rawColumns4ParseExceptionRow = ImmutableMap.of("t", "2019-04-22T12:00",
"dim1", "foo2",
"met1", "invalidNumber"
);
"met1", "invalidNumber");
final List<String> inputTestRows = Lists.newArrayList(getTestRows());
inputTestRows.add(ParserType.STR_CSV.equals(parserType) ?
"2019-04-22T12:00,foo2,,invalidNumber" :
OBJECT_MAPPER.writeValueAsString(rawColumns4ParseExceptionRow));
final InputSource inputSource = createInputSource(inputTestRows, dataSchema);
final InputSource inputSource = createInputSource(inputTestRows);
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(inputSource, inputFormat, dataSchema, null);
@ -1438,20 +1122,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(7, response.getNumRowsRead());
Assert.assertEquals(5, response.getNumRowsIndexed());
Assert.assertEquals(4, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1PlusBar")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1PlusBar", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
@ -1495,8 +1165,8 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
// the last row has parse exception when indexing, check if rawColumns and exception message match the expected
//
String indexParseExceptioMessage = ParserType.STR_CSV.equals(parserType)
? "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, dim2=null, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]"
: "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]";
? "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, dim2=null, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]"
: "Found unparseable columns in row: [SamplerInputRow{row=TransformedInputRow{row=MapBasedInputRow{timestamp=2019-04-22T12:00:00.000Z, event={t=2019-04-22T12:00, dim1=foo2, met1=invalidNumber}, dimensions=[dim1PlusBar]}}}], exceptions: [Unable to parse value[invalidNumber] for field[met1]]";
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
rawColumns4ParseExceptionRow,
@ -1509,13 +1179,15 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
}
/**
*
* This case tests sampling for multiple json lines in one text block
* Currently only RecordSupplierInputSource supports this kind of input, see https://github.com/apache/druid/pull/10383 for more information
* <p>
*
* This test combines illegal json block and legal json block together to verify:
* 1. all lines in the illegal json block should not be parsed
* 2. the illegal json block should not affect the processing of the 2nd record
* 3. all lines in legal json block should be parsed successfully
*
*/
@Test
public void testMultipleJsonStringInOneBlock() throws IOException
@ -1571,20 +1243,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(illegalRows + legalRows, response.getNumRowsRead());
Assert.assertEquals(legalRows, response.getNumRowsIndexed());
Assert.assertEquals(illegalRows + 2, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1PlusBar")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1PlusBar", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
List<SamplerResponseRow> data = response.getData();
List<Map<String, Object>> rawColumnList = this.getRawColumns();
@ -1594,11 +1252,8 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
// first n rows are related to the first json block which fails to parse
//
String parseExceptionMessage;
if (useInputFormatApi) {
parseExceptionMessage = "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}";
} else {
parseExceptionMessage = "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}";
}
parseExceptionMessage = "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}";
for (; index < illegalRows; index++) {
assertEqualsSamplerResponseRow(
new SamplerResponseRow(
@ -1691,7 +1346,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
null
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(
@ -1704,33 +1359,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(4, response.getNumRowsRead());
Assert.assertEquals(4, response.getNumRowsIndexed());
Assert.assertEquals(2, response.getData().size());
Assert.assertEquals(
parserType == ParserType.STR_JSON
? ImmutableList.of(
new StringDimensionSchema("dim1")
)
: ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
parserType == ParserType.STR_JSON
? RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build()
: RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
}
@ -1753,7 +1381,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
null
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(
@ -1766,32 +1394,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(4, response.getNumRowsRead());
Assert.assertEquals(4, response.getNumRowsIndexed());
Assert.assertEquals(2, response.getData().size());
Assert.assertEquals(
parserType == ParserType.STR_JSON
? ImmutableList.of(
new StringDimensionSchema("dim1")
)
: ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
parserType == ParserType.STR_JSON
? RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build()
: RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
}
@Test
@ -1813,7 +1415,7 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
granularitySpec,
null
);
final InputSource inputSource = createInputSource(getTestRows(), dataSchema);
final InputSource inputSource = createInputSource(getTestRows());
final InputFormat inputFormat = createInputFormat();
SamplerResponse response = inputSourceSampler.sample(
@ -1826,22 +1428,6 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
Assert.assertEquals(4, response.getNumRowsRead());
Assert.assertEquals(4, response.getNumRowsIndexed());
Assert.assertEquals(2, response.getData().size());
Assert.assertEquals(
ImmutableList.of(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim2")
),
response.getLogicalDimensions()
);
Assert.assertEquals(
RowSignature.builder()
.addTimeColumn()
.add("dim1", ColumnType.STRING)
.add("dim2", ColumnType.STRING)
.add("met1", ColumnType.LONG)
.build(),
response.getLogicalSegmentSchema()
);
}
private List<String> getTestRows()
@ -1940,42 +1526,22 @@ public class InputSourceSamplerTest extends InitializedNullHandlingTest
return OBJECT_MAPPER.readValue(OBJECT_MAPPER.writeValueAsBytes(parser), Map.class);
}
private InputSource createInputSource(List<String> rows, DataSchema dataSchema)
private InputSource createInputSource(List<String> rows)
{
final String data = String.join("\n", rows);
if (useInputFormatApi) {
return new InlineInputSource(data);
} else {
return new FirehoseFactoryToInputSourceAdaptor(
new InlineFirehoseFactory(data),
createInputRowParser(
dataSchema == null ? new TimestampSpec(null, null, null) : dataSchema.getTimestampSpec(),
dataSchema == null ? new DimensionsSpec(null) : dataSchema.getDimensionsSpec()
)
);
}
return new InlineInputSource(data);
}
private String getUnparseableTimestampString()
{
if (useInputFormatApi) {
return ParserType.STR_CSV.equals(parserType)
? "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6} (Line: 6)"
: "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6} (Line: 6)";
} else {
return ParserType.STR_CSV.equals(parserType)
? "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6}"
: "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6}";
}
return ParserType.STR_CSV.equals(parserType)
? "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, dim2=null, met1=6} (Line: 6)"
: "Timestamp[bad_timestamp] is unparseable! Event: {t=bad_timestamp, dim1=foo, met1=6} (Line: 6)";
}
private String unparseableTimestampErrorString(Map<String, Object> rawColumns, int line)
{
if (useInputFormatApi) {
return StringUtils.format("Timestamp[null] is unparseable! Event: %s (Line: %d)", rawColumns, line);
} else {
return StringUtils.format("Timestamp[null] is unparseable! Event: %s", rawColumns);
}
return StringUtils.format("Timestamp[null] is unparseable! Event: %s (Line: %d)", rawColumns, line);
}
@Nullable

View File

@ -0,0 +1,354 @@
/*
* 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.indexing.seekablestream;
import com.fasterxml.jackson.databind.ObjectMapper;
import com.google.common.collect.ImmutableList;
import com.google.common.collect.ImmutableMap;
import com.google.common.collect.ImmutableSet;
import org.apache.druid.client.indexing.SamplerResponse;
import org.apache.druid.common.config.NullHandling;
import org.apache.druid.data.input.InputFormat;
import org.apache.druid.data.input.impl.ByteEntity;
import org.apache.druid.data.input.impl.DimensionsSpec;
import org.apache.druid.data.input.impl.FloatDimensionSchema;
import org.apache.druid.data.input.impl.JSONParseSpec;
import org.apache.druid.data.input.impl.LongDimensionSchema;
import org.apache.druid.data.input.impl.StringDimensionSchema;
import org.apache.druid.data.input.impl.StringInputRowParser;
import org.apache.druid.data.input.impl.TimestampSpec;
import org.apache.druid.indexing.overlord.sampler.InputSourceSampler;
import org.apache.druid.indexing.overlord.sampler.SamplerConfig;
import org.apache.druid.indexing.overlord.sampler.SamplerTestUtils;
import org.apache.druid.indexing.seekablestream.common.OrderedPartitionableRecord;
import org.apache.druid.indexing.seekablestream.common.RecordSupplier;
import org.apache.druid.indexing.seekablestream.common.StreamPartition;
import org.apache.druid.indexing.seekablestream.supervisor.IdleConfig;
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorIOConfig;
import org.apache.druid.indexing.seekablestream.supervisor.SeekableStreamSupervisorSpec;
import org.apache.druid.indexing.seekablestream.supervisor.autoscaler.AutoScalerConfig;
import org.apache.druid.jackson.DefaultObjectMapper;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.java.util.common.granularity.Granularities;
import org.apache.druid.java.util.common.parsers.JSONPathSpec;
import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.query.aggregation.CountAggregatorFactory;
import org.apache.druid.query.aggregation.DoubleSumAggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.granularity.UniformGranularitySpec;
import org.easymock.EasyMock;
import org.easymock.EasyMockSupport;
import org.joda.time.DateTime;
import org.joda.time.Period;
import org.junit.Assert;
import org.junit.Test;
import javax.annotation.Nullable;
import java.util.Arrays;
import java.util.Collections;
import java.util.Iterator;
import java.util.List;
import java.util.Map;
public class SeekableStreamSamplerSpecTest extends EasyMockSupport
{
private static final ObjectMapper OBJECT_MAPPER = new DefaultObjectMapper();
private static final String STREAM = "sampling";
private static final String SHARD_ID = "1";
private final SeekableStreamSupervisorSpec supervisorSpec = mock(SeekableStreamSupervisorSpec.class);
static {
NullHandling.initializeForTests();
}
private final RecordSupplier recordSupplier = mock(RecordSupplier.class);
private static List<OrderedPartitionableRecord<String, String, ByteEntity>> generateRecords(String stream)
{
return ImmutableList.of(
new OrderedPartitionableRecord<>(stream, "1", "0", jb("2008", "a", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "1", jb("2009", "b", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(stream, "1", "2", jb("2010", "c", "y", "10", "20.0", "1.0")),
new OrderedPartitionableRecord<>(
stream,
"1",
"5",
jb("246140482-04-24T15:36:27.903Z", "x", "z", "10", "20.0", "1.0")
),
new OrderedPartitionableRecord<>(
stream,
"1",
"6",
Collections.singletonList(new ByteEntity(StringUtils.toUtf8("unparseable")))
),
new OrderedPartitionableRecord<>(stream, "1", "8", Collections.singletonList(new ByteEntity(StringUtils.toUtf8("{}"))))
);
}
@Test(timeout = 10_000L)
public void testSampleWithInputRowParser() throws Exception
{
final DataSchema dataSchema = new DataSchema(
"test_ds",
OBJECT_MAPPER.convertValue(
new StringInputRowParser(
new JSONParseSpec(
new TimestampSpec("timestamp", "iso", null),
new DimensionsSpec(
Arrays.asList(
new StringDimensionSchema("dim1"),
new StringDimensionSchema("dim1t"),
new StringDimensionSchema("dim2"),
new LongDimensionSchema("dimLong"),
new FloatDimensionSchema("dimFloat")
)
),
new JSONPathSpec(true, ImmutableList.of()),
ImmutableMap.of(),
false
)
),
Map.class
),
new AggregatorFactory[]{
new DoubleSumAggregatorFactory("met1sum", "met1"),
new CountAggregatorFactory("rows")
},
new UniformGranularitySpec(Granularities.DAY, Granularities.NONE, null),
null,
OBJECT_MAPPER
);
final SeekableStreamSupervisorIOConfig supervisorIOConfig = new TestableSeekableStreamSupervisorIOConfig(
STREAM,
null,
null,
null,
null,
null,
null,
true,
null,
null,
null,
null,
null,
null
);
EasyMock.expect(recordSupplier.getPartitionIds(STREAM)).andReturn(ImmutableSet.of(SHARD_ID)).once();
EasyMock.expect(supervisorSpec.getDataSchema()).andReturn(dataSchema).once();
EasyMock.expect(supervisorSpec.getIoConfig()).andReturn(supervisorIOConfig).once();
EasyMock.expect(supervisorSpec.getTuningConfig()).andReturn(null).once();
recordSupplier.assign(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID)));
EasyMock.expectLastCall().once();
recordSupplier.seekToEarliest(ImmutableSet.of(StreamPartition.of(STREAM, SHARD_ID)));
EasyMock.expectLastCall().once();
EasyMock.expect(recordSupplier.poll(EasyMock.anyLong())).andReturn(generateRecords(STREAM)).once();
recordSupplier.close();
EasyMock.expectLastCall().once();
replayAll();
SeekableStreamSamplerSpec samplerSpec = new TestableSeekableStreamSamplerSpec(
supervisorSpec,
new SamplerConfig(5, null, null, null),
new InputSourceSampler(new DefaultObjectMapper())
);
SamplerResponse response = samplerSpec.sample();
verifyAll();
Assert.assertEquals(5, response.getNumRowsRead());
Assert.assertEquals(3, response.getNumRowsIndexed());
Assert.assertEquals(5, response.getData().size());
Iterator<SamplerResponse.SamplerResponseRow> it = response.getData().iterator();
Assert.assertEquals(new SamplerResponse.SamplerResponseRow(
ImmutableMap.<String, Object>builder()
.put("timestamp", "2008")
.put("dim1", "a")
.put("dim2", "y")
.put("dimLong", "10")
.put("dimFloat", "20.0")
.put("met1", "1.0")
.build(),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1199145600000L)
.put("dim1", "a")
.put("dim1t", null)
.put("dim2", "y")
.put("dimLong", 10L)
.put("dimFloat", 20.0F)
.put("rows", 1L)
.put("met1sum", 1.0)
.build(),
null,
null
), it.next());
Assert.assertEquals(new SamplerResponse.SamplerResponseRow(
ImmutableMap.<String, Object>builder()
.put("timestamp", "2009")
.put("dim1", "b")
.put("dim2", "y")
.put("dimLong", "10")
.put("dimFloat", "20.0")
.put("met1", "1.0")
.build(),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1230768000000L)
.put("dim1", "b")
.put("dim1t", null)
.put("dim2", "y")
.put("dimLong", 10L)
.put("dimFloat", 20.0F)
.put("rows", 1L)
.put("met1sum", 1.0)
.build(),
null,
null
), it.next());
Assert.assertEquals(new SamplerResponse.SamplerResponseRow(
ImmutableMap.<String, Object>builder()
.put("timestamp", "2010")
.put("dim1", "c")
.put("dim2", "y")
.put("dimLong", "10")
.put("dimFloat", "20.0")
.put("met1", "1.0")
.build(),
new SamplerTestUtils.MapAllowingNullValuesBuilder<String, Object>()
.put("__time", 1262304000000L)
.put("dim1", "c")
.put("dim1t", null)
.put("dim2", "y")
.put("dimLong", 10L)
.put("dimFloat", 20.0F)
.put("rows", 1L)
.put("met1sum", 1.0)
.build(),
null,
null
), it.next());
Assert.assertEquals(new SamplerResponse.SamplerResponseRow(
ImmutableMap.<String, Object>builder()
.put("timestamp", "246140482-04-24T15:36:27.903Z")
.put("dim1", "x")
.put("dim2", "z")
.put("dimLong", "10")
.put("dimFloat", "20.0")
.put("met1", "1.0")
.build(),
null,
true,
"Encountered row with timestamp[246140482-04-24T15:36:27.903Z] that cannot be represented as a long: [{timestamp=246140482-04-24T15:36:27.903Z, dim1=x, dim2=z, dimLong=10, dimFloat=20.0, met1=1.0}]"
), it.next());
Assert.assertEquals(new SamplerResponse.SamplerResponseRow(
null,
null,
true,
"Unable to parse row [unparseable]"
), it.next());
Assert.assertFalse(it.hasNext());
}
private static List<ByteEntity> jb(String ts, String dim1, String dim2, String dimLong, String dimFloat, String met1)
{
try {
return Collections.singletonList(new ByteEntity(new ObjectMapper().writeValueAsBytes(
ImmutableMap.builder()
.put("timestamp", ts)
.put("dim1", dim1)
.put("dim2", dim2)
.put("dimLong", dimLong)
.put("dimFloat", dimFloat)
.put("met1", met1)
.build()
)));
}
catch (Exception e) {
throw new RuntimeException(e);
}
}
private class TestableSeekableStreamSamplerSpec extends SeekableStreamSamplerSpec
{
private TestableSeekableStreamSamplerSpec(
SeekableStreamSupervisorSpec ingestionSpec,
SamplerConfig samplerConfig,
InputSourceSampler inputSourceSampler
)
{
super(ingestionSpec, samplerConfig, inputSourceSampler);
}
@Override
protected RecordSupplier createRecordSupplier()
{
return recordSupplier;
}
}
private static class TestableSeekableStreamSupervisorIOConfig extends SeekableStreamSupervisorIOConfig
{
private TestableSeekableStreamSupervisorIOConfig(
String stream,
@Nullable InputFormat inputFormat,
Integer replicas,
Integer taskCount,
Period taskDuration,
Period startDelay,
Period period,
Boolean useEarliestSequenceNumber,
Period completionTimeout,
Period lateMessageRejectionPeriod,
Period earlyMessageRejectionPeriod,
@Nullable AutoScalerConfig autoScalerConfig,
DateTime lateMessageRejectionStartDateTime,
@Nullable IdleConfig idleConfig
)
{
super(
stream,
inputFormat,
replicas,
taskCount,
taskDuration,
startDelay,
period,
useEarliestSequenceNumber,
completionTimeout,
lateMessageRejectionPeriod,
earlyMessageRejectionPeriod,
autoScalerConfig,
lateMessageRejectionStartDateTime,
idleConfig
);
}
}
}

View File

@ -31,12 +31,10 @@ import org.apache.druid.query.aggregation.AggregatorFactory;
import org.apache.druid.segment.indexing.DataSchema;
import org.apache.druid.segment.indexing.RealtimeIOConfig;
import org.apache.druid.segment.realtime.FireDepartment;
import org.apache.druid.segment.realtime.firehose.LocalFirehoseFactory;
import org.easymock.EasyMock;
import org.junit.Assert;
import org.junit.Test;
import java.io.File;
public class TaskAnnouncementTest
{
private final ObjectMapper jsonMapper;
@ -50,15 +48,13 @@ public class TaskAnnouncementTest
@Test
public void testBackwardsCompatibleSerde() throws Exception
{
final RealtimeIOConfig realtimeIOConfig = EasyMock.createNiceMock(RealtimeIOConfig.class);
final Task task = new RealtimeIndexTask(
"theid",
new TaskResource("rofl", 2),
new FireDepartment(
new DataSchema("foo", null, new AggregatorFactory[0], null, null, new DefaultObjectMapper()),
new RealtimeIOConfig(
new LocalFirehoseFactory(new File("lol"), "rofl", null),
(schema, config, metrics) -> null
),
realtimeIOConfig,
null
),
null

View File

@ -34,7 +34,7 @@ import org.apache.druid.indexing.common.IndexingServiceCondition;
import org.apache.druid.indexing.common.SegmentCacheManagerFactory;
import org.apache.druid.indexing.common.TaskStorageDirTracker;
import org.apache.druid.indexing.common.TaskToolboxFactory;
import org.apache.druid.indexing.common.TestRealtimeTask;
import org.apache.druid.indexing.common.TestIndexTask;
import org.apache.druid.indexing.common.TestTasks;
import org.apache.druid.indexing.common.TestUtils;
import org.apache.druid.indexing.common.actions.TaskActionClient;
@ -146,7 +146,7 @@ public class WorkerTaskMonitorTest
// Start a task monitor
workerTaskMonitor = createTaskMonitor();
TestTasks.registerSubtypes(jsonMapper);
jsonMapper.registerSubtypes(new NamedType(TestRealtimeTask.class, "test_realtime"));
jsonMapper.registerSubtypes(new NamedType(TestIndexTask.class, "test_index"));
workerTaskMonitor.start();
task = TestTasks.immediateSuccess("test");

View File

@ -1,101 +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.testsEx.indexer;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.testsEx.categories.BatchIndex;
import org.apache.druid.testsEx.config.DruidTestRunner;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
import java.io.Closeable;
import java.util.function.Function;
@RunWith(DruidTestRunner.class)
@Category(BatchIndex.class)
public class ITCombiningFirehoseFactoryIndexTest extends AbstractITBatchIndexTest
{
private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json";
private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
private static final String INDEX_DATASOURCE = "wikipedia_index_test";
private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_firehose_index_task.json";
private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json";
private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test";
@Test
public void testIndexData() throws Exception
{
try (
final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
) {
final Function<String, String> combiningFirehoseSpecTransform = spec -> {
try {
return StringUtils.replace(
spec,
"%%COMBINING_DATASOURCE%%",
INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()
);
}
catch (Exception e) {
throw new RuntimeException(e);
}
};
final Function<String, String> transform = spec -> {
try {
return StringUtils.replace(
spec,
"%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
jsonMapper.writeValueAsString("0")
);
}
catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
};
doIndexTest(
INDEX_DATASOURCE,
INDEX_TASK,
transform,
INDEX_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
doIndexTest(
COMBINING_INDEX_DATASOURCE,
COMBINING_INDEX_TASK,
combiningFirehoseSpecTransform,
COMBINING_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
}
}
}

View File

@ -42,7 +42,7 @@ public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchInde
private static final String INDEX_DATASOURCE = "wikipedia_index_test";
private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_input_source_index_parallel_task.json";
private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json";
private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_input_source_index_queries.json";
private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test";
@Test

View File

@ -22,7 +22,6 @@ package org.apache.druid.testsEx.indexer;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.testsEx.categories.BatchIndex;
import org.apache.druid.testsEx.config.DruidTestRunner;
import org.junit.Ignore;
import org.junit.Test;
import org.junit.experimental.categories.Category;
import org.junit.runner.RunWith;
@ -34,12 +33,10 @@ import java.io.IOException;
@Category(BatchIndex.class)
public class ITTransformTest extends AbstractITBatchIndexTest
{
private static final String INDEX_TASK_WITH_FIREHOSE = "/indexer/wikipedia_index_task_with_transform.json";
private static final String INDEX_TASK_WITH_INPUT_SOURCE = "/indexer/wikipedia_index_task_with_inputsource_transform.json";
private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries_with_transform.json";
private static final String INDEX_DATASOURCE = "wikipedia_index_test";
private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task_with_transforms.json";
private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json";
private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries_with_transforms.json";
private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test";
@ -71,55 +68,4 @@ public class ITTransformTest extends AbstractITBatchIndexTest
);
}
}
@Test
@Ignore
public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOException
{
// TODO: re-instate this test when https://github.com/apache/druid/issues/9591 is fixed
// Move the re-index step into testIndexAndReIndexWithTransformSpec for faster tests!
final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData";
try (
final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix())
) {
doIndexTest(
INDEX_DATASOURCE,
INDEX_TASK_WITH_INPUT_SOURCE,
INDEX_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
doReindexTest(
INDEX_DATASOURCE,
reindexDatasource,
REINDEX_TASK,
REINDEX_QUERIES_RESOURCE,
new Pair<>(false, false)
);
}
}
@Test
@Ignore
public void testIndexWithFirehoseAndTransforms() throws IOException
{
// TODO: re-instate this test when https://github.com/apache/druid/issues/9589 is fixed
final String indexDatasource = INDEX_DATASOURCE + "-firehose";
try (
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
) {
doIndexTest(
indexDatasource,
INDEX_TASK_WITH_FIREHOSE,
INDEX_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
}
}
}

View File

@ -1,95 +0,0 @@
{
"type": "index",
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
},
{
"name": "thetaSketch",
"type": "thetaSketch",
"fieldName": "user"
},
{
"name": "quantilesDoublesSketch",
"type": "quantilesDoublesSketch",
"fieldName": "delta"
},
{
"name": "HLLSketchBuild",
"type": "HLLSketchBuild",
"fieldName": "user"
}
],
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "combining",
"delegates": [
{
"type": "local",
"baseDir": "/resources/indexer",
"filter": "wikipedia_combining_index_data.json"
},
{
"type": "ingestSegment",
"dataSource": "%%COMBINING_DATASOURCE%%",
"interval": "2013-08-31/2013-09-02"
}
]
}
},
"tuningConfig": {
"type": "index",
"maxRowsPerSegment": 3
}
}
}

View File

@ -3,6 +3,26 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp",
"format": "auto"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
},
"metricsSpec": [
{
"type": "count",
@ -43,39 +63,22 @@
"segmentGranularity": "DAY",
"queryGranularity": "DAY",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
}
"files": [
"/resources/data/batch_index/json/wikipedia_index_data1.json",
"/resources/data/batch_index/json/wikipedia_index_data2.json",
"/resources/data/batch_index/json/wikipedia_index_data3.json"
]
},
"inputFormat": {
"type": "json"
},
"appendToExisting": false
},
"tuningConfig": {
"type": "index",

View File

@ -3,6 +3,14 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
{"type": "string", "name": "language", "createBitmapIndex": false}
]
},
"metricsSpec": [
{
"type": "count",
@ -39,27 +47,17 @@
"fieldName": "user"
}
],
"granularitySpec": %%GRANULARITYSPEC%%,
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
{"type": "string", "name": "language", "createBitmapIndex": false}
]
}
}
}
"granularitySpec": %%GRANULARITYSPEC%%
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -1,103 +0,0 @@
{
"type": "index",
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "triple-added",
"fieldName": "triple-added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
},
{
"name": "thetaSketch",
"type": "thetaSketch",
"fieldName": "user"
},
{
"name": "quantilesDoublesSketch",
"type": "quantilesDoublesSketch",
"fieldName": "delta"
},
{
"name": "HLLSketchBuild",
"type": "HLLSketchBuild",
"fieldName": "user"
}
],
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
},
"transformSpec": {
"transforms": [
{
"type": "expression",
"name": "language",
"expression": "concat('language-', language)"
},
{
"type": "expression",
"name": "triple-added",
"expression": "added * 3"
}
]
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
}
},
"tuningConfig": {
"type": "index",
"maxRowsPerSegment": 3
}
}
}

View File

@ -3,6 +3,25 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
},
"metricsSpec": [
{
"type": "count",
@ -43,38 +62,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,14 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
},
"metricsSpec": [
{
"type": "count",
@ -38,27 +46,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "DAY",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,15 @@
"spec": {
"dataSchema": {
"dataSource": "%%REINDEX_DATASOURCE%%",
"timestampSpec": {
"column": "timestamp",
"format": "iso"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
},
"metricsSpec": [
{
"type": "doubleSum",
@ -34,28 +43,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "DAY",
"intervals" : [ "2013-08-31/2013-09-01" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp",
"format": "iso"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "ingestSegment",
"inputSource": {
"type": "druid",
"dataSource": "%%DATASOURCE%%",
"interval": "2013-08-31/2013-09-01"
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,25 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
},
"metricsSpec": [
{
"type": "count",
@ -28,38 +47,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type" : "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,15 @@
"spec": {
"dataSchema": {
"dataSource": "%%REINDEX_DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensionExclusions": [
"robot",
"continent"
]
},
"metricsSpec": [
{
"type": "count",
@ -30,29 +39,18 @@
"intervals": [
"2013-08-31/2013-09-02"
]
},
"parser": {
"parseSpec": {
"format": "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensionExclusions": [
"robot",
"continent"
]
}
}
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"type": "ingestSegment",
"inputSource": {
"type": "druid",
"dataSource": "%%DATASOURCE%%",
"interval": "2013-08-31/2013-09-02",
"maxInputSegmentBytesPerTask": 1
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,25 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
},
"metricsSpec": [
{
"type": "count",
@ -27,38 +46,17 @@
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second"
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data2*"
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,13 @@
"spec": {
"dataSchema": {
"dataSource": "%%REINDEX_DATASOURCE%%",
"timestampSpec": {
"column": "timestamp",
"format": "iso"
},
"dimensionsSpec": {
"dimensionExclusions" : ["robot", "continent"]
},
"metricsSpec": [
{
"type": "doubleSum",
@ -24,26 +31,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-01" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp",
"format": "iso"
},
"dimensionsSpec": {
"dimensionExclusions" : ["robot", "continent"]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "ingestSegment",
"inputSource": {
"type": "druid",
"dataSource": "%%DATASOURCE%%",
"interval": "2013-08-31/2013-09-01"
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -1,108 +0,0 @@
{
"type": "index",
"spec": {
"dataSchema": {
"dataSource": "%%REINDEX_DATASOURCE%%",
"metricsSpec": [
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "triple-added",
"fieldName": "triple-added"
},
{
"type": "doubleSum",
"name": "one-plus-triple-added",
"fieldName": "one-plus-triple-added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "double-deleted",
"fieldName": "double-deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
}
],
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-01" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp",
"format": "iso"
},
"dimensionsSpec": {
"dimensions": [
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"page",
"newPage",
"anonymous",
"namespace",
"country",
"region",
"city"
]
},
"transformSpec": {
"transforms": [
{
"type": "expression",
"name": "newPage",
"expression": "page"
},
{
"type": "expression",
"name": "city",
"expression": "concat('city-', city)"
},
{
"type": "expression",
"name": "one-plus-triple-added",
"expression": "\"triple-added\" + 1"
},
{
"type": "expression",
"name": "delta",
"expression": "\"delta\" / 2"
},
{
"type": "expression",
"name": "double-deleted",
"expression": "deleted * 2"
}
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "ingestSegment",
"dataSource": "%%DATASOURCE%%",
"interval": "2013-08-31/2013-09-01"
}
},
"tuningConfig": {
"type": "index"
}
}
}

View File

@ -3,6 +3,27 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp",
"format": "auto"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city",
"timestamp"
]
},
"metricsSpec": [
{
"type": "count",
@ -43,40 +64,22 @@
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city",
"timestamp"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
}
"files": [
"/resources/data/batch_index/json/wikipedia_index_data1.json",
"/resources/data/batch_index/json/wikipedia_index_data2.json",
"/resources/data/batch_index/json/wikipedia_index_data3.json"
]
},
"inputFormat": {
"type": "json"
},
"appendToExisting": false
},
"tuningConfig": {
"type": "index",

View File

@ -3,6 +3,17 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"dimensionsSpec": {
"dimensions": [
"channel",
"page",
"user"
]
},
"timestampSpec": {
"format": "auto",
"column": "time"
},
"granularitySpec": {
"type": "uniform",
"segmentGranularity": "day",
@ -11,23 +22,6 @@
"2015-09-12/2015-09-13"
]
},
"parser": {
"type": "hadoopyString",
"parseSpec": {
"format": "json",
"dimensionsSpec": {
"dimensions": [
"channel",
"page",
"user"
]
},
"timestampSpec": {
"format": "auto",
"column": "time"
}
}
},
"metricsSpec": [
{
"name": "count",
@ -52,10 +46,13 @@
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/shared/wikiticker-it",
"filter": "wikiticker-2015-09-12-sampled.json.gz"
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

View File

@ -1,100 +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.tests.indexer;
import com.fasterxml.jackson.core.JsonProcessingException;
import org.apache.druid.java.util.common.Pair;
import org.apache.druid.java.util.common.StringUtils;
import org.apache.druid.testing.guice.DruidTestModuleFactory;
import org.apache.druid.tests.TestNGGroup;
import org.testng.annotations.Guice;
import org.testng.annotations.Test;
import java.io.Closeable;
import java.util.function.Function;
@Test(groups = {TestNGGroup.BATCH_INDEX, TestNGGroup.QUICKSTART_COMPATIBLE})
@Guice(moduleFactory = DruidTestModuleFactory.class)
public class ITCombiningFirehoseFactoryIndexTest extends AbstractITBatchIndexTest
{
private static final String INDEX_TASK = "/indexer/wikipedia_index_task.json";
private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
private static final String INDEX_DATASOURCE = "wikipedia_index_test";
private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_firehose_index_task.json";
private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json";
private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test";
@Test
public void testIndexData() throws Exception
{
try (
final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
final Closeable ignored2 = unloader(COMBINING_INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
) {
final Function<String, String> combiningFirehoseSpecTransform = spec -> {
try {
return StringUtils.replace(
spec,
"%%COMBINING_DATASOURCE%%",
INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix()
);
}
catch (Exception e) {
throw new RuntimeException(e);
}
};
final Function<String, String> transform = spec -> {
try {
return StringUtils.replace(
spec,
"%%SEGMENT_AVAIL_TIMEOUT_MILLIS%%",
jsonMapper.writeValueAsString("0")
);
}
catch (JsonProcessingException e) {
throw new RuntimeException(e);
}
};
doIndexTest(
INDEX_DATASOURCE,
INDEX_TASK,
transform,
INDEX_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
doIndexTest(
COMBINING_INDEX_DATASOURCE,
COMBINING_INDEX_TASK,
combiningFirehoseSpecTransform,
COMBINING_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
}
}
}

View File

@ -41,7 +41,7 @@ public class ITCombiningInputSourceParallelIndexTest extends AbstractITBatchInde
private static final String INDEX_DATASOURCE = "wikipedia_index_test";
private static final String COMBINING_INDEX_TASK = "/indexer/wikipedia_combining_input_source_index_parallel_task.json";
private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_firehose_index_queries.json";
private static final String COMBINING_QUERIES_RESOURCE = "/indexer/wikipedia_combining_input_source_index_queries.json";
private static final String COMBINING_INDEX_DATASOURCE = "wikipedia_comb_index_test";
@Test

View File

@ -32,12 +32,10 @@ import java.io.IOException;
@Guice(moduleFactory = DruidTestModuleFactory.class)
public class ITTransformTest extends AbstractITBatchIndexTest
{
private static final String INDEX_TASK_WITH_FIREHOSE = "/indexer/wikipedia_index_task_with_transform.json";
private static final String INDEX_TASK_WITH_INPUT_SOURCE = "/indexer/wikipedia_index_task_with_inputsource_transform.json";
private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries_with_transform.json";
private static final String INDEX_DATASOURCE = "wikipedia_index_test";
private static final String REINDEX_TASK = "/indexer/wikipedia_reindex_task_with_transforms.json";
private static final String REINDEX_TASK_WITH_DRUID_INPUT_SOURCE = "/indexer/wikipedia_reindex_druid_input_source_task_with_transforms.json";
private static final String REINDEX_QUERIES_RESOURCE = "/indexer/wikipedia_reindex_queries_with_transforms.json";
private static final String REINDEX_DATASOURCE = "wikipedia_reindex_test";
@ -69,53 +67,4 @@ public class ITTransformTest extends AbstractITBatchIndexTest
);
}
}
@Test(enabled = false)
public void testIndexAndReIndexUsingIngestSegmentWithTransforms() throws IOException
{
// TODO: re-instate this test when https://github.com/apache/druid/issues/9591 is fixed
// Move the re-index step into testIndexAndReIndexWithTransformSpec for faster tests!
final String reindexDatasource = REINDEX_DATASOURCE + "-testIndexData";
try (
final Closeable ignored1 = unloader(INDEX_DATASOURCE + config.getExtraDatasourceNameSuffix());
final Closeable ignored2 = unloader(reindexDatasource + config.getExtraDatasourceNameSuffix())
) {
doIndexTest(
INDEX_DATASOURCE,
INDEX_TASK_WITH_INPUT_SOURCE,
INDEX_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
doReindexTest(
INDEX_DATASOURCE,
reindexDatasource,
REINDEX_TASK,
REINDEX_QUERIES_RESOURCE,
new Pair<>(false, false)
);
}
}
@Test(enabled = false)
public void testIndexWithFirehoseAndTransforms() throws IOException
{
// TODO: re-instate this test when https://github.com/apache/druid/issues/9589 is fixed
final String indexDatasource = INDEX_DATASOURCE + "-firehose";
try (
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
) {
doIndexTest(
indexDatasource,
INDEX_TASK_WITH_FIREHOSE,
INDEX_QUERIES_RESOURCE,
false,
true,
true,
new Pair<>(false, false)
);
}
}
}

View File

@ -1,95 +0,0 @@
{
"type": "index",
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
},
{
"name": "thetaSketch",
"type": "thetaSketch",
"fieldName": "user"
},
{
"name": "quantilesDoublesSketch",
"type": "quantilesDoublesSketch",
"fieldName": "delta"
},
{
"name": "HLLSketchBuild",
"type": "HLLSketchBuild",
"fieldName": "user"
}
],
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "combining",
"delegates": [
{
"type": "local",
"baseDir": "/resources/indexer",
"filter": "wikipedia_combining_index_data.json"
},
{
"type": "ingestSegment",
"dataSource": "%%COMBINING_DATASOURCE%%",
"interval": "2013-08-31/2013-09-02"
}
]
}
},
"tuningConfig": {
"type": "index",
"maxRowsPerSegment": 3
}
}
}

View File

@ -3,6 +3,26 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp",
"format": "auto"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
},
"metricsSpec": [
{
"type": "count",
@ -43,39 +63,22 @@
"segmentGranularity": "DAY",
"queryGranularity": "DAY",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
}
"files": [
"/resources/data/batch_index/json/wikipedia_index_data1.json",
"/resources/data/batch_index/json/wikipedia_index_data2.json",
"/resources/data/batch_index/json/wikipedia_index_data3.json"
]
},
"inputFormat": {
"type": "json"
},
"appendToExisting": false
},
"tuningConfig": {
"type": "index",

View File

@ -3,6 +3,14 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
{"type": "string", "name": "language", "createBitmapIndex": false}
]
},
"metricsSpec": [
{
"type": "count",
@ -39,27 +47,17 @@
"fieldName": "user"
}
],
"granularitySpec": %%GRANULARITYSPEC%%,
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
{"type": "string", "name": "language", "createBitmapIndex": false}
]
}
}
}
"granularitySpec": %%GRANULARITYSPEC%%
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type" : "json"
}
},
"tuningConfig": {

View File

@ -1,103 +0,0 @@
{
"type": "index",
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"metricsSpec": [
{
"type": "count",
"name": "count"
},
{
"type": "doubleSum",
"name": "added",
"fieldName": "added"
},
{
"type": "doubleSum",
"name": "triple-added",
"fieldName": "triple-added"
},
{
"type": "doubleSum",
"name": "deleted",
"fieldName": "deleted"
},
{
"type": "doubleSum",
"name": "delta",
"fieldName": "delta"
},
{
"name": "thetaSketch",
"type": "thetaSketch",
"fieldName": "user"
},
{
"name": "quantilesDoublesSketch",
"type": "quantilesDoublesSketch",
"fieldName": "delta"
},
{
"name": "HLLSketchBuild",
"type": "HLLSketchBuild",
"fieldName": "user"
}
],
"granularitySpec": {
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
},
"transformSpec": {
"transforms": [
{
"type": "expression",
"name": "language",
"expression": "concat('language-', language)"
},
{
"type": "expression",
"name": "triple-added",
"expression": "added * 3"
}
]
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
}
},
"tuningConfig": {
"type": "index",
"maxRowsPerSegment": 3
}
}
}

View File

@ -3,6 +3,25 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
},
"metricsSpec": [
{
"type": "count",
@ -43,38 +62,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type" : "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,14 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
},
"metricsSpec": [
{
"type": "count",
@ -38,27 +46,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "DAY",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type" : "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,15 @@
"spec": {
"dataSchema": {
"dataSource": "%%REINDEX_DATASOURCE%%",
"timestampSpec": {
"column": "timestamp",
"format": "iso"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
},
"metricsSpec": [
{
"type": "doubleSum",
@ -34,28 +43,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "DAY",
"intervals" : [ "2013-08-31/2013-09-01" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp",
"format": "iso"
},
"dimensionsSpec": {
"dimensions": [
"continent"
]
}
}
}
},
"ioConfig": {
"type": "index",
"firehose": {
"type": "ingestSegment",
"inputSource": {
"type": "druid",
"dataSource": "%%DATASOURCE%%",
"interval": "2013-08-31/2013-09-01"
},
"inputFormat": {
"type" : "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,25 @@
"spec": {
"dataSchema": {
"dataSource": "%%DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
},
"metricsSpec": [
{
"type": "count",
@ -28,38 +47,17 @@
"segmentGranularity": "DAY",
"queryGranularity": "second",
"intervals" : [ "2013-08-31/2013-09-02" ]
},
"parser": {
"parseSpec": {
"format" : "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensions": [
"page",
{"type": "string", "name": "language", "createBitmapIndex": false},
"user",
"unpatrolled",
"newPage",
"robot",
"anonymous",
"namespace",
"continent",
"country",
"region",
"city"
]
}
}
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"inputSource": {
"type": "local",
"baseDir": "/resources/data/batch_index/json",
"filter": "wikipedia_index_data*"
},
"inputFormat": {
"type" : "json"
}
},
"tuningConfig": {

View File

@ -3,6 +3,15 @@
"spec": {
"dataSchema": {
"dataSource": "%%REINDEX_DATASOURCE%%",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensionExclusions": [
"robot",
"continent"
]
},
"metricsSpec": [
{
"type": "count",
@ -30,29 +39,18 @@
"intervals": [
"2013-08-31/2013-09-02"
]
},
"parser": {
"parseSpec": {
"format": "json",
"timestampSpec": {
"column": "timestamp"
},
"dimensionsSpec": {
"dimensionExclusions": [
"robot",
"continent"
]
}
}
}
},
"ioConfig": {
"type": "index_parallel",
"firehose": {
"type": "ingestSegment",
"inputSource": {
"type": "druid",
"dataSource": "%%DATASOURCE%%",
"interval": "2013-08-31/2013-09-02",
"maxInputSegmentBytesPerTask": 1
},
"inputFormat": {
"type": "json"
}
},
"tuningConfig": {

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