makeLoadSpec(String bucket, String key)
- {
- return ImmutableMap.of(
- "type",
- OssStorageDruidModule.SCHEME_ZIP,
- "bucket",
- bucket,
- "key",
- key
- );
- }
-
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssInputDataConfig.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssInputDataConfig.java
deleted file mode 100644
index c2ef2dfb465b..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssInputDataConfig.java
+++ /dev/null
@@ -1,52 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import javax.validation.constraints.Max;
-import javax.validation.constraints.Min;
-
-/**
- * Stores the configuration for options related to reading
- * input data from aliyun OSS into Druid
- */
-public class OssInputDataConfig
-{
- /**
- * The maximum number of input files matching a given prefix to retrieve
- * from aliyun OSS at a time.
- * valid range is [1,1000]
- */
- @JsonProperty
- @Min(1)
- @Max(OssUtils.MAX_LISTING_LENGTH)
- private int maxListingLength = OssUtils.MAX_LISTING_LENGTH;
-
- public void setMaxListingLength(int maxListingLength)
- {
- this.maxListingLength = maxListingLength;
- }
-
- public int getMaxListingLength()
- {
- return maxListingLength;
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssLoadSpec.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssLoadSpec.java
deleted file mode 100644
index 155c26fbf3ca..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssLoadSpec.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Preconditions;
-import org.apache.druid.data.input.impl.CloudObjectLocation;
-import org.apache.druid.segment.loading.LoadSpec;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-
-import java.io.File;
-
-@JsonTypeName(OssStorageDruidModule.SCHEME_ZIP)
-public class OssLoadSpec implements LoadSpec
-{
- private final String bucket;
- private final String key;
-
- private final OssDataSegmentPuller puller;
-
- @JsonCreator
- public OssLoadSpec(
- @JacksonInject OssDataSegmentPuller puller,
- @JsonProperty(OssDataSegmentPuller.BUCKET) String bucket,
- @JsonProperty(OssDataSegmentPuller.KEY) String key
- )
- {
- Preconditions.checkNotNull(bucket);
- Preconditions.checkNotNull(key);
- this.bucket = bucket;
- this.key = key;
- this.puller = puller;
- }
-
- @Override
- public LoadSpecResult loadSegment(File outDir) throws SegmentLoadingException
- {
- return new LoadSpecResult(puller.getSegmentFiles(new CloudObjectLocation(bucket, key), outDir).size());
- }
-
- @JsonProperty(OssDataSegmentPuller.BUCKET)
- public String getBucket()
- {
- return bucket;
- }
-
- @JsonProperty(OssDataSegmentPuller.KEY)
- public String getKey()
- {
- return key;
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssObjectSummaryIterator.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssObjectSummaryIterator.java
deleted file mode 100644
index 8bba8961eeeb..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssObjectSummaryIterator.java
+++ /dev/null
@@ -1,156 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import org.apache.druid.java.util.common.RE;
-
-import java.net.URI;
-import java.util.Iterator;
-import java.util.NoSuchElementException;
-
-/**
- * Iterator class used by {@link OssUtils#objectSummaryIterator}.
- *
- * As required by the specification of that method, this iterator is computed incrementally in batches of
- * {@code maxListLength}. The first call is made at the same time the iterator is constructed.
- *
- */
-public class OssObjectSummaryIterator implements Iterator
-{
- private final OSS client;
- private final Iterator prefixesIterator;
- private final int maxListingLength;
-
- private ListObjectsRequest request;
- private ObjectListing result;
- private Iterator objectSummaryIterator;
- private OSSObjectSummary currentObjectSummary;
-
- OssObjectSummaryIterator(
- final OSS client,
- final Iterable prefixes,
- final int maxListingLength
- )
- {
- this.client = client;
- this.prefixesIterator = prefixes.iterator();
- this.maxListingLength = Math.min(OssUtils.MAX_LISTING_LENGTH, maxListingLength);
-
- prepareNextRequest();
- fetchNextBatch();
- advanceObjectSummary();
- }
-
- @Override
- public boolean hasNext()
- {
- return currentObjectSummary != null;
- }
-
- @Override
- public OSSObjectSummary next()
- {
- if (currentObjectSummary == null) {
- throw new NoSuchElementException();
- }
-
- final OSSObjectSummary retVal = currentObjectSummary;
- advanceObjectSummary();
- return retVal;
- }
-
- private void prepareNextRequest()
- {
- final URI currentUri = prefixesIterator.next();
- final String currentBucket = currentUri.getAuthority();
- final String currentPrefix = OssUtils.extractKey(currentUri);
-
- request = new ListObjectsRequest(currentBucket, currentPrefix, null, null, maxListingLength);
- }
-
- private void fetchNextBatch()
- {
- try {
- result = OssUtils.retry(() -> client.listObjects(request));
- request.setMarker(result.getNextMarker());
- objectSummaryIterator = result.getObjectSummaries().iterator();
- }
- catch (OSSException e) {
- throw new RE(
- e,
- "Failed to get object summaries from aliyun OSS bucket[%s], prefix[%s]; error: %s",
- request.getBucketName(),
- request.getPrefix(),
- e.getMessage()
- );
- }
- catch (Exception e) {
- throw new RE(
- e,
- "Failed to get object summaries from aliyun OSS bucket[%s], prefix[%s]",
- request.getBucketName(),
- request.getPrefix()
- );
- }
- }
-
- /**
- * Advance objectSummaryIterator to the next non-placeholder, updating "currentObjectSummary".
- */
- private void advanceObjectSummary()
- {
- while (objectSummaryIterator.hasNext() || result.isTruncated() || prefixesIterator.hasNext()) {
- while (objectSummaryIterator.hasNext()) {
- currentObjectSummary = objectSummaryIterator.next();
- // skips directories and empty objects
- if (currentObjectSummary.getSize() > 0 && !isDirectory(currentObjectSummary)) {
- return;
- }
- }
-
- // Exhausted "objectSummaryIterator" without finding a non-placeholder.
- if (result.isTruncated()) {
- fetchNextBatch();
- } else if (prefixesIterator.hasNext()) {
- prepareNextRequest();
- fetchNextBatch();
- }
- }
-
- // Truly nothing left to read.
- currentObjectSummary = null;
- }
-
- /**
- * Checks if a given object is a directory placeholder and should be ignored.
- *
- * Based on {@link org.apache.druid.storage.s3.ObjectSummaryIterator} which is adapted from org.jets3t.service.model.StorageObject.isDirectoryPlaceholder().
- *
- */
- private static boolean isDirectory(final OSSObjectSummary objectSummary)
- {
- return objectSummary.getSize() == 0 && objectSummary.getKey().endsWith("/");
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssStorageConfig.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssStorageConfig.java
deleted file mode 100644
index d3edcd6105c6..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssStorageConfig.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-public class OssStorageConfig
-{
- @JsonProperty
- private String bucket = "";
-
- @JsonProperty
- private String prefix = "";
-
- public void setBucket(String bucket)
- {
- this.bucket = bucket;
- }
- public void setPrefix(String prefix)
- {
- this.prefix = prefix;
- }
-
- public String getBucket()
- {
- return bucket;
- }
-
- public String getPrefix()
- {
- return prefix;
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssStorageDruidModule.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssStorageDruidModule.java
deleted file mode 100644
index d682bbac8232..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssStorageDruidModule.java
+++ /dev/null
@@ -1,106 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.fasterxml.jackson.core.Version;
-import com.fasterxml.jackson.databind.Module;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import com.google.inject.Provides;
-import com.google.inject.multibindings.MapBinder;
-import org.apache.druid.data.SearchableVersionedDataFinder;
-import org.apache.druid.data.input.aliyun.OssClientConfig;
-import org.apache.druid.guice.Binders;
-import org.apache.druid.guice.JsonConfigProvider;
-import org.apache.druid.guice.LazySingleton;
-import org.apache.druid.initialization.DruidModule;
-
-import java.util.List;
-
-public class OssStorageDruidModule implements DruidModule
-{
- public static final String SCHEME = "oss";
- public static final String SCHEME_ZIP = "oss_zip";
-
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new Module()
- {
- @Override
- public String getModuleName()
- {
- return "DruidAliyunOss-" + System.identityHashCode(this);
- }
-
- @Override
- public Version version()
- {
- return Version.unknownVersion();
- }
-
- @Override
- public void setupModule(SetupContext context)
- {
- context.registerSubtypes(OssLoadSpec.class);
- }
- }
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
- MapBinder.newMapBinder(binder, String.class, SearchableVersionedDataFinder.class)
- .addBinding(SCHEME)
- .to(OssTimestampVersionedDataFinder.class)
- .in(LazySingleton.class);
- Binders.dataSegmentKillerBinder(binder)
- .addBinding(SCHEME_ZIP)
- .to(OssDataSegmentKiller.class)
- .in(LazySingleton.class);
- Binders.dataSegmentMoverBinder(binder)
- .addBinding(SCHEME_ZIP)
- .to(OssDataSegmentMover.class)
- .in(LazySingleton.class);
- Binders.dataSegmentArchiverBinder(binder)
- .addBinding(SCHEME_ZIP)
- .to(OssDataSegmentArchiver.class)
- .in(LazySingleton.class);
- Binders.dataSegmentPusherBinder(binder).addBinding(SCHEME).to(OssDataSegmentPusher.class).in(LazySingleton.class);
- JsonConfigProvider.bind(binder, "druid.oss", OssClientConfig.class);
- JsonConfigProvider.bind(binder, "druid.storage.oss", OssInputDataConfig.class);
- JsonConfigProvider.bind(binder, "druid.storage.oss", OssStorageConfig.class);
- JsonConfigProvider.bind(binder, "druid.storage.oss", OssDataSegmentArchiverConfig.class);
-
- Binders.taskLogsBinder(binder).addBinding(SCHEME).to(OssTaskLogs.class);
- JsonConfigProvider.bind(binder, "druid.indexer.logs.oss", OssTaskLogsConfig.class);
- binder.bind(OssTaskLogs.class).in(LazySingleton.class);
- }
-
- @Provides
- @LazySingleton
- public OSS initializeOssClient(OssClientConfig inputSourceConfig)
- {
- return inputSourceConfig.buildClient();
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTaskLogs.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTaskLogs.java
deleted file mode 100644
index 515d85096e03..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTaskLogs.java
+++ /dev/null
@@ -1,201 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.model.GetObjectRequest;
-import com.aliyun.oss.model.ObjectMetadata;
-import com.google.common.base.Optional;
-import com.google.common.base.Throwables;
-import com.google.common.io.ByteSource;
-import com.google.inject.Inject;
-import org.apache.druid.common.utils.CurrentTimeMillisSupplier;
-import org.apache.druid.java.util.common.IOE;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.tasklogs.TaskLogs;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.util.Collections;
-import java.util.Date;
-
-/**
- * Provides task logs archived in aliyun OSS
- */
-public class OssTaskLogs implements TaskLogs
-{
- private static final Logger log = new Logger(OssTaskLogs.class);
-
- private final OSS client;
- private final OssTaskLogsConfig config;
- private final OssInputDataConfig inputDataConfig;
- private final CurrentTimeMillisSupplier timeSupplier;
-
- @Inject
- public OssTaskLogs(
- OSS service,
- OssTaskLogsConfig config,
- OssInputDataConfig inputDataConfig,
- CurrentTimeMillisSupplier timeSupplier
- )
- {
- this.client = service;
- this.config = config;
- this.inputDataConfig = inputDataConfig;
- this.timeSupplier = timeSupplier;
- }
-
- @Override
- public Optional streamTaskLog(final String taskid, final long offset) throws IOException
- {
- final String taskKey = getTaskLogKey(taskid, "log");
- return streamTaskFile(offset, taskKey);
- }
-
- @Override
- public Optional streamTaskReports(String taskid) throws IOException
- {
- final String taskKey = getTaskLogKey(taskid, "report.json");
- return streamTaskFile(0, taskKey);
- }
-
- private Optional streamTaskFile(final long offset, String taskKey) throws IOException
- {
- try {
- final ObjectMetadata objectMetadata = client.getObjectMetadata(config.getBucket(), taskKey);
-
- return Optional.of(
- new ByteSource()
- {
- @Override
- public InputStream openStream() throws IOException
- {
- try {
- final long start;
- final long end = objectMetadata.getContentLength() - 1;
-
- if (offset > 0 && offset < objectMetadata.getContentLength()) {
- start = offset;
- } else if (offset < 0 && (-1 * offset) < objectMetadata.getContentLength()) {
- start = objectMetadata.getContentLength() + offset;
- } else {
- start = 0;
- }
-
- final GetObjectRequest request = new GetObjectRequest(config.getBucket(), taskKey);
- request.setMatchingETagConstraints(Collections.singletonList(objectMetadata.getETag()));
- request.setRange(start, end);
-
- return client.getObject(request).getObjectContent();
- }
- catch (OSSException e) {
- throw new IOException(e);
- }
- }
- }
- );
- }
- catch (OSSException e) {
- if ("NoSuchKey".equals(e.getErrorCode())
- || "NoSuchBucket".equals(e.getErrorCode())) {
- return Optional.absent();
- } else {
- throw new IOE(e, "Failed to stream logs from: %s", taskKey);
- }
- }
- }
-
- @Override
- public void pushTaskLog(final String taskid, final File logFile) throws IOException
- {
- final String taskKey = getTaskLogKey(taskid, "log");
- log.info("Pushing task log %s to: %s", logFile, taskKey);
- pushTaskFile(logFile, taskKey);
- }
-
- @Override
- public void pushTaskReports(String taskid, File reportFile) throws IOException
- {
- final String taskKey = getTaskLogKey(taskid, "report.json");
- log.info("Pushing task reports %s to: %s", reportFile, taskKey);
- pushTaskFile(reportFile, taskKey);
- }
-
- private void pushTaskFile(final File logFile, String taskKey) throws IOException
- {
- try {
- OssUtils.retry(
- () -> {
- OssUtils.uploadFileIfPossible(client, config.getBucket(), taskKey, logFile);
- return null;
- }
- );
- }
- catch (Exception e) {
- Throwables.propagateIfInstanceOf(e, IOException.class);
- throw new RuntimeException(e);
- }
- }
-
- String getTaskLogKey(String taskid, String filename)
- {
- return StringUtils.format("%s/%s/%s", config.getPrefix(), taskid, filename);
- }
-
- @Override
- public void killAll() throws IOException
- {
- log.info(
- "Deleting all task logs from aliyun OSS location [bucket: '%s' prefix: '%s'].",
- config.getBucket(),
- config.getPrefix()
- );
-
- long now = timeSupplier.getAsLong();
- killOlderThan(now);
- }
-
- @Override
- public void killOlderThan(long timestamp) throws IOException
- {
- log.info(
- "Deleting all task logs from aliyun OSS location [bucket: '%s' prefix: '%s'] older than %s.",
- config.getBucket(),
- config.getPrefix(),
- new Date(timestamp)
- );
- try {
- OssUtils.deleteObjectsInPath(
- client,
- inputDataConfig,
- config.getBucket(),
- config.getPrefix(),
- (object) -> object.getLastModified().getTime() < timestamp
- );
- }
- catch (Exception e) {
- log.error("Error occurred while deleting task log files from aliyun OSS. Error: %s", e.getMessage());
- throw new IOException(e);
- }
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTaskLogsConfig.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTaskLogsConfig.java
deleted file mode 100644
index 3a3e5c37c1b8..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTaskLogsConfig.java
+++ /dev/null
@@ -1,73 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.annotations.VisibleForTesting;
-
-import javax.validation.constraints.NotNull;
-
-public class OssTaskLogsConfig
-{
- @JsonProperty
- @NotNull
- private String bucket = null;
-
- @JsonProperty
- @NotNull
- private String prefix = null;
-
- @JsonProperty
- private boolean disableAcl = false;
-
- @VisibleForTesting
- void setDisableAcl(boolean disableAcl)
- {
- this.disableAcl = disableAcl;
- }
-
- public String getBucket()
- {
- return bucket;
- }
-
- @VisibleForTesting
- void setBucket(String bucket)
- {
- this.bucket = bucket;
- }
-
- public String getPrefix()
- {
- return prefix;
- }
-
- @VisibleForTesting
- void setPrefix(String prefix)
- {
- this.prefix = prefix;
- }
-
- public boolean getDisableAcl()
- {
- return disableAcl;
- }
-
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTimestampVersionedDataFinder.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTimestampVersionedDataFinder.java
deleted file mode 100644
index 0b3f708477de..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssTimestampVersionedDataFinder.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.google.inject.Inject;
-import org.apache.druid.data.SearchableVersionedDataFinder;
-import org.apache.druid.data.input.impl.CloudObjectLocation;
-import org.apache.druid.java.util.common.StringUtils;
-
-import javax.annotation.Nullable;
-import java.net.URI;
-import java.util.Collections;
-import java.util.Iterator;
-import java.util.regex.Pattern;
-
-public class OssTimestampVersionedDataFinder extends OssDataSegmentPuller implements SearchableVersionedDataFinder
-{
- @Inject
- public OssTimestampVersionedDataFinder(OSS client)
- {
- super(client);
- }
-
- /**
- * Gets the key with the most recently modified timestamp.
- * `pattern` is evaluated against the entire key AFTER the path given in `uri`.
- * The substring `pattern` is matched against will have a leading `/` removed.
- * For example `oss://some_bucket/some_prefix/some_key` with a URI of `oss://some_bucket/some_prefix` will match against `some_key`.
- * `oss://some_bucket/some_prefixsome_key` with a URI of `oss://some_bucket/some_prefix` will match against `some_key`
- * `oss://some_bucket/some_prefix//some_key` with a URI of `oss://some_bucket/some_prefix` will match against `/some_key`
- *
- * @param uri The URI of in the form of `oss://some_bucket/some_key`
- * @param pattern The pattern matcher to determine if a *key* is of interest, or `null` to match everything.
- * @return A URI to the most recently modified object which matched the pattern.
- */
- @Override
- public URI getLatestVersion(final URI uri, final @Nullable Pattern pattern)
- {
- try {
- final CloudObjectLocation coords = new CloudObjectLocation(OssUtils.checkURI(uri));
- long mostRecent = Long.MIN_VALUE;
- URI latest = null;
- final Iterator objectSummaryIterator = OssUtils.objectSummaryIterator(
- client,
- Collections.singletonList(uri),
- OssUtils.MAX_LISTING_LENGTH
- );
- while (objectSummaryIterator.hasNext()) {
- final OSSObjectSummary objectSummary = objectSummaryIterator.next();
- final CloudObjectLocation objectLocation = OssUtils.summaryToCloudObjectLocation(objectSummary);
- // remove coords path prefix from object path
- String keyString = StringUtils.maybeRemoveLeadingSlash(
- objectLocation.getPath().substring(coords.getPath().length())
- );
- if (pattern != null && !pattern.matcher(keyString).matches()) {
- continue;
- }
- final long latestModified = objectSummary.getLastModified().getTime();
- if (latestModified >= mostRecent) {
- mostRecent = latestModified;
- latest = objectLocation.toUri(OssStorageDruidModule.SCHEME);
- }
- }
- return latest;
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
- }
-
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssUtils.java b/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssUtils.java
deleted file mode 100644
index 1a707c785570..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssUtils.java
+++ /dev/null
@@ -1,271 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.model.DeleteObjectsRequest;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import com.aliyun.oss.model.PutObjectRequest;
-import com.google.common.base.Joiner;
-import com.google.common.base.Predicate;
-import com.google.common.collect.ImmutableList;
-import org.apache.druid.data.input.impl.CloudObjectLocation;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.RetryUtils;
-import org.apache.druid.java.util.common.RetryUtils.Task;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.logger.Logger;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
-
-public class OssUtils
-{
- private static final String SCHEME = OssStorageDruidModule.SCHEME;
- private static final Joiner JOINER = Joiner.on("/").skipNulls();
- private static final Logger log = new Logger(OssUtils.class);
- public static final int MAX_LISTING_LENGTH = 1000; //limited by Aliyun OSS SDK
-
-
- static boolean isServiceExceptionRecoverable(OSSException ex)
- {
- final boolean isIOException = ex.getCause() instanceof IOException;
- final boolean isTimeout = "RequestTimeout".equals(ex.getErrorCode());
- final boolean badStatusCode = false; //ex. == 400 || ex.getStatusCode() == 403 || ex.getStatusCode() == 404;
- return !badStatusCode && (isIOException || isTimeout);
- }
-
- public static final Predicate RETRYABLE = new Predicate()
- {
- @Override
- public boolean apply(Throwable e)
- {
- if (e == null) {
- return false;
- } else if (e instanceof IOException) {
- return true;
- } else if (e instanceof OSSException) {
- return isServiceExceptionRecoverable((OSSException) e);
- } else {
- return apply(e.getCause());
- }
- }
- };
-
- /**
- * Retries aliyun OSS operations that fail due to io-related exceptions. Service-level exceptions (access denied, file not
- * found, etc) are not retried.
- */
- static T retry(Task f) throws Exception
- {
- return RetryUtils.retry(f, RETRYABLE, RetryUtils.DEFAULT_MAX_TRIES);
- }
-
- static boolean isObjectInBucketIgnoringPermission(
- OSS client,
- String bucketName,
- String objectKey
- )
- {
- try {
- return client.doesObjectExist(bucketName, objectKey);
- }
- catch (OSSException e) {
- if (e.getErrorCode().equals("NoSuchKey")) {
- // Object is inaccessible to current user, but does exist.
- return true;
- }
- // Something else has gone wrong
- throw e;
- }
- }
-
- /**
- * Create an iterator over a set of aliyun OSS objects specified by a set of prefixes.
- *
- * For each provided prefix URI, the iterator will walk through all objects that are in the same bucket as the
- * provided URI and whose keys start with that URI's path, except for directory placeholders (which will be
- * ignored). The iterator is computed incrementally by calling {@link OSS#listObjects} for
- * each prefix in batches of {@param maxListingLength}. The first call is made at the same time the iterator is
- * constructed.
- */
- public static Iterator objectSummaryIterator(
- final OSS client,
- final Iterable prefixes,
- final int maxListingLength
- )
- {
- return new OssObjectSummaryIterator(client, prefixes, maxListingLength);
- }
-
- /**
- * Create an {@link URI} from the given {@link OSSObjectSummary}. The result URI is composed as below.
- *
- *
- * {@code oss://{BUCKET_NAME}/{OBJECT_KEY}}
- *
- */
- public static URI summaryToUri(OSSObjectSummary object)
- {
- return summaryToCloudObjectLocation(object).toUri(SCHEME);
- }
-
- public static CloudObjectLocation summaryToCloudObjectLocation(OSSObjectSummary object)
- {
- return new CloudObjectLocation(object.getBucketName(), object.getKey());
- }
-
- static String constructSegmentPath(String baseKey, String storageDir)
- {
- return JOINER.join(
- baseKey.isEmpty() ? null : baseKey,
- storageDir
- ) + "/index.zip";
- }
-
- public static String extractKey(URI uri)
- {
- return StringUtils.maybeRemoveLeadingSlash(uri.getPath());
- }
-
- public static URI checkURI(URI uri)
- {
- if (uri.getScheme().equalsIgnoreCase(OssStorageDruidModule.SCHEME_ZIP)) {
- uri = URI.create(SCHEME + uri.toString().substring(OssStorageDruidModule.SCHEME_ZIP.length()));
- }
- return CloudObjectLocation.validateUriScheme(SCHEME, uri);
- }
-
- /**
- * Gets a single {@link OSSObjectSummary} from aliyun OSS. Since this method might return a wrong object if there are multiple
- * objects that match the given key, this method should be used only when it's guaranteed that the given key is unique
- * in the given bucket.
- *
- * @param client aliyun OSS client
- * @param bucket aliyun OSS bucket
- * @param key unique key for the object to be retrieved
- */
- public static OSSObjectSummary getSingleObjectSummary(OSS client, String bucket, String key)
- {
- final ListObjectsRequest request = new ListObjectsRequest();
- request.setBucketName(bucket);
- request.setPrefix(key);
- request.setMaxKeys(1);
- final ObjectListing result = client.listObjects(request);
-
- // Using getObjectSummaries().size() instead of getKeyCount as, in some cases
- // it is observed that even though the getObjectSummaries returns some data
- // keyCount is still zero.
- if (result.getObjectSummaries().size() == 0) {
- throw new ISE("Cannot find object for bucket[%s] and key[%s]", bucket, key);
- }
- final OSSObjectSummary objectSummary = result.getObjectSummaries().get(0);
- if (!objectSummary.getBucketName().equals(bucket) || !objectSummary.getKey().equals(key)) {
- throw new ISE("Wrong object[%s] for bucket[%s] and key[%s]", objectSummary, bucket, key);
- }
-
- return objectSummary;
- }
-
- /**
- * Delete the files from aliyun OSS in a specified bucket, matching a specified prefix and filter
- *
- * @param client aliyun OSS client
- * @param config specifies the configuration to use when finding matching files in aliyun OSS to delete
- * @param bucket aliyun OSS bucket
- * @param prefix the file prefix
- * @param filter function which returns true if the prefix file found should be deleted and false otherwise.
- * @throws Exception
- */
- public static void deleteObjectsInPath(
- OSS client,
- OssInputDataConfig config,
- String bucket,
- String prefix,
- Predicate filter
- )
- throws Exception
- {
- final List keysToDelete = new ArrayList<>(config.getMaxListingLength());
- final OssObjectSummaryIterator iterator = new OssObjectSummaryIterator(
- client,
- ImmutableList.of(new CloudObjectLocation(bucket, prefix).toUri("http")),
- config.getMaxListingLength()
- );
-
- while (iterator.hasNext()) {
- final OSSObjectSummary nextObject = iterator.next();
- if (filter.apply(nextObject)) {
- keysToDelete.add(nextObject.getKey());
- if (keysToDelete.size() == config.getMaxListingLength()) {
- deleteBucketKeys(client, bucket, keysToDelete);
- log.info("Deleted %d files", keysToDelete.size());
- keysToDelete.clear();
- }
- }
- }
-
- if (keysToDelete.size() > 0) {
- deleteBucketKeys(client, bucket, keysToDelete);
- log.info("Deleted %d files", keysToDelete.size());
- }
- }
-
- private static void deleteBucketKeys(
- OSS client,
- String bucket,
- List keysToDelete
- )
- throws Exception
- {
- DeleteObjectsRequest deleteRequest = new DeleteObjectsRequest(bucket).withKeys(keysToDelete);
- OssUtils.retry(() -> {
- client.deleteObjects(deleteRequest);
- return null;
- });
- }
-
- /**
- * Uploads a file to aliyun OSS if possible. First trying to set ACL to give the bucket owner full control of the file before uploading.
- *
- * @param client aliyun OSS client
- * @param key The key under which to store the new object.
- * @param file The path of the file to upload to aliyun OSS.
- */
- static void uploadFileIfPossible(
- OSS client,
- String bucket,
- String key,
- File file
- )
- {
- final PutObjectRequest putObjectRequest = new PutObjectRequest(bucket, key, file);
-
- log.info("Pushing [%s] to bucket[%s] and key[%s].", file, bucket, key);
- client.putObject(putObjectRequest);
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
deleted file mode 100644
index 3d434e7c9021..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ /dev/null
@@ -1,18 +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.
-
-org.apache.druid.storage.aliyun.OssStorageDruidModule
-org.apache.druid.firehose.aliyun.OssFirehoseDruidModule
-org.apache.druid.data.input.aliyun.OssInputSourceDruidModule
\ No newline at end of file
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java
deleted file mode 100644
index 2bd9d5816acc..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/data/input/aliyun/OssInputSourceTest.java
+++ /dev/null
@@ -1,660 +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.data.input.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSClient;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.model.GetObjectRequest;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObject;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import com.fasterxml.jackson.core.JsonParser;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.Module;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.deser.std.StdDeserializer;
-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.data.input.InputRow;
-import org.apache.druid.data.input.InputRowSchema;
-import org.apache.druid.data.input.InputSourceReader;
-import org.apache.druid.data.input.InputSplit;
-import org.apache.druid.data.input.MaxSizeSplitHintSpec;
-import org.apache.druid.data.input.impl.CloudObjectLocation;
-import org.apache.druid.data.input.impl.CsvInputFormat;
-import org.apache.druid.data.input.impl.DimensionsSpec;
-import org.apache.druid.data.input.impl.JsonInputFormat;
-import org.apache.druid.data.input.impl.TimestampSpec;
-import org.apache.druid.initialization.DruidModule;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.parsers.CloseableIterator;
-import org.apache.druid.java.util.common.parsers.JSONPathSpec;
-import org.apache.druid.metadata.DefaultPasswordProvider;
-import org.apache.druid.storage.aliyun.OssInputDataConfig;
-import org.apache.druid.storage.aliyun.OssUtils;
-import org.apache.druid.testing.InitializedNullHandlingTest;
-import org.apache.druid.utils.CompressionUtils;
-import org.easymock.EasyMock;
-import org.easymock.IArgumentMatcher;
-import org.hamcrest.CoreMatchers;
-import org.joda.time.DateTime;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.internal.matchers.ThrowableMessageMatcher;
-import org.junit.rules.ExpectedException;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.ByteArrayInputStream;
-import java.io.ByteArrayOutputStream;
-import java.io.IOException;
-import java.net.URI;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-import java.util.stream.Stream;
-
-public class OssInputSourceTest extends InitializedNullHandlingTest
-{
- private static final ObjectMapper MAPPER = createObjectMapper();
- private static final OSS OSSCLIENT = EasyMock.createMock(OSSClient.class);
- private static final OssInputDataConfig INPUT_DATA_CONFIG;
- private static final int MAX_LISTING_LENGTH = 10;
-
- private static final List EXPECTED_URIS = Arrays.asList(
- URI.create("oss://foo/bar/file.csv"),
- URI.create("oss://bar/foo/file2.csv")
- );
-
- private static final List EXPECTED_COMPRESSED_URIS = Arrays.asList(
- URI.create("oss://foo/bar/file.csv.gz"),
- URI.create("oss://bar/foo/file2.csv.gz")
- );
-
- private static final List> EXPECTED_COORDS =
- EXPECTED_URIS.stream()
- .map(uri -> Collections.singletonList(new CloudObjectLocation(uri)))
- .collect(Collectors.toList());
-
- private static final List PREFIXES = Arrays.asList(
- URI.create("oss://foo/bar"),
- URI.create("oss://bar/foo")
- );
-
- private static final OssClientConfig CLOUD_CONFIG_PROPERTIES = new OssClientConfig(
- "test.oss-cn.aliyun.com",
- new DefaultPasswordProvider("myKey"),
- new DefaultPasswordProvider("mySecret"));
-
- private static final List EXPECTED_LOCATION =
- ImmutableList.of(new CloudObjectLocation("foo", "bar/file.csv"));
-
- private static final DateTime NOW = DateTimes.nowUtc();
- private static final byte[] CONTENT =
- StringUtils.toUtf8(StringUtils.format("%d,hello,world", NOW.getMillis()));
-
- static {
- INPUT_DATA_CONFIG = new OssInputDataConfig();
- INPUT_DATA_CONFIG.setMaxListingLength(MAX_LISTING_LENGTH);
- }
-
- @Rule
- public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
- @Rule
- public ExpectedException expectedException = ExpectedException.none();
-
- @Test
- public void testSerdeWithUris() throws Exception
- {
- final OssInputSource withUris = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- EXPECTED_URIS,
- null,
- null,
- null
- );
- final OssInputSource serdeWithUris = MAPPER.readValue(MAPPER.writeValueAsString(withUris), OssInputSource.class);
- Assert.assertEquals(withUris, serdeWithUris);
- }
-
- @Test
- public void testSerdeWithPrefixes() throws Exception
- {
- final OssInputSource withPrefixes = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- PREFIXES,
- null,
- null
- );
- final OssInputSource serdeWithPrefixes =
- MAPPER.readValue(MAPPER.writeValueAsString(withPrefixes), OssInputSource.class);
- Assert.assertEquals(withPrefixes, serdeWithPrefixes);
- }
-
- @Test
- public void testSerdeWithObjects() throws Exception
- {
- final OssInputSource withPrefixes = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- null,
- EXPECTED_LOCATION,
- null
- );
- final OssInputSource serdeWithPrefixes =
- MAPPER.readValue(MAPPER.writeValueAsString(withPrefixes), OssInputSource.class);
- Assert.assertEquals(withPrefixes, serdeWithPrefixes);
- }
-
- @Test
- public void testInputSourceUseDefaultPasswordWhenCloudConfigPropertiesWithoutCrediential()
- {
- OssClientConfig mockConfigPropertiesWithoutKeyAndSecret = EasyMock.createMock(OssClientConfig.class);
- EasyMock.reset(mockConfigPropertiesWithoutKeyAndSecret);
- EasyMock.expect(mockConfigPropertiesWithoutKeyAndSecret.isCredentialsConfigured())
- .andStubReturn(false);
- EasyMock.expect(mockConfigPropertiesWithoutKeyAndSecret.buildClient())
- .andReturn(OSSCLIENT);
- EasyMock.replay(mockConfigPropertiesWithoutKeyAndSecret);
- final OssInputSource withPrefixes = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- null,
- EXPECTED_LOCATION,
- mockConfigPropertiesWithoutKeyAndSecret
- );
- Assert.assertNotNull(withPrefixes);
-
- withPrefixes.createEntity(new CloudObjectLocation("bucket", "path"));
- EasyMock.verify(mockConfigPropertiesWithoutKeyAndSecret);
- }
-
- @Test
- public void testSerdeOssClientLazyInitializedWithCrediential() throws Exception
- {
- OssClientConfig clientConfig = EasyMock.createMock(OssClientConfig.class);
- EasyMock.replay(clientConfig);
- final OssInputSource withPrefixes = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- null,
- EXPECTED_LOCATION,
- CLOUD_CONFIG_PROPERTIES
- );
- final OssInputSource serdeWithPrefixes =
- MAPPER.readValue(MAPPER.writeValueAsString(withPrefixes), OssInputSource.class);
- Assert.assertEquals(withPrefixes, serdeWithPrefixes);
- EasyMock.verify(clientConfig);
- }
-
- @Test
- public void testSerdeOssClientLazyInitializedWithoutCrediential() throws Exception
- {
- OssClientConfig clientConfig = EasyMock.createMock(OssClientConfig.class);
- EasyMock.replay(clientConfig);
- final OssInputSource withPrefixes = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- null,
- EXPECTED_LOCATION,
- null
- );
- final OssInputSource serdeWithPrefixes =
- MAPPER.readValue(MAPPER.writeValueAsString(withPrefixes), OssInputSource.class);
- Assert.assertEquals(withPrefixes, serdeWithPrefixes);
- EasyMock.verify(clientConfig);
- }
-
- @Test
- public void testSerdeWithExtraEmptyLists() throws Exception
- {
- final OssInputSource withPrefixes = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- ImmutableList.of(),
- ImmutableList.of(),
- EXPECTED_LOCATION,
- null
- );
- final OssInputSource serdeWithPrefixes =
- MAPPER.readValue(MAPPER.writeValueAsString(withPrefixes), OssInputSource.class);
- Assert.assertEquals(withPrefixes, serdeWithPrefixes);
- }
-
- @Test
- public void testSerdeWithInvalidArgs()
- {
- expectedException.expect(IllegalArgumentException.class);
- // constructor will explode
- new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- EXPECTED_URIS,
- PREFIXES,
- EXPECTED_LOCATION,
- null
- );
- }
-
- @Test
- public void testSerdeWithOtherInvalidArgs()
- {
- expectedException.expect(IllegalArgumentException.class);
- // constructor will explode
- new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- EXPECTED_URIS,
- PREFIXES,
- ImmutableList.of(),
- null
- );
- }
-
- @Test
- public void testSerdeWithOtherOtherInvalidArgs()
- {
- expectedException.expect(IllegalArgumentException.class);
- // constructor will explode
- new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- ImmutableList.of(),
- PREFIXES,
- EXPECTED_LOCATION,
- null
- );
- }
-
- @Test
- public void testWithUrisSplit()
- {
- OssInputSource inputSource = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- EXPECTED_URIS,
- null,
- null,
- null
- );
-
- Stream>> splits = inputSource.createSplits(
- new JsonInputFormat(JSONPathSpec.DEFAULT, null, null),
- null
- );
-
- Assert.assertEquals(EXPECTED_COORDS, splits.map(InputSplit::get).collect(Collectors.toList()));
- }
-
- @Test
- public void testWithPrefixesSplit()
- {
- EasyMock.reset(OSSCLIENT);
- expectListObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_URIS.get(0)), CONTENT);
- expectListObjects(PREFIXES.get(1), ImmutableList.of(EXPECTED_URIS.get(1)), CONTENT);
- EasyMock.replay(OSSCLIENT);
-
- OssInputSource inputSource = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- PREFIXES,
- null,
- null
- );
-
- Stream>> splits = inputSource.createSplits(
- new JsonInputFormat(JSONPathSpec.DEFAULT, null, null),
- new MaxSizeSplitHintSpec(1L) // set maxSplitSize to 1 so that each inputSplit has only one object
- );
-
- Assert.assertEquals(EXPECTED_COORDS, splits.map(InputSplit::get).collect(Collectors.toList()));
- EasyMock.verify(OSSCLIENT);
- }
-
- @Test
- public void testCreateSplitsWithSplitHintSpecRespectingHint()
- {
- EasyMock.reset(OSSCLIENT);
- expectListObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_URIS.get(0)), CONTENT);
- expectListObjects(PREFIXES.get(1), ImmutableList.of(EXPECTED_URIS.get(1)), CONTENT);
- EasyMock.replay(OSSCLIENT);
-
- OssInputSource inputSource = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- PREFIXES,
- null,
- null
- );
-
- Stream>> splits = inputSource.createSplits(
- new JsonInputFormat(JSONPathSpec.DEFAULT, null, null),
- new MaxSizeSplitHintSpec(CONTENT.length * 3L)
- );
-
- Assert.assertEquals(
- ImmutableList.of(EXPECTED_URIS.stream().map(CloudObjectLocation::new).collect(Collectors.toList())),
- splits.map(InputSplit::get).collect(Collectors.toList())
- );
- EasyMock.verify(OSSCLIENT);
- }
-
- @Test
- public void testCreateSplitsWithEmptyObjectsIteratingOnlyNonEmptyObjects()
- {
- EasyMock.reset(OSSCLIENT);
- expectListObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_URIS.get(0)), CONTENT);
- expectListObjects(PREFIXES.get(1), ImmutableList.of(EXPECTED_URIS.get(1)), new byte[0]);
- EasyMock.replay(OSSCLIENT);
-
- OssInputSource inputSource = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- PREFIXES,
- null,
- null
- );
-
- Stream>> splits = inputSource.createSplits(
- new JsonInputFormat(JSONPathSpec.DEFAULT, null, null),
- null
- );
- Assert.assertEquals(
- ImmutableList.of(ImmutableList.of(new CloudObjectLocation(EXPECTED_URIS.get(0)))),
- splits.map(InputSplit::get).collect(Collectors.toList())
- );
- EasyMock.verify(OSSCLIENT);
- }
-
- @Test
- public void testAccessDeniedWhileListingPrefix()
- {
- EasyMock.reset(OSSCLIENT);
- expectListObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_URIS.get(0)), CONTENT);
- expectListObjectsAndThrowAccessDenied(EXPECTED_URIS.get(1));
- EasyMock.replay(OSSCLIENT);
-
- OssInputSource inputSource = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- ImmutableList.of(PREFIXES.get(0), EXPECTED_URIS.get(1)),
- null,
- null
- );
-
- expectedException.expectMessage("Failed to get object summaries from aliyun OSS bucket[bar], prefix[foo/file2.csv]");
- expectedException.expectCause(
- ThrowableMessageMatcher.hasMessage(CoreMatchers.containsString("can't list that bucket"))
- );
-
- inputSource.createSplits(
- new JsonInputFormat(JSONPathSpec.DEFAULT, null, null),
- null
- ).collect(Collectors.toList());
- }
-
- @Test
- public void testReader() throws IOException
- {
- EasyMock.reset(OSSCLIENT);
- expectListObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_URIS.get(0)), CONTENT);
- expectListObjects(EXPECTED_URIS.get(1), ImmutableList.of(EXPECTED_URIS.get(1)), CONTENT);
- expectGetObject(EXPECTED_URIS.get(0));
- expectGetObject(EXPECTED_URIS.get(1));
- EasyMock.replay(OSSCLIENT);
-
- OssInputSource inputSource = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- ImmutableList.of(PREFIXES.get(0), EXPECTED_URIS.get(1)),
- null,
- null
- );
-
- InputRowSchema someSchema = new InputRowSchema(
- new TimestampSpec("time", "auto", null),
- new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2"))),
- ImmutableList.of("count")
- );
-
- InputSourceReader reader = inputSource.reader(
- someSchema,
- new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
- temporaryFolder.newFolder()
- );
-
- CloseableIterator iterator = reader.read();
-
- while (iterator.hasNext()) {
- InputRow nextRow = iterator.next();
- Assert.assertEquals(NOW, nextRow.getTimestamp());
- Assert.assertEquals("hello", nextRow.getDimension("dim1").get(0));
- Assert.assertEquals("world", nextRow.getDimension("dim2").get(0));
- }
-
- EasyMock.verify(OSSCLIENT);
- }
-
- @Test
- public void testCompressedReader() throws IOException
- {
- EasyMock.reset(OSSCLIENT);
- expectListObjects(PREFIXES.get(0), ImmutableList.of(EXPECTED_COMPRESSED_URIS.get(0)), CONTENT);
- expectListObjects(EXPECTED_COMPRESSED_URIS.get(1), ImmutableList.of(EXPECTED_COMPRESSED_URIS.get(1)), CONTENT);
- expectGetObjectCompressed(EXPECTED_COMPRESSED_URIS.get(0));
- expectGetObjectCompressed(EXPECTED_COMPRESSED_URIS.get(1));
- EasyMock.replay(OSSCLIENT);
-
- OssInputSource inputSource = new OssInputSource(
- OSSCLIENT,
- INPUT_DATA_CONFIG,
- null,
- ImmutableList.of(PREFIXES.get(0), EXPECTED_COMPRESSED_URIS.get(1)),
- null,
- null
- );
-
- InputRowSchema someSchema = new InputRowSchema(
- new TimestampSpec("time", "auto", null),
- new DimensionsSpec(DimensionsSpec.getDefaultSchemas(ImmutableList.of("dim1", "dim2"))),
- ImmutableList.of("count")
- );
-
- InputSourceReader reader = inputSource.reader(
- someSchema,
- new CsvInputFormat(ImmutableList.of("time", "dim1", "dim2"), "|", false, null, 0),
- temporaryFolder.newFolder()
- );
-
- CloseableIterator iterator = reader.read();
-
- while (iterator.hasNext()) {
- InputRow nextRow = iterator.next();
- Assert.assertEquals(NOW, nextRow.getTimestamp());
- Assert.assertEquals("hello", nextRow.getDimension("dim1").get(0));
- Assert.assertEquals("world", nextRow.getDimension("dim2").get(0));
- }
-
- EasyMock.verify(OSSCLIENT);
- }
-
- private static void expectListObjects(URI prefix, List uris, byte[] content)
- {
- final ObjectListing result = new ObjectListing();
- result.setBucketName(prefix.getAuthority());
- result.setMaxKeys(uris.size());
- for (URI uri : uris) {
- final String bucket = uri.getAuthority();
- final String key = OssUtils.extractKey(uri);
- final OSSObjectSummary objectSummary = new OSSObjectSummary();
- objectSummary.setBucketName(bucket);
- objectSummary.setKey(key);
- objectSummary.setSize(content.length);
- result.getObjectSummaries().add(objectSummary);
- }
-
- EasyMock.expect(
- OSSCLIENT.listObjects(matchListObjectsRequest(prefix))
- ).andReturn(result).once();
- }
-
- private static void expectListObjectsAndThrowAccessDenied(final URI prefix)
- {
- OSSException boom = new OSSException("oh dang, you can't list that bucket friend");
- boom.setRawResponseError("403");
- EasyMock.expect(
- OSSCLIENT.listObjects(matchListObjectsRequest(prefix))
- ).andThrow(boom).once();
- }
-
- private static void expectGetObject(URI uri)
- {
- final String bucket = uri.getAuthority();
- final String key = OssUtils.extractKey(uri);
-
- OSSObject someObject = new OSSObject();
- someObject.setBucketName(bucket);
- someObject.setKey(key);
- someObject.setObjectContent(new ByteArrayInputStream(CONTENT));
- EasyMock.expect(OSSCLIENT.getObject(EasyMock.anyObject(GetObjectRequest.class))).andReturn(someObject).once();
- }
-
- private static void expectGetObjectCompressed(URI uri) throws IOException
- {
- final String bucket = uri.getAuthority();
- final String key = OssUtils.extractKey(uri);
-
- OSSObject someObject = new OSSObject();
- someObject.setBucketName(bucket);
- someObject.setKey(key);
- ByteArrayOutputStream gzipped = new ByteArrayOutputStream();
- CompressionUtils.gzip(new ByteArrayInputStream(CONTENT), gzipped);
- someObject.setObjectContent(new ByteArrayInputStream(gzipped.toByteArray()));
- EasyMock.expect(OSSCLIENT.getObject(EasyMock.anyObject(GetObjectRequest.class))).andReturn(someObject).once();
- }
-
- private static ListObjectsRequest matchListObjectsRequest(final URI prefixUri)
- {
- // Use an IArgumentMatcher to verify that the request has the correct bucket and prefix.
- EasyMock.reportMatcher(
- new IArgumentMatcher()
- {
- @Override
- public boolean matches(Object argument)
- {
- if (!(argument instanceof ListObjectsRequest)) {
- return false;
- }
-
- final ListObjectsRequest request = (ListObjectsRequest) argument;
- return prefixUri.getAuthority().equals(request.getBucketName())
- && OssUtils.extractKey(prefixUri).equals(request.getPrefix());
- }
-
- @Override
- public void appendTo(StringBuffer buffer)
- {
- buffer.append("");
- }
- }
- );
-
- return null;
- }
-
- public static ObjectMapper createObjectMapper()
- {
- DruidModule baseModule = new TestOssModule();
- final Injector injector = Guice.createInjector(
- new ObjectMapperModule(),
- baseModule
- );
- final ObjectMapper baseMapper = injector.getInstance(ObjectMapper.class);
-
- baseModule.getJacksonModules().forEach(baseMapper::registerModule);
- return baseMapper;
- }
-
- public static class TestOssModule implements DruidModule
- {
- @Override
- public List extends Module> getJacksonModules()
- {
- // Deserializer is need for OSS even though it is injected.
- // See https://github.com/FasterXML/jackson-databind/issues/962.
- return ImmutableList.of(
- new SimpleModule()
- .addDeserializer(OSS.class, new ItemDeserializer())
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
- }
-
- @Provides
- public OSS getOssClient()
- {
- return OSSCLIENT;
- }
- }
-
- public static class ItemDeserializer extends StdDeserializer
- {
- ItemDeserializer()
- {
- this(null);
- }
-
- ItemDeserializer(Class> vc)
- {
- super(vc);
- }
-
- @Override
- public T deserialize(JsonParser jp, DeserializationContext ctxt)
- {
- throw new UnsupportedOperationException();
- }
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentArchiverTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentArchiverTest.java
deleted file mode 100644
index 02a88ec885c9..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentArchiverTest.java
+++ /dev/null
@@ -1,195 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSClient;
-import com.fasterxml.jackson.databind.BeanProperty;
-import com.fasterxml.jackson.databind.DeserializationContext;
-import com.fasterxml.jackson.databind.InjectableValues;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.fasterxml.jackson.databind.module.SimpleModule;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import org.apache.druid.jackson.DefaultObjectMapper;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.timeline.DataSegment;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.BeforeClass;
-import org.junit.Test;
-
-import java.util.Map;
-
-public class OssDataSegmentArchiverTest
-{
- private static final ObjectMapper MAPPER = new DefaultObjectMapper()
- .setInjectableValues(
- new InjectableValues()
- {
- @Override
- public Object findInjectableValue(
- Object valueId,
- DeserializationContext ctxt,
- BeanProperty forProperty,
- Object beanInstance
- )
- {
- return PULLER;
- }
- }
- )
- .registerModule(new SimpleModule("aliyun-oss-archive-test-module").registerSubtypes(OssLoadSpec.class));
- private static final OssDataSegmentArchiverConfig ARCHIVER_CONFIG = new OssDataSegmentArchiverConfig()
- {
- @Override
- public String getArchiveBucket()
- {
- return "archive_bucket";
- }
-
- @Override
- public String getArchiveBaseKey()
- {
- return "archive_base_key";
- }
- };
- private static final OssStorageConfig PUSHER_CONFIG = new OssStorageConfig();
- private static final OSS OSS_CLIENT = EasyMock.createStrictMock(OSSClient.class);
- private static final OssDataSegmentPuller PULLER = new OssDataSegmentPuller(OSS_CLIENT);
- private static final DataSegment SOURCE_SEGMENT = DataSegment
- .builder()
- .binaryVersion(1)
- .dataSource("dataSource")
- .dimensions(ImmutableList.of())
- .interval(Intervals.of("2015/2016"))
- .version("version")
- .loadSpec(ImmutableMap.of(
- "type",
- OssStorageDruidModule.SCHEME_ZIP,
- OssDataSegmentPuller.BUCKET,
- "source_bucket",
- OssDataSegmentPuller.KEY,
- "source_key"
- ))
- .size(0)
- .build();
-
- @BeforeClass
- public static void setUpStatic()
- {
- PUSHER_CONFIG.setPrefix("push_base");
- PUSHER_CONFIG.setBucket("push_bucket");
- }
-
- @Test
- public void testSimpleArchive() throws Exception
- {
- final DataSegment archivedSegment = SOURCE_SEGMENT
- .withLoadSpec(ImmutableMap.of(
- "type",
- OssStorageDruidModule.SCHEME_ZIP,
- OssDataSegmentPuller.BUCKET,
- ARCHIVER_CONFIG.getArchiveBucket(),
- OssDataSegmentPuller.KEY,
- ARCHIVER_CONFIG.getArchiveBaseKey() + "archived"
- ));
- final OssDataSegmentArchiver archiver = new OssDataSegmentArchiver(
- MAPPER,
- OSS_CLIENT,
- ARCHIVER_CONFIG,
- PUSHER_CONFIG
- )
- {
- @Override
- public DataSegment move(DataSegment segment, Map targetLoadSpec)
- {
- return archivedSegment;
- }
- };
- Assert.assertEquals(archivedSegment, archiver.archive(SOURCE_SEGMENT));
- }
-
- @Test
- public void testSimpleArchiveDoesntMove() throws Exception
- {
- final OssDataSegmentArchiver archiver = new OssDataSegmentArchiver(
- MAPPER,
- OSS_CLIENT,
- ARCHIVER_CONFIG,
- PUSHER_CONFIG
- )
- {
- @Override
- public DataSegment move(DataSegment segment, Map targetLoadSpec)
- {
- return SOURCE_SEGMENT;
- }
- };
- Assert.assertNull(archiver.archive(SOURCE_SEGMENT));
- }
-
- @Test
- public void testSimpleRestore() throws Exception
- {
- final DataSegment archivedSegment = SOURCE_SEGMENT
- .withLoadSpec(ImmutableMap.of(
- "type",
- OssStorageDruidModule.SCHEME_ZIP,
- OssDataSegmentPuller.BUCKET,
- ARCHIVER_CONFIG.getArchiveBucket(),
- OssDataSegmentPuller.KEY,
- ARCHIVER_CONFIG.getArchiveBaseKey() + "archived"
- ));
- final OssDataSegmentArchiver archiver = new OssDataSegmentArchiver(
- MAPPER,
- OSS_CLIENT,
- ARCHIVER_CONFIG,
- PUSHER_CONFIG
- )
- {
- @Override
- public DataSegment move(DataSegment segment, Map targetLoadSpec)
- {
- return archivedSegment;
- }
- };
- Assert.assertEquals(archivedSegment, archiver.restore(SOURCE_SEGMENT));
- }
-
- @Test
- public void testSimpleRestoreDoesntMove() throws Exception
- {
- final OssDataSegmentArchiver archiver = new OssDataSegmentArchiver(
- MAPPER,
- OSS_CLIENT,
- ARCHIVER_CONFIG,
- PUSHER_CONFIG
- )
- {
- @Override
- public DataSegment move(DataSegment segment, Map targetLoadSpec)
- {
- return SOURCE_SEGMENT;
- }
- };
- Assert.assertNull(archiver.restore(SOURCE_SEGMENT));
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentKillerTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentKillerTest.java
deleted file mode 100644
index 638348379a27..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentKillerTest.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.ClientException;
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.model.DeleteObjectsRequest;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.StringUtils;
-import org.easymock.EasyMock;
-import org.easymock.EasyMockRunner;
-import org.easymock.EasyMockSupport;
-import org.easymock.Mock;
-import org.junit.Assert;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import java.io.IOException;
-import java.net.URI;
-import java.util.Collections;
-
-@RunWith(EasyMockRunner.class)
-public class OssDataSegmentKillerTest extends EasyMockSupport
-{
- private static final String KEY_1 = "key1";
- private static final String KEY_2 = "key2";
- private static final String TEST_BUCKET = "test_bucket";
- private static final String TEST_PREFIX = "test_prefix";
- private static final URI PREFIX_URI = URI.create(StringUtils.format(OssStorageDruidModule.SCHEME + "://%s/%s", TEST_BUCKET, TEST_PREFIX));
- private static final long TIME_0 = 0L;
- private static final long TIME_1 = 1L;
- private static final int MAX_KEYS = 1;
- private static final Exception RECOVERABLE_EXCEPTION = new ClientException(new IOException("mocked by test case"));
- private static final Exception NON_RECOVERABLE_EXCEPTION = new ClientException(new NullPointerException("mocked by test case"));
-
- @Mock
- private OSS client;
- @Mock
- private OssStorageConfig segmentPusherConfig;
- @Mock
- private OssInputDataConfig inputDataConfig;
-
- private OssDataSegmentKiller segmentKiller;
-
- @Test
- public void test_killAll_accountConfigWithNullBucketAndBaseKey_throwsISEException() throws IOException
- {
- EasyMock.expect(segmentPusherConfig.getBucket()).andReturn(null);
- EasyMock.expectLastCall().atLeastOnce();
- EasyMock.expect(segmentPusherConfig.getPrefix()).andReturn(null);
- EasyMock.expectLastCall().anyTimes();
-
- boolean thrownISEException = false;
-
- try {
-
- EasyMock.replay(client, segmentPusherConfig, inputDataConfig);
-
- segmentKiller = new OssDataSegmentKiller(client, segmentPusherConfig, inputDataConfig);
- segmentKiller.killAll();
- }
- catch (ISE e) {
- thrownISEException = true;
- }
- Assert.assertTrue(thrownISEException);
- EasyMock.verify(client, segmentPusherConfig, inputDataConfig);
- }
-
- @Test
- public void test_killAll_noException_deletesAllSegments() throws IOException
- {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
- OSSObjectSummary objectSummary2 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_2, TIME_1);
-
- OssTestUtils.expectListObjects(
- client,
- PREFIX_URI,
- ImmutableList.of(objectSummary1, objectSummary2)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.setKeys(Collections.singletonList(KEY_1));
- DeleteObjectsRequest deleteRequest2 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest2.setKeys(Collections.singletonList(KEY_2));
-
- OssTestUtils.mockClientDeleteObjects(
- client,
- ImmutableList.of(deleteRequest1, deleteRequest2),
- ImmutableMap.of()
- );
-
- EasyMock.expect(segmentPusherConfig.getBucket()).andReturn(TEST_BUCKET);
- EasyMock.expectLastCall().anyTimes();
- EasyMock.expect(segmentPusherConfig.getPrefix()).andReturn(TEST_PREFIX);
- EasyMock.expectLastCall().anyTimes();
-
- EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_KEYS);
- EasyMock.expectLastCall().anyTimes();
-
- EasyMock.replay(client, segmentPusherConfig, inputDataConfig);
-
- segmentKiller = new OssDataSegmentKiller(client, segmentPusherConfig, inputDataConfig);
- segmentKiller.killAll();
- EasyMock.verify(client, segmentPusherConfig, inputDataConfig);
- }
-
- @Test
- public void test_killAll_recoverableExceptionWhenListingObjects_deletesAllSegments() throws IOException
- {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
-
- OssTestUtils.expectListObjects(
- client,
- PREFIX_URI,
- ImmutableList.of(objectSummary1)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.setKeys(Collections.singletonList(KEY_1));
-
- OssTestUtils.mockClientDeleteObjects(
- client,
- ImmutableList.of(deleteRequest1),
- ImmutableMap.of(deleteRequest1, RECOVERABLE_EXCEPTION)
- );
-
- EasyMock.expect(segmentPusherConfig.getBucket()).andReturn(TEST_BUCKET);
- EasyMock.expectLastCall().anyTimes();
- EasyMock.expect(segmentPusherConfig.getPrefix()).andReturn(TEST_PREFIX);
- EasyMock.expectLastCall().anyTimes();
-
- EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_KEYS);
- EasyMock.expectLastCall().anyTimes();
-
- EasyMock.replay(client, segmentPusherConfig, inputDataConfig);
-
- segmentKiller = new OssDataSegmentKiller(client, segmentPusherConfig, inputDataConfig);
- segmentKiller.killAll();
- EasyMock.verify(client, segmentPusherConfig, inputDataConfig);
- }
-
- @Test
- public void test_killAll_nonrecoverableExceptionWhenListingObjects_deletesAllSegments()
- {
- boolean ioExceptionThrown = false;
- try {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
-
- OssTestUtils.expectListObjects(
- client,
- PREFIX_URI,
- ImmutableList.of(objectSummary1)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.withKeys(ImmutableList.of(KEY_1));
-
- OssTestUtils.mockClientDeleteObjects(
- client,
- ImmutableList.of(),
- ImmutableMap.of(deleteRequest1, NON_RECOVERABLE_EXCEPTION)
- );
-
-
- EasyMock.expect(segmentPusherConfig.getBucket()).andReturn(TEST_BUCKET);
- EasyMock.expectLastCall().anyTimes();
- EasyMock.expect(segmentPusherConfig.getPrefix()).andReturn(TEST_PREFIX);
- EasyMock.expectLastCall().anyTimes();
-
- EasyMock.expect(inputDataConfig.getMaxListingLength()).andReturn(MAX_KEYS);
- EasyMock.expectLastCall().anyTimes();
-
- EasyMock.replay(client, segmentPusherConfig, inputDataConfig);
-
- segmentKiller = new OssDataSegmentKiller(client, segmentPusherConfig, inputDataConfig);
- segmentKiller.killAll();
- }
- catch (IOException e) {
- ioExceptionThrown = true;
- }
-
- Assert.assertTrue(ioExceptionThrown);
- EasyMock.verify(client, segmentPusherConfig, inputDataConfig);
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentMoverTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentMoverTest.java
deleted file mode 100644
index 66c6f25006f1..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentMoverTest.java
+++ /dev/null
@@ -1,266 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSSClient;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.model.CopyObjectRequest;
-import com.aliyun.oss.model.CopyObjectResult;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import com.aliyun.oss.model.PutObjectResult;
-import com.aliyun.oss.model.StorageClass;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.java.util.common.MapUtils;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.partition.NoneShardSpec;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.File;
-import java.util.HashMap;
-import java.util.HashSet;
-import java.util.Map;
-import java.util.Set;
-
-public class OssDataSegmentMoverTest
-{
- private static final DataSegment SOURCE_SEGMENT = new DataSegment(
- "test",
- Intervals.of("2013-01-01/2013-01-02"),
- "1",
- ImmutableMap.of(
- "key",
- "baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip",
- "bucket",
- "main"
- ),
- ImmutableList.of("dim1", "dim1"),
- ImmutableList.of("metric1", "metric2"),
- NoneShardSpec.instance(),
- 0,
- 1
- );
-
- @Test
- public void testMove() throws Exception
- {
- MockClient mockClient = new MockClient();
- OssDataSegmentMover mover = new OssDataSegmentMover(mockClient, new OssStorageConfig());
-
- mockClient.putObject(
- "main",
- "baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip"
- );
-
- DataSegment movedSegment = mover.move(
- SOURCE_SEGMENT,
- ImmutableMap.of("baseKey", "targetBaseKey", "bucket", "archive")
- );
-
- Map targetLoadSpec = movedSegment.getLoadSpec();
- Assert.assertEquals(
- "targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip",
- MapUtils.getString(targetLoadSpec, "key")
- );
- Assert.assertEquals("archive", MapUtils.getString(targetLoadSpec, "bucket"));
- Assert.assertTrue(mockClient.didMove());
- }
-
- @Test
- public void testMoveNoop() throws Exception
- {
- MockClient mockOssClient = new MockClient();
- OssDataSegmentMover mover = new OssDataSegmentMover(mockOssClient, new OssStorageConfig());
-
- mockOssClient.putObject(
- "archive",
- "targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip"
- );
-
- DataSegment movedSegment = mover.move(
- SOURCE_SEGMENT,
- ImmutableMap.of("baseKey", "targetBaseKey", "bucket", "archive")
- );
-
- Map targetLoadSpec = movedSegment.getLoadSpec();
-
- Assert.assertEquals(
- "targetBaseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip",
- MapUtils.getString(targetLoadSpec, "key")
- );
- Assert.assertEquals("archive", MapUtils.getString(targetLoadSpec, "bucket"));
- Assert.assertFalse(mockOssClient.didMove());
- }
-
- @Test(expected = SegmentLoadingException.class)
- public void testMoveException() throws Exception
- {
- MockClient mockClient = new MockClient();
- OssDataSegmentMover mover = new OssDataSegmentMover(mockClient, new OssStorageConfig());
-
- mover.move(
- SOURCE_SEGMENT,
- ImmutableMap.of("baseKey", "targetBaseKey", "bucket", "archive")
- );
- }
-
- @Test
- public void testIgnoresGoneButAlreadyMoved() throws Exception
- {
- MockClient mockOssClient = new MockClient();
- OssDataSegmentMover mover = new OssDataSegmentMover(mockOssClient, new OssStorageConfig());
- mover.move(new DataSegment(
- "test",
- Intervals.of("2013-01-01/2013-01-02"),
- "1",
- ImmutableMap.of(
- "key",
- "baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip",
- "bucket",
- "DOES NOT EXIST"
- ),
- ImmutableList.of("dim1", "dim1"),
- ImmutableList.of("metric1", "metric2"),
- NoneShardSpec.instance(),
- 0,
- 1
- ), ImmutableMap.of("bucket", "DOES NOT EXIST", "baseKey", "baseKey"));
- }
-
- @Test(expected = SegmentLoadingException.class)
- public void testFailsToMoveMissing() throws Exception
- {
- MockClient client = new MockClient();
- OssDataSegmentMover mover = new OssDataSegmentMover(client, new OssStorageConfig());
- mover.move(new DataSegment(
- "test",
- Intervals.of("2013-01-01/2013-01-02"),
- "1",
- ImmutableMap.of(
- "key",
- "baseKey/test/2013-01-01T00:00:00.000Z_2013-01-02T00:00:00.000Z/1/0/index.zip",
- "bucket",
- "DOES NOT EXIST"
- ),
- ImmutableList.of("dim1", "dim1"),
- ImmutableList.of("metric1", "metric2"),
- NoneShardSpec.instance(),
- 0,
- 1
- ), ImmutableMap.of("bucket", "DOES NOT EXIST", "baseKey", "baseKey2"));
- }
-
- private static class MockClient extends OSSClient
- {
- Map> storage = new HashMap<>();
- boolean copied = false;
- boolean deletedOld = false;
-
- private MockClient()
- {
- super("endpoint", "accessKeyId", "keySecret");
- }
-
- public boolean didMove()
- {
- return copied && deletedOld;
- }
-
- @Override
- public boolean doesObjectExist(String bucketName, String objectKey)
- {
- Set objects = storage.get(bucketName);
- return (objects != null && objects.contains(objectKey));
- }
-
- @Override
- public ObjectListing listObjects(ListObjectsRequest listObjectsV2Request)
- {
- final String bucketName = listObjectsV2Request.getBucketName();
- final String objectKey = listObjectsV2Request.getPrefix();
- if (doesObjectExist(bucketName, objectKey)) {
- final OSSObjectSummary objectSummary = new OSSObjectSummary();
- objectSummary.setBucketName(bucketName);
- objectSummary.setKey(objectKey);
- objectSummary.setStorageClass(StorageClass.Standard.name());
-
- final ObjectListing result = new ObjectListing();
- result.setBucketName(bucketName);
- result.setPrefix(objectKey);
- //result.setKeyCount(1);
- result.getObjectSummaries().add(objectSummary);
- result.setTruncated(true);
- return result;
- } else {
- return new ObjectListing();
- }
- }
-
- @Override
- public CopyObjectResult copyObject(CopyObjectRequest copyObjectRequest)
- {
- final String sourceBucketName = copyObjectRequest.getSourceBucketName();
- final String sourceObjectKey = copyObjectRequest.getSourceKey();
- final String destinationBucketName = copyObjectRequest.getDestinationBucketName();
- final String destinationObjectKey = copyObjectRequest.getDestinationKey();
- copied = true;
- if (doesObjectExist(sourceBucketName, sourceObjectKey)) {
- storage.computeIfAbsent(destinationBucketName, k -> new HashSet<>())
- .add(destinationObjectKey);
- return new CopyObjectResult();
- } else {
- final OSSException exception = new OSSException(
- "OssDataSegmentMoverTest",
- "NoSuchKey",
- null,
- null,
- null,
- null,
- null
- );
- throw exception;
- }
- }
-
- @Override
- public void deleteObject(String bucket, String objectKey)
- {
- deletedOld = true;
- storage.get(bucket).remove(objectKey);
- }
-
- public PutObjectResult putObject(String bucketName, String key)
- {
- return putObject(bucketName, key, (File) null);
- }
-
- @Override
- public PutObjectResult putObject(String bucketName, String key, File file)
- {
- storage.computeIfAbsent(bucketName, bName -> new HashSet<>()).add(key);
- return new PutObjectResult();
- }
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPullerTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPullerTest.java
deleted file mode 100644
index 46584cac5e18..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPullerTest.java
+++ /dev/null
@@ -1,205 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSException;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObject;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import org.apache.druid.data.input.impl.CloudObjectLocation;
-import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.io.FileInputStream;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.net.URI;
-import java.nio.charset.StandardCharsets;
-import java.util.Date;
-import java.util.zip.GZIPOutputStream;
-
-/**
- *
- */
-public class OssDataSegmentPullerTest
-{
- @Rule
- public TemporaryFolder temporaryFolder = new TemporaryFolder();
-
- @Test
- public void testSimpleGetVersion() throws IOException
- {
- String bucket = "bucket";
- String keyPrefix = "prefix/dir/0";
- OSS ossClient = EasyMock.createStrictMock(OSS.class);
-
- final OSSObjectSummary objectSummary = new OSSObjectSummary();
- objectSummary.setBucketName(bucket);
- objectSummary.setKey(keyPrefix + "/renames-0.gz");
- objectSummary.setLastModified(new Date(0));
-
- final ObjectListing result = new ObjectListing();
- result.getObjectSummaries().add(objectSummary);
-
- EasyMock.expect(ossClient.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(result)
- .once();
- OssDataSegmentPuller puller = new OssDataSegmentPuller(ossClient);
-
- EasyMock.replay(ossClient);
-
- String version = puller.getVersion(URI.create(StringUtils.format(OssStorageDruidModule.SCHEME + "://%s/%s", bucket, objectSummary.getKey())));
-
- EasyMock.verify(ossClient);
-
- Assert.assertEquals(StringUtils.format("%d", new Date(0).getTime()), version);
- }
-
- @Test
- public void testGZUncompress() throws IOException, SegmentLoadingException
- {
- final String bucket = "bucket";
- final String keyPrefix = "prefix/dir/0";
- final OSS ossClient = EasyMock.createStrictMock(OSS.class);
- final byte[] value = bucket.getBytes(StandardCharsets.UTF_8);
-
- final File tmpFile = temporaryFolder.newFile("gzTest.gz");
-
- try (OutputStream outputStream = new GZIPOutputStream(new FileOutputStream(tmpFile))) {
- outputStream.write(value);
- }
-
- final OSSObject object0 = new OSSObject();
- object0.setBucketName(bucket);
- object0.setKey(keyPrefix + "/renames-0.gz");
- object0.getObjectMetadata().setLastModified(new Date(0));
- object0.setObjectContent(new FileInputStream(tmpFile));
-
- final OSSObjectSummary objectSummary = new OSSObjectSummary();
- objectSummary.setBucketName(bucket);
- objectSummary.setKey(keyPrefix + "/renames-0.gz");
- objectSummary.setLastModified(new Date(0));
-
- final ObjectListing listObjectsResult = new ObjectListing();
- listObjectsResult.getObjectSummaries().add(objectSummary);
-
- final File tmpDir = temporaryFolder.newFolder("gzTestDir");
-
- EasyMock.expect(ossClient.doesObjectExist(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey())))
- .andReturn(true)
- .once();
- EasyMock.expect(ossClient.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(listObjectsResult)
- .once();
- EasyMock.expect(ossClient.getObject(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey())))
- .andReturn(object0)
- .once();
- OssDataSegmentPuller puller = new OssDataSegmentPuller(ossClient);
-
- EasyMock.replay(ossClient);
- FileUtils.FileCopyResult result = puller.getSegmentFiles(
- new CloudObjectLocation(
- bucket,
- object0.getKey()
- ), tmpDir
- );
- EasyMock.verify(ossClient);
-
- Assert.assertEquals(value.length, result.size());
- File expected = new File(tmpDir, "renames-0");
- Assert.assertTrue(expected.exists());
- Assert.assertEquals(value.length, expected.length());
- }
-
- @Test
- public void testGZUncompressRetries() throws IOException, SegmentLoadingException
- {
- final String bucket = "bucket";
- final String keyPrefix = "prefix/dir/0";
- final OSS ossClient = EasyMock.createStrictMock(OSS.class);
- final byte[] value = bucket.getBytes(StandardCharsets.UTF_8);
-
- final File tmpFile = temporaryFolder.newFile("gzTest.gz");
-
- try (OutputStream outputStream = new GZIPOutputStream(new FileOutputStream(tmpFile))) {
- outputStream.write(value);
- }
-
- OSSObject object0 = new OSSObject();
-
- object0.setBucketName(bucket);
- object0.setKey(keyPrefix + "/renames-0.gz");
- object0.getObjectMetadata().setLastModified(new Date(0));
- object0.setObjectContent(new FileInputStream(tmpFile));
-
- final OSSObjectSummary objectSummary = new OSSObjectSummary();
- objectSummary.setBucketName(bucket);
- objectSummary.setKey(keyPrefix + "/renames-0.gz");
- objectSummary.setLastModified(new Date(0));
-
- final ObjectListing listObjectsResult = new ObjectListing();
- listObjectsResult.getObjectSummaries().add(objectSummary);
-
- File tmpDir = temporaryFolder.newFolder("gzTestDir");
-
- OSSException exception = new OSSException("OssDataSegmentPullerTest", "NoSuchKey", null, null, null, null, null);
- EasyMock.expect(ossClient.doesObjectExist(EasyMock.eq(object0.getBucketName()), EasyMock.eq(object0.getKey())))
- .andReturn(true)
- .once();
- EasyMock.expect(ossClient.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(listObjectsResult)
- .once();
- EasyMock.expect(ossClient.getObject(EasyMock.eq(bucket), EasyMock.eq(object0.getKey())))
- .andThrow(exception)
- .once();
- EasyMock.expect(ossClient.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(listObjectsResult)
- .once();
- EasyMock.expect(ossClient.getObject(EasyMock.eq(bucket), EasyMock.eq(object0.getKey())))
- .andReturn(object0)
- .once();
- OssDataSegmentPuller puller = new OssDataSegmentPuller(ossClient);
-
- EasyMock.replay(ossClient);
- FileUtils.FileCopyResult result = puller.getSegmentFiles(
- new CloudObjectLocation(
- bucket,
- object0.getKey()
- ), tmpDir
- );
- EasyMock.verify(ossClient);
-
- Assert.assertEquals(value.length, result.size());
- File expected = new File(tmpDir, "renames-0");
- Assert.assertTrue(expected.exists());
- Assert.assertEquals(value.length, expected.length());
- }
-
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPusherConfigTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPusherConfigTest.java
deleted file mode 100644
index d558a08068cd..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPusherConfigTest.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.druid.jackson.DefaultObjectMapper;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-
-public class OssDataSegmentPusherConfigTest
-{
- private static final ObjectMapper JSON_MAPPER = new DefaultObjectMapper();
-
- @Test
- public void testSerialization() throws IOException
- {
- String jsonConfig = "{\"bucket\":\"bucket1\",\"prefix\":\"dataSource1\"}";
-
- OssStorageConfig config = JSON_MAPPER.readValue(jsonConfig, OssStorageConfig.class);
- Assert.assertEquals(jsonConfig, JSON_MAPPER.writeValueAsString(config));
- }
-
- @Test
- public void testSerializationWithDefaults() throws IOException
- {
- String jsonConfig = "{\"bucket\":\"bucket1\",\"prefix\":\"dataSource1\"}";
- String expectedJsonConfig = "{\"bucket\":\"bucket1\",\"prefix\":\"dataSource1\"}";
-
- OssStorageConfig config = JSON_MAPPER.readValue(jsonConfig, OssStorageConfig.class);
- Assert.assertEquals(expectedJsonConfig, JSON_MAPPER.writeValueAsString(config));
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPusherTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPusherTest.java
deleted file mode 100644
index b3d91c7af548..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssDataSegmentPusherTest.java
+++ /dev/null
@@ -1,125 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.model.PutObjectResult;
-import com.google.common.io.Files;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.partition.NoneShardSpec;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.HashMap;
-import java.util.regex.Pattern;
-
-/**
- *
- */
-public class OssDataSegmentPusherTest
-{
- private static class ValueContainer
- {
- private T value;
-
- public T getValue()
- {
- return value;
- }
-
- public void setValue(T value)
- {
- this.value = value;
- }
- }
-
- @Rule
- public final TemporaryFolder tempFolder = new TemporaryFolder();
-
- @Test
- public void testPush() throws Exception
- {
- testPushInternal(false, "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/index\\.zip");
- }
-
- @Test
- public void testPushUseUniquePath() throws Exception
- {
- testPushInternal(
- true,
- "key/foo/2015-01-01T00:00:00\\.000Z_2016-01-01T00:00:00\\.000Z/0/0/[A-Za-z0-9-]{36}/index\\.zip"
- );
- }
-
- private void testPushInternal(boolean useUniquePath, String matcher) throws Exception
- {
- OSS client = EasyMock.createStrictMock(OSS.class);
-
- EasyMock.expect(client.putObject(EasyMock.anyObject()))
- .andReturn(new PutObjectResult())
- .once();
-
- EasyMock.replay(client);
-
- OssStorageConfig config = new OssStorageConfig();
- config.setBucket("bucket");
- config.setPrefix("key");
-
- OssDataSegmentPusher pusher = new OssDataSegmentPusher(client, config);
-
- // Create a mock segment on disk
- File tmp = tempFolder.newFile("version.bin");
-
- final byte[] data = new byte[]{0x0, 0x0, 0x0, 0x1};
- Files.write(data, tmp);
- final long size = data.length;
-
- DataSegment segmentToPush = new DataSegment(
- "foo",
- Intervals.of("2015/2016"),
- "0",
- new HashMap<>(),
- new ArrayList<>(),
- new ArrayList<>(),
- NoneShardSpec.instance(),
- 0,
- size
- );
-
- DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush, useUniquePath);
-
- Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
- Assert.assertEquals(1, (int) segment.getBinaryVersion());
- Assert.assertEquals("bucket", segment.getLoadSpec().get("bucket"));
- Assert.assertTrue(
- segment.getLoadSpec().get("key").toString(),
- Pattern.compile(matcher).matcher(segment.getLoadSpec().get("key").toString()).matches()
- );
- Assert.assertEquals("oss_zip", segment.getLoadSpec().get("type"));
-
- EasyMock.verify(client);
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssObjectSummaryIteratorTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssObjectSummaryIteratorTest.java
deleted file mode 100644
index d124b6bf6f9c..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssObjectSummaryIteratorTest.java
+++ /dev/null
@@ -1,276 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.OSSClient;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.Iterables;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.List;
-import java.util.stream.Collectors;
-
-public class OssObjectSummaryIteratorTest
-{
- private static final ImmutableList TEST_OBJECTS =
- ImmutableList.of(
- makeObjectSummary("b", "foo", 10L),
- makeObjectSummary("b", "foo/", 0L), // directory
- makeObjectSummary("b", "foo/bar1", 10L),
- makeObjectSummary("b", "foo/bar2", 10L),
- makeObjectSummary("b", "foo/bar3", 10L),
- makeObjectSummary("b", "foo/bar4", 10L),
- makeObjectSummary("b", "foo/bar5", 0L), // empty object
- makeObjectSummary("b", "foo/baz", 10L),
- makeObjectSummary("bucketnotmine", "a/different/bucket", 10L),
- makeObjectSummary("b", "foo/bar/", 0L) // another directory at the end of list
- );
-
- @Test
- public void testSingleObject()
- {
- test(
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/baz"),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/baz"),
- 5
- );
- }
-
- @Test
- public void testMultiObjectOneKeyAtATime()
- {
- test(
- ImmutableList.of(
- OssStorageDruidModule.SCHEME + "://b/foo/bar1",
- OssStorageDruidModule.SCHEME + "://b/foo/bar2",
- OssStorageDruidModule.SCHEME + "://b/foo/bar3",
- OssStorageDruidModule.SCHEME + "://b/foo/bar4",
- OssStorageDruidModule.SCHEME + "://b/foo/baz"
- ),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/"),
- 1
- );
- }
-
- @Test
- public void testMultiObjectTwoKeysAtATime()
- {
- test(
- ImmutableList.of(
- OssStorageDruidModule.SCHEME + "://b/foo/bar1",
- OssStorageDruidModule.SCHEME + "://b/foo/bar2",
- OssStorageDruidModule.SCHEME + "://b/foo/bar3",
- OssStorageDruidModule.SCHEME + "://b/foo/bar4",
- OssStorageDruidModule.SCHEME + "://b/foo/baz"
- ),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/"),
- 2
- );
- }
-
- @Test
- public void testMultiObjectTenKeysAtATime()
- {
- test(
- ImmutableList.of(
- OssStorageDruidModule.SCHEME + "://b/foo/bar1",
- OssStorageDruidModule.SCHEME + "://b/foo/bar2",
- OssStorageDruidModule.SCHEME + "://b/foo/bar3",
- OssStorageDruidModule.SCHEME + "://b/foo/bar4",
- OssStorageDruidModule.SCHEME + "://b/foo/baz"
- ),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/"),
- 10
- );
- }
-
- @Test
- public void testPrefixInMiddleOfKey()
- {
- test(
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/bar1", OssStorageDruidModule.SCHEME + "://b/foo/bar2", OssStorageDruidModule.SCHEME + "://b/foo/bar3", OssStorageDruidModule.SCHEME + "://b/foo/bar4"),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/bar"),
- 10
- );
- }
-
- @Test
- public void testNoPath()
- {
- test(
- ImmutableList.of(
- OssStorageDruidModule.SCHEME + "://b/foo",
- OssStorageDruidModule.SCHEME + "://b/foo/bar1",
- OssStorageDruidModule.SCHEME + "://b/foo/bar2",
- OssStorageDruidModule.SCHEME + "://b/foo/bar3",
- OssStorageDruidModule.SCHEME + "://b/foo/bar4",
- OssStorageDruidModule.SCHEME + "://b/foo/baz"
- ),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b"),
- 10
- );
- }
-
- @Test
- public void testSlashPath()
- {
- test(
- ImmutableList.of(
- OssStorageDruidModule.SCHEME + "://b/foo",
- OssStorageDruidModule.SCHEME + "://b/foo/bar1",
- OssStorageDruidModule.SCHEME + "://b/foo/bar2",
- OssStorageDruidModule.SCHEME + "://b/foo/bar3",
- OssStorageDruidModule.SCHEME + "://b/foo/bar4",
- OssStorageDruidModule.SCHEME + "://b/foo/baz"
- ),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/"),
- 10
- );
- }
-
- @Test
- public void testDifferentBucket()
- {
- test(
- ImmutableList.of(),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://bx/foo/"),
- 10
- );
- }
-
- @Test
- public void testWithMultiplePrefixesReturningAllNonEmptyObjectsStartingWithOneOfPrefixes()
- {
- test(
- ImmutableList.of(
- OssStorageDruidModule.SCHEME + "://b/foo/bar1",
- OssStorageDruidModule.SCHEME + "://b/foo/bar2",
- OssStorageDruidModule.SCHEME + "://b/foo/bar3",
- OssStorageDruidModule.SCHEME + "://b/foo/bar4",
- OssStorageDruidModule.SCHEME + "://b/foo/baz"
- ),
- ImmutableList.of(OssStorageDruidModule.SCHEME + "://b/foo/bar", OssStorageDruidModule.SCHEME + "://b/foo/baz"),
- 10
- );
- }
-
- private static void test(
- final List expectedUris,
- final List prefixes,
- final int maxListingLength
- )
- {
- final List expectedObjects = new ArrayList<>();
-
- // O(N^2) but who cares -- the list is short.
- for (final String uri : expectedUris) {
- final List matches = TEST_OBJECTS.stream()
- .filter(
- summary ->
- OssUtils.summaryToUri(summary).toString().equals(uri)
- )
- .collect(Collectors.toList());
-
- expectedObjects.add(Iterables.getOnlyElement(matches));
- }
-
- final List actualObjects = ImmutableList.copyOf(
- OssUtils.objectSummaryIterator(
- makeMockClient(TEST_OBJECTS),
- prefixes.stream().map(URI::create).collect(Collectors.toList()),
- maxListingLength
- )
- );
-
- Assert.assertEquals(
- prefixes.toString(),
- expectedObjects.stream().map(OssUtils::summaryToUri).collect(Collectors.toList()),
- actualObjects.stream().map(OssUtils::summaryToUri).collect(Collectors.toList())
- );
- }
-
- /**
- * Makes a mock OSS client that handles enough of "listObjects" to test the functionality of the
- * {@link OssObjectSummaryIterator} class.
- */
- private static OSS makeMockClient(
- final List objects
- )
- {
- return new OSSClient("endpoint", "accessKey", "keySecret")
- {
- @Override
- public ObjectListing listObjects(final ListObjectsRequest request)
- {
- // Continuation token is an index in the "objects" list.q
- final String continuationToken = request.getMarker();
- final int startIndex = continuationToken == null ? 0 : Integer.parseInt(continuationToken);
-
- // Find matching objects.
- final List summaries = new ArrayList<>();
- int nextIndex = -1;
-
- for (int i = startIndex; i < objects.size(); i++) {
- final OSSObjectSummary summary = objects.get(i);
-
- if (summary.getBucketName().equals(request.getBucketName())
- && summary.getKey().startsWith(request.getPrefix())) {
-
- if (summaries.size() == request.getMaxKeys()) {
- // We reached our max key limit; set nextIndex (which will lead to a result with truncated = true).
- nextIndex = i;
- break;
- }
-
- // Generate a summary.
- summaries.add(summary);
- }
- }
-
- // Generate the result.
- final ObjectListing retVal = new ObjectListing();
- retVal.getObjectSummaries().addAll(summaries);
-
- if (nextIndex >= 0) {
- retVal.setTruncated(true);
- retVal.setNextMarker(String.valueOf(nextIndex));
- }
-
- return retVal;
- }
- };
- }
-
- private static OSSObjectSummary makeObjectSummary(final String bucket, final String key, final long size)
- {
- final OSSObjectSummary summary = new OSSObjectSummary();
- summary.setBucketName(bucket);
- summary.setKey(key);
- summary.setSize(size);
- return summary;
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTaskLogsTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTaskLogsTest.java
deleted file mode 100644
index 1264a0fe9d7a..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTaskLogsTest.java
+++ /dev/null
@@ -1,336 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.ClientException;
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.model.AccessControlList;
-import com.aliyun.oss.model.DeleteObjectsRequest;
-import com.aliyun.oss.model.Grant;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.Owner;
-import com.aliyun.oss.model.PutObjectRequest;
-import com.aliyun.oss.model.PutObjectResult;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableMap;
-import org.apache.druid.common.utils.CurrentTimeMillisSupplier;
-import org.apache.druid.java.util.common.StringUtils;
-import org.easymock.EasyMock;
-import org.easymock.EasyMockRunner;
-import org.easymock.EasyMockSupport;
-import org.easymock.Mock;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-import org.junit.runner.RunWith;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URI;
-import java.util.ArrayList;
-import java.util.Collections;
-import java.util.List;
-
-@RunWith(EasyMockRunner.class)
-public class OssTaskLogsTest extends EasyMockSupport
-{
-
- private static final String KEY_1 = "key1";
- private static final String KEY_2 = "key2";
- private static final String TEST_BUCKET = "test_bucket";
- private static final String TEST_PREFIX = "test_prefix";
- private static final URI PREFIX_URI = URI.create(StringUtils.format("oss://%s/%s", TEST_BUCKET, TEST_PREFIX));
- private static final long TIME_0 = 0L;
- private static final long TIME_1 = 1L;
- private static final long TIME_NOW = 2L;
- private static final long TIME_FUTURE = 3L;
- private static final int MAX_KEYS = 1;
- private static final Exception RECOVERABLE_EXCEPTION = new ClientException(new IOException());
- private static final Exception NON_RECOVERABLE_EXCEPTION = new ClientException(new NullPointerException());
-
- @Mock
- private CurrentTimeMillisSupplier timeSupplier;
- @Mock
- private OSS ossClient;
-
- @Rule
- public final TemporaryFolder tempFolder = new TemporaryFolder();
-
- @Test
- public void testTaskLogsPushWithAclDisabled() throws Exception
- {
- String ownerId = "test_owner";
- String ownerDisplayName = "test_owner";
-
- List grantList = testPushInternal(true, ownerId, ownerDisplayName);
-
- Assert.assertNotNull("Grant list should not be null", grantList);
- Assert.assertEquals("Grant list should be empty as ACL is disabled", 0, grantList.size());
- }
-
- @Test
- public void test_killAll_noException_deletesAllTaskLogs() throws IOException
- {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
- OSSObjectSummary objectSummary2 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_2, TIME_1);
-
- EasyMock.expect(timeSupplier.getAsLong()).andReturn(TIME_NOW);
-
- OssTestUtils.expectListObjects(
- ossClient,
- PREFIX_URI,
- ImmutableList.of(objectSummary1, objectSummary2)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.setKeys(Collections.singletonList(KEY_1));
- DeleteObjectsRequest deleteRequest2 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest2.setKeys(Collections.singletonList(KEY_2));
-
- OssTestUtils.mockClientDeleteObjects(
- ossClient,
- ImmutableList.of(deleteRequest1, deleteRequest2),
- ImmutableMap.of()
- );
-
- EasyMock.replay(ossClient, timeSupplier);
-
- OssTaskLogsConfig config = new OssTaskLogsConfig();
- config.setBucket(TEST_BUCKET);
- config.setPrefix(TEST_PREFIX);
- OssInputDataConfig inputDataConfig = new OssInputDataConfig();
- inputDataConfig.setMaxListingLength(MAX_KEYS);
- OssTaskLogs taskLogs = new OssTaskLogs(ossClient, config, inputDataConfig, timeSupplier);
- taskLogs.killAll();
-
- EasyMock.verify(ossClient, timeSupplier);
- }
-
- @Test
- public void test_killAll_recoverableExceptionWhenDeletingObjects_deletesAllTaskLogs() throws IOException
- {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
-
- EasyMock.expect(timeSupplier.getAsLong()).andReturn(TIME_NOW);
-
- OssTestUtils.expectListObjects(
- ossClient,
- PREFIX_URI,
- ImmutableList.of(objectSummary1)
- );
-
- DeleteObjectsRequest expectedRequest = new DeleteObjectsRequest(TEST_BUCKET);
- expectedRequest.setKeys(Collections.singletonList(KEY_1));
- OssTestUtils.mockClientDeleteObjects(
- ossClient,
- ImmutableList.of(expectedRequest),
- ImmutableMap.of(expectedRequest, RECOVERABLE_EXCEPTION)
- );
-
- EasyMock.replay(ossClient, timeSupplier);
-
- OssTaskLogsConfig config = new OssTaskLogsConfig();
- config.setBucket(TEST_BUCKET);
- config.setPrefix(TEST_PREFIX);
- OssInputDataConfig inputDataConfig = new OssInputDataConfig();
- inputDataConfig.setMaxListingLength(MAX_KEYS);
- OssTaskLogs taskLogs = new OssTaskLogs(ossClient, config, inputDataConfig, timeSupplier);
- taskLogs.killAll();
-
- EasyMock.verify(ossClient, timeSupplier);
- }
-
- @Test
- public void test_killAll_nonrecoverableExceptionWhenListingObjects_doesntDeleteAnyTaskLogs()
- {
- boolean ioExceptionThrown = false;
- try {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
- EasyMock.expect(timeSupplier.getAsLong()).andReturn(TIME_NOW);
- OssTestUtils.expectListObjects(
- ossClient,
- PREFIX_URI,
- ImmutableList.of(objectSummary1)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.setKeys(Collections.singletonList(KEY_1));
- OssTestUtils.mockClientDeleteObjects(
- ossClient,
- ImmutableList.of(),
- ImmutableMap.of(deleteRequest1, NON_RECOVERABLE_EXCEPTION)
- );
-
- EasyMock.replay(ossClient, timeSupplier);
-
- OssTaskLogsConfig config = new OssTaskLogsConfig();
- config.setBucket(TEST_BUCKET);
- config.setPrefix(TEST_PREFIX);
- OssInputDataConfig inputDataConfig = new OssInputDataConfig();
- inputDataConfig.setMaxListingLength(MAX_KEYS);
- OssTaskLogs taskLogs = new OssTaskLogs(ossClient, config, inputDataConfig, timeSupplier);
- taskLogs.killAll();
- }
- catch (IOException e) {
- ioExceptionThrown = true;
- }
-
- Assert.assertTrue(ioExceptionThrown);
-
- EasyMock.verify(ossClient, timeSupplier);
- }
-
- @Test
- public void test_killOlderThan_noException_deletesOnlyTaskLogsOlderThan() throws IOException
- {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
- OSSObjectSummary objectSummary2 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_2, TIME_FUTURE);
-
- OssTestUtils.expectListObjects(
- ossClient,
- PREFIX_URI,
- ImmutableList.of(objectSummary1, objectSummary2)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.setKeys(Collections.singletonList(KEY_1));
-
- OssTestUtils.mockClientDeleteObjects(ossClient, ImmutableList.of(deleteRequest1), ImmutableMap.of());
-
- EasyMock.replay(ossClient, timeSupplier);
-
- OssTaskLogsConfig config = new OssTaskLogsConfig();
- config.setBucket(TEST_BUCKET);
- config.setPrefix(TEST_PREFIX);
- OssInputDataConfig inputDataConfig = new OssInputDataConfig();
- inputDataConfig.setMaxListingLength(MAX_KEYS);
- OssTaskLogs taskLogs = new OssTaskLogs(ossClient, config, inputDataConfig, timeSupplier);
- taskLogs.killOlderThan(TIME_NOW);
-
- EasyMock.verify(ossClient, timeSupplier);
- }
-
- @Test
- public void test_killOlderThan_recoverableExceptionWhenListingObjects_deletesAllTaskLogs() throws IOException
- {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
-
- OssTestUtils.expectListObjects(
- ossClient,
- PREFIX_URI,
- ImmutableList.of(objectSummary1)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.setKeys(Collections.singletonList(KEY_1));
-
- OssTestUtils.mockClientDeleteObjects(
- ossClient,
- ImmutableList.of(deleteRequest1),
- ImmutableMap.of(deleteRequest1, RECOVERABLE_EXCEPTION)
- );
-
- EasyMock.replay(ossClient, timeSupplier);
-
- OssTaskLogsConfig config = new OssTaskLogsConfig();
- config.setBucket(TEST_BUCKET);
- config.setPrefix(TEST_PREFIX);
- OssInputDataConfig inputDataConfig = new OssInputDataConfig();
- inputDataConfig.setMaxListingLength(MAX_KEYS);
- OssTaskLogs taskLogs = new OssTaskLogs(ossClient, config, inputDataConfig, timeSupplier);
- taskLogs.killOlderThan(TIME_NOW);
-
- EasyMock.verify(ossClient, timeSupplier);
- }
-
- @Test
- public void test_killOlderThan_nonrecoverableExceptionWhenListingObjects_doesntDeleteAnyTaskLogs()
- {
- boolean ioExceptionThrown = false;
- try {
- OSSObjectSummary objectSummary1 = OssTestUtils.newOSSObjectSummary(TEST_BUCKET, KEY_1, TIME_0);
- OssTestUtils.expectListObjects(
- ossClient,
- PREFIX_URI,
- ImmutableList.of(objectSummary1)
- );
-
- DeleteObjectsRequest deleteRequest1 = new DeleteObjectsRequest(TEST_BUCKET);
- deleteRequest1.setKeys(Collections.singletonList(KEY_1));
- OssTestUtils.mockClientDeleteObjects(
- ossClient,
- ImmutableList.of(),
- ImmutableMap.of(deleteRequest1, NON_RECOVERABLE_EXCEPTION)
- );
-
- EasyMock.replay(ossClient, timeSupplier);
-
- OssTaskLogsConfig config = new OssTaskLogsConfig();
- config.setBucket(TEST_BUCKET);
- config.setPrefix(TEST_PREFIX);
- OssInputDataConfig inputDataConfig = new OssInputDataConfig();
- inputDataConfig.setMaxListingLength(MAX_KEYS);
- OssTaskLogs taskLogs = new OssTaskLogs(ossClient, config, inputDataConfig, timeSupplier);
- taskLogs.killOlderThan(TIME_NOW);
- }
- catch (IOException e) {
- ioExceptionThrown = true;
- }
-
- Assert.assertTrue(ioExceptionThrown);
-
- EasyMock.verify(ossClient, timeSupplier);
- }
-
- private List testPushInternal(boolean disableAcl, String ownerId, String ownerDisplayName) throws Exception
- {
- EasyMock.expect(ossClient.putObject(EasyMock.anyObject()))
- .andReturn(new PutObjectResult())
- .once();
-
- AccessControlList aclExpected = new AccessControlList();
- aclExpected.setOwner(new Owner(ownerId, ownerDisplayName));
-
- EasyMock.expect(ossClient.getBucketAcl(TEST_BUCKET))
- .andReturn(aclExpected)
- .once();
-
- EasyMock.expect(ossClient.putObject(EasyMock.anyObject(PutObjectRequest.class)))
- .andReturn(new PutObjectResult())
- .once();
-
- EasyMock.replay(ossClient);
-
- OssTaskLogsConfig config = new OssTaskLogsConfig();
- config.setDisableAcl(disableAcl);
- config.setBucket(TEST_BUCKET);
- CurrentTimeMillisSupplier timeSupplier = new CurrentTimeMillisSupplier();
- OssInputDataConfig inputDataConfig = new OssInputDataConfig();
- OssTaskLogs taskLogs = new OssTaskLogs(ossClient, config, inputDataConfig, timeSupplier);
-
- String taskId = "index_test-datasource_2019-06-18T13:30:28.887Z";
- File logFile = tempFolder.newFile("test_log_file");
-
- taskLogs.pushTaskLog(taskId, logFile);
-
- return new ArrayList<>(aclExpected.getGrants());
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTestUtils.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTestUtils.java
deleted file mode 100644
index 35ef96663e07..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTestUtils.java
+++ /dev/null
@@ -1,177 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.model.DeleteObjectsRequest;
-import com.aliyun.oss.model.DeleteObjectsResult;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.StringUtils;
-import org.easymock.EasyMock;
-import org.easymock.EasyMockSupport;
-import org.easymock.IArgumentMatcher;
-import org.easymock.IExpectationSetters;
-import org.joda.time.DateTime;
-
-import java.net.URI;
-import java.util.Date;
-import java.util.HashMap;
-import java.util.List;
-import java.util.Map;
-
-public class OssTestUtils extends EasyMockSupport
-{
- private static final DateTime NOW = DateTimes.nowUtc();
- private static final byte[] CONTENT =
- StringUtils.toUtf8(StringUtils.format("%d,hello,world", NOW.getMillis()));
-
- public static DeleteObjectsRequest deleteObjectsRequestArgumentMatcher(DeleteObjectsRequest deleteObjectsRequest)
- {
- EasyMock.reportMatcher(new IArgumentMatcher()
- {
- @Override
- public boolean matches(Object argument)
- {
-
- boolean matches = argument instanceof DeleteObjectsRequest
- && deleteObjectsRequest.getBucketName()
- .equals(((DeleteObjectsRequest) argument).getBucketName())
- && deleteObjectsRequest.getKeys().size() == ((DeleteObjectsRequest) argument).getKeys()
- .size();
- if (matches) {
- List expectedKeysAndVersions = deleteObjectsRequest.getKeys();
- List actualKeysAndVersions = ((DeleteObjectsRequest) argument).getKeys();
- matches = expectedKeysAndVersions.equals(actualKeysAndVersions);
- }
- return matches;
- }
-
- @Override
- public void appendTo(StringBuffer buffer)
- {
- String str = "DeleteObjectsRequest(\"bucketName:\" \""
- + deleteObjectsRequest.getBucketName()
- + "\", \"keys:\""
- + deleteObjectsRequest.getKeys()
- + "\")";
- buffer.append(str);
- }
- });
- return null;
- }
-
- public static void expectListObjects(
- OSS client,
- URI prefix,
- List objectSummaries
- )
- {
- final ObjectListing result = new ObjectListing();
- result.setBucketName(prefix.getAuthority());
- //result.setsetKeyCount(objectSummaries.size());
- for (OSSObjectSummary objectSummary : objectSummaries) {
- result.getObjectSummaries().add(objectSummary);
- }
-
- EasyMock.expect(
- client.listObjects(matchListObjectsRequest(prefix))
- ).andReturn(result).once();
- }
-
- public static void mockClientDeleteObjects(
- OSS client,
- List deleteRequestsExpected,
- Map requestToException
- )
- {
- Map> requestToResultExpectationSetter = new HashMap<>();
-
- for (Map.Entry requestsAndErrors : requestToException.entrySet()) {
- DeleteObjectsRequest request = requestsAndErrors.getKey();
- Exception exception = requestsAndErrors.getValue();
- IExpectationSetters resultExpectationSetter = requestToResultExpectationSetter.get(request);
- if (resultExpectationSetter == null) {
- client.deleteObjects(
- OssTestUtils.deleteObjectsRequestArgumentMatcher(request));
- resultExpectationSetter = EasyMock.expectLastCall().andThrow(exception);
- requestToResultExpectationSetter.put(request, resultExpectationSetter);
- } else {
- resultExpectationSetter.andThrow(exception);
- }
- }
-
- for (DeleteObjectsRequest request : deleteRequestsExpected) {
- IExpectationSetters resultExpectationSetter = requestToResultExpectationSetter.get(request);
- if (resultExpectationSetter == null) {
- client.deleteObjects(OssTestUtils.deleteObjectsRequestArgumentMatcher(request));
- resultExpectationSetter = EasyMock.expectLastCall();
- requestToResultExpectationSetter.put(request, resultExpectationSetter);
- }
- resultExpectationSetter.andReturn(new DeleteObjectsResult());
- }
- }
-
- public static ListObjectsRequest matchListObjectsRequest(final URI prefixUri)
- {
- // Use an IArgumentMatcher to verify that the request has the correct bucket and prefix.
- EasyMock.reportMatcher(
- new IArgumentMatcher()
- {
- @Override
- public boolean matches(Object argument)
- {
- if (!(argument instanceof ListObjectsRequest)) {
- return false;
- }
-
- final ListObjectsRequest request = (ListObjectsRequest) argument;
- return prefixUri.getAuthority().equals(request.getBucketName())
- && OssUtils.extractKey(prefixUri).equals(request.getPrefix());
- }
-
- @Override
- public void appendTo(StringBuffer buffer)
- {
- buffer.append("");
- }
- }
- );
-
- return null;
- }
-
- public static OSSObjectSummary newOSSObjectSummary(
- String bucket,
- String key,
- long lastModifiedTimestamp
- )
- {
- OSSObjectSummary objectSummary = new OSSObjectSummary();
- objectSummary.setBucketName(bucket);
- objectSummary.setKey(key);
- objectSummary.setLastModified(new Date(lastModifiedTimestamp));
- objectSummary.setETag("etag");
- objectSummary.setSize(CONTENT.length);
- return objectSummary;
- }
-}
diff --git a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTimestampVersionedDataFinderTest.java b/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTimestampVersionedDataFinderTest.java
deleted file mode 100644
index 8443d2f4abc6..000000000000
--- a/extensions-contrib/aliyun-oss-extensions/src/test/java/org/apache/druid/storage/aliyun/OssTimestampVersionedDataFinderTest.java
+++ /dev/null
@@ -1,178 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.aliyun;
-
-import com.aliyun.oss.OSS;
-import com.aliyun.oss.model.ListObjectsRequest;
-import com.aliyun.oss.model.OSSObjectSummary;
-import com.aliyun.oss.model.ObjectListing;
-import org.apache.druid.java.util.common.StringUtils;
-import org.easymock.EasyMock;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.net.URI;
-import java.util.Date;
-import java.util.regex.Pattern;
-
-public class OssTimestampVersionedDataFinderTest
-{
-
- @Test
- public void testSimpleLatestVersion()
- {
- String bucket = "bucket";
- String keyPrefix = "prefix/dir/0";
- OSS client = EasyMock.createStrictMock(OSS.class);
-
- OSSObjectSummary object0 = new OSSObjectSummary(), object1 = new OSSObjectSummary();
-
- object0.setBucketName(bucket);
- object0.setKey(keyPrefix + "/renames-0.gz");
- object0.setLastModified(new Date(0));
- object0.setSize(10);
-
- object1.setBucketName(bucket);
- object1.setKey(keyPrefix + "/renames-1.gz");
- object1.setLastModified(new Date(1));
- object1.setSize(10);
-
- final ObjectListing result = new ObjectListing();
- result.getObjectSummaries().add(object0);
- result.getObjectSummaries().add(object1);
- result.setTruncated(false);
-
- EasyMock.expect(client.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(result)
- .once();
- OssTimestampVersionedDataFinder finder = new OssTimestampVersionedDataFinder(client);
-
- Pattern pattern = Pattern.compile("renames-[0-9]*\\.gz");
-
- EasyMock.replay(client);
-
-
- URI latest = finder.getLatestVersion(URI.create(StringUtils.format("%s://%s/%s", OssStorageDruidModule.SCHEME, bucket, keyPrefix)), pattern);
-
- EasyMock.verify(client);
-
- URI expected = URI.create(StringUtils.format("%s://%s/%s", OssStorageDruidModule.SCHEME, bucket, object1.getKey()));
-
- Assert.assertEquals(expected, latest);
- }
-
- @Test
- public void testMissing()
- {
- String bucket = "bucket";
- String keyPrefix = "prefix/dir/0";
- OSS oss = EasyMock.createStrictMock(OSS.class);
-
- final ObjectListing result = new ObjectListing();
- result.setTruncated(false);
-
- EasyMock.expect(oss.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(result)
- .once();
- OssTimestampVersionedDataFinder finder = new OssTimestampVersionedDataFinder(oss);
-
- Pattern pattern = Pattern.compile("renames-[0-9]*\\.gz");
-
- EasyMock.replay(oss);
-
-
- URI latest = finder.getLatestVersion(URI.create(StringUtils.format("%s://%s/%s", OssStorageDruidModule.SCHEME, bucket, keyPrefix)), pattern);
-
- EasyMock.verify(oss);
-
- Assert.assertEquals(null, latest);
- }
-
- @Test
- public void testFindSelf()
- {
- String bucket = "bucket";
- String keyPrefix = "prefix/dir/0";
- OSS ossClient = EasyMock.createStrictMock(OSS.class);
-
- OSSObjectSummary object0 = new OSSObjectSummary();
-
- object0.setBucketName(bucket);
- object0.setKey(keyPrefix + "/renames-0.gz");
- object0.setLastModified(new Date(0));
- object0.setSize(10);
-
- final ObjectListing result = new ObjectListing();
- result.getObjectSummaries().add(object0);
- result.setTruncated(false);
-
- EasyMock.expect(ossClient.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(result)
- .once();
- OssTimestampVersionedDataFinder finder = new OssTimestampVersionedDataFinder(ossClient);
-
- Pattern pattern = Pattern.compile("renames-[0-9]*\\.gz");
-
- EasyMock.replay(ossClient);
-
-
- URI latest = finder.getLatestVersion(URI.create(StringUtils.format("%s://%s/%s", OssStorageDruidModule.SCHEME, bucket, keyPrefix)), pattern);
-
- EasyMock.verify(ossClient);
-
- URI expected = URI.create(StringUtils.format("%s://%s/%s", OssStorageDruidModule.SCHEME, bucket, object0.getKey()));
-
- Assert.assertEquals(expected, latest);
- }
-
- @Test
- public void testFindExact()
- {
- String bucket = "bucket";
- String keyPrefix = "prefix/dir/0";
- OSS ossClient = EasyMock.createStrictMock(OSS.class);
-
- OSSObjectSummary object0 = new OSSObjectSummary();
-
- object0.setBucketName(bucket);
- object0.setKey(keyPrefix + "/renames-0.gz");
- object0.setLastModified(new Date(0));
- object0.setSize(10);
-
- final ObjectListing result = new ObjectListing();
- result.getObjectSummaries().add(object0);
- result.setTruncated(false);
-
- EasyMock.expect(ossClient.listObjects(EasyMock.anyObject(ListObjectsRequest.class)))
- .andReturn(result)
- .once();
- OssTimestampVersionedDataFinder finder = new OssTimestampVersionedDataFinder(ossClient);
-
- EasyMock.replay(ossClient);
-
- URI latest = finder.getLatestVersion(URI.create(StringUtils.format("%s://%s/%s", OssStorageDruidModule.SCHEME, bucket, object0.getKey())), null);
-
- EasyMock.verify(ossClient);
-
- URI expected = URI.create(StringUtils.format("%s://%s/%s", OssStorageDruidModule.SCHEME, bucket, object0.getKey()));
-
- Assert.assertEquals(expected, latest);
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/pom.xml b/extensions-contrib/ambari-metrics-emitter/pom.xml
deleted file mode 100644
index 316b59f2d145..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/pom.xml
+++ /dev/null
@@ -1,169 +0,0 @@
-
-
-
-
- 4.0.0
-
-
- org.apache.druid
- druid
- 0.19.0-iap2-SNAPSHOT
- ../../pom.xml
-
-
- org.apache.druid.extensions.contrib
- ambari-metrics-emitter
- ambari-metrics-emitter
- Druid emitter extension to support ambari metrics server
-
-
-
- org.apache.druid
- druid-core
- ${project.parent.version}
- provided
-
-
- org.apache.druid
- druid-server
- ${project.parent.version}
- test-jar
- test
-
-
- org.apache.ambari
- ambari-metrics-common
- 2.4.1.0.22
-
-
- org.codehaus.jackson
- jackson-core-asl
-
-
- org.codehaus.jackson
- jackson-mapper-asl
-
-
-
- jdk.tools
- jdk.tools
-
-
-
-
-
- org.apache.hadoop
- hadoop-annotations
- ${hadoop.compile.version}
-
-
- commons-io
- commons-io
- provided
-
-
- com.fasterxml.jackson.core
- jackson-annotations
- provided
-
-
- joda-time
- joda-time
- provided
-
-
- com.google.inject
- guice
- provided
-
-
- com.fasterxml.jackson.core
- jackson-databind
- provided
-
-
- com.fasterxml.jackson.core
- jackson-core
- provided
-
-
- com.google.guava
- guava
- provided
-
-
- org.apache.druid
- druid-processing
- ${project.parent.version}
- test
-
-
- junit
- junit
- test
-
-
- org.easymock
- easymock
- test
-
-
- pl.pragmatists
- JUnitParams
- test
-
-
- org.codehaus.jackson
- jackson-core-asl
- ${codehaus.jackson.version}
- test
-
-
- org.codehaus.jackson
- jackson-mapper-asl
- ${codehaus.jackson.version}
- test
-
-
-
-
-
-
- org.apache.maven.plugins
- maven-dependency-plugin
-
-
-
- org.apache.hadoop:hadoop-annotations
-
-
-
-
-
-
-
-
- hortonworks
- hortonworks
- https://repo.hortonworks.com/content/repositories/releases
-
-
-
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
deleted file mode 100644
index d1ff8e362473..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitter.java
+++ /dev/null
@@ -1,233 +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.emitter.ambari.metrics;
-
-import com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.java.util.emitter.core.Emitter;
-import org.apache.druid.java.util.emitter.core.Event;
-import org.apache.druid.java.util.emitter.service.AlertEvent;
-import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.hadoop.metrics2.sink.timeline.AbstractTimelineMetricsSink;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetrics;
-
-import java.util.List;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
-import java.util.concurrent.LinkedBlockingQueue;
-import java.util.concurrent.ScheduledExecutorService;
-import java.util.concurrent.TimeUnit;
-import java.util.concurrent.TimeoutException;
-import java.util.concurrent.atomic.AtomicBoolean;
-import java.util.concurrent.atomic.AtomicLong;
-import java.util.regex.Pattern;
-
-
-public class AmbariMetricsEmitter extends AbstractTimelineMetricsSink implements Emitter
-{
- private static final Logger log = new Logger(AmbariMetricsEmitter.class);
-
- private final DruidToTimelineMetricConverter timelineMetricConverter;
- private final List emitterList;
- private final AtomicBoolean started = new AtomicBoolean(false);
- private final LinkedBlockingQueue eventsQueue;
- private final AmbariMetricsEmitterConfig config;
- private final String collectorURI;
- private static final long DEFAULT_FLUSH_TIMEOUT_MILLIS = 60000; // default flush wait 1 min
- private static final Pattern DOT_OR_WHITESPACE_PATTERN = Pattern.compile("[\\s]+|[.]+");
- private final ScheduledExecutorService exec = Executors.newScheduledThreadPool(
- 2, // Thread pool of two in order to schedule flush runnable
- new ThreadFactoryBuilder().setDaemon(true).setNameFormat("AmbariMetricsEmitter-%s").build()
- );
- private final AtomicLong countLostEvents = new AtomicLong(0);
-
- public AmbariMetricsEmitter(
- AmbariMetricsEmitterConfig config,
- List emitterList
- )
- {
- this.config = config;
- this.emitterList = emitterList;
- this.timelineMetricConverter = config.getDruidToTimelineEventConverter();
- this.eventsQueue = new LinkedBlockingQueue<>(config.getMaxQueueSize());
- this.collectorURI = StringUtils.format(
- "%s://%s:%s%s",
- config.getProtocol(),
- config.getHostname(),
- config.getPort(),
- WS_V1_TIMELINE_METRICS
- );
- }
-
- @Override
- public void start()
- {
- synchronized (started) {
- log.info("Starting Ambari Metrics Emitter.");
- if (!started.get()) {
- if ("https".equals(config.getProtocol())) {
- loadTruststore(config.getTrustStorePath(), config.getTrustStoreType(), config.getTrustStorePassword());
- }
- exec.scheduleAtFixedRate(
- new ConsumerRunnable(),
- config.getFlushPeriod(),
- config.getFlushPeriod(),
- TimeUnit.MILLISECONDS
- );
- started.set(true);
- }
- }
- }
-
-
- @Override
- public void emit(Event event)
- {
- if (!started.get()) {
- throw new ISE("WTF emit was called while service is not started yet");
- }
- if (event instanceof ServiceMetricEvent) {
- final TimelineMetric timelineEvent = timelineMetricConverter.druidEventToTimelineMetric((ServiceMetricEvent) event);
- if (timelineEvent == null) {
- return;
- }
- try {
- final boolean isSuccessful = eventsQueue.offer(
- timelineEvent,
- config.getEmitWaitTime(),
- TimeUnit.MILLISECONDS
- );
- if (!isSuccessful) {
- if (countLostEvents.getAndIncrement() % 1000 == 0) {
- log.error(
- "Lost total of [%s] events because of emitter queue is full. Please increase the capacity or/and the consumer frequency",
- countLostEvents.get()
- );
- }
- }
- }
- catch (InterruptedException e) {
- log.error(e, "got interrupted with message [%s]", e.getMessage());
- Thread.currentThread().interrupt();
- }
- } else if (event instanceof AlertEvent) {
- for (Emitter emitter : emitterList) {
- emitter.emit(event);
- }
- } else {
- throw new ISE("unknown event type [%s]", event.getClass());
- }
- }
-
- @Override
- protected String getCollectorUri()
- {
- return collectorURI;
- }
-
- @Override
- protected int getTimeoutSeconds()
- {
- return (int) (DEFAULT_FLUSH_TIMEOUT_MILLIS / 1000);
- }
-
- private class ConsumerRunnable implements Runnable
- {
- @Override
- public void run()
- {
- try {
- int batchSize = config.getBatchSize();
- TimelineMetrics metrics = new TimelineMetrics();
- while (eventsQueue.size() > 0 && !exec.isShutdown()) {
- try {
- final TimelineMetric metricEvent = eventsQueue.poll(
- config.getWaitForEventTime(),
- TimeUnit.MILLISECONDS
- );
- if (metricEvent != null) {
- metrics.addOrMergeTimelineMetric(metricEvent);
- if (metrics.getMetrics().size() == batchSize) {
- emitMetrics(metrics);
- log.debug(
- "sent [%d] events",
- metrics.getMetrics().size()
- );
- metrics = new TimelineMetrics();
- }
- }
- }
- catch (InterruptedException e) {
- log.error(e, e.getMessage());
- Thread.currentThread().interrupt();
- }
- }
- if (metrics.getMetrics().size() > 0) {
- emitMetrics(metrics);
- log.debug(
- "sent [%d] events",
- metrics.getMetrics().size()
- );
- }
- }
- catch (Exception e) {
- log.error(e, e.getMessage());
- }
-
- }
- }
-
- @Override
- public void flush()
- {
- synchronized (started) {
- if (started.get()) {
- Future future = exec.schedule(new ConsumerRunnable(), 0, TimeUnit.MILLISECONDS);
- try {
- future.get(DEFAULT_FLUSH_TIMEOUT_MILLIS, TimeUnit.MILLISECONDS);
- }
- catch (InterruptedException | ExecutionException | TimeoutException e) {
- if (e instanceof InterruptedException) {
- throw new RuntimeException("interrupted flushing elements from queue", e);
- }
- }
- }
- }
- }
-
- @Override
- public void close()
- {
- synchronized (started) {
- flush();
- exec.shutdown();
- started.set(false);
- }
- }
-
- protected static String sanitize(String namespace)
- {
- return DOT_OR_WHITESPACE_PATTERN.matcher(namespace).replaceAll("_");
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfig.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfig.java
deleted file mode 100644
index 69c6999b2c9f..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfig.java
+++ /dev/null
@@ -1,262 +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.emitter.ambari.metrics;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.concurrent.TimeUnit;
-
-public class AmbariMetricsEmitterConfig
-{
- private static final int DEFAULT_BATCH_SIZE = 100;
- private static final long DEFAULT_FLUSH_PERIOD_MILLIS = TimeUnit.MINUTES.toMillis(1); // flush every one minute
- private static final long DEFAULT_GET_TIMEOUT_MILLIS = TimeUnit.SECONDS.toMillis(1); // default wait for get operations on the queue 1 sec
- private static final String DEFAULT_PROTOCOL = "http";
-
- @JsonProperty
- private final String hostname;
-
- @JsonProperty
- private final int port;
-
- @JsonProperty
- private final String protocol;
-
- @JsonProperty
- private final String trustStorePath;
-
- @JsonProperty
- private final String trustStoreType;
-
- @JsonProperty
- private final String trustStorePassword;
-
- @JsonProperty
- private final int batchSize;
-
- @JsonProperty
- private final long flushPeriod;
-
- @JsonProperty
- private final int maxQueueSize;
-
- @JsonProperty("eventConverter")
- private final DruidToTimelineMetricConverter druidToTimelineEventConverter;
-
- @JsonProperty
- private final List alertEmitters;
-
- @JsonProperty
- private final long emitWaitTime;
-
- //waiting up to the specified wait time if necessary for an event to become available.
- @JsonProperty
- private final long waitForEventTime;
-
- @JsonCreator
- public AmbariMetricsEmitterConfig(
- @JsonProperty("hostname") String hostname,
- @JsonProperty("port") Integer port,
- @JsonProperty("protocol") String protocol,
- @JsonProperty("trustStorePath") String trustStorePath,
- @JsonProperty("trustStoreType") String trustStoreType,
- @JsonProperty("trustStorePassword") String trustStorePassword,
- @JsonProperty("batchSize") Integer batchSize,
- @JsonProperty("flushPeriod") Long flushPeriod,
- @JsonProperty("maxQueueSize") Integer maxQueueSize,
- @JsonProperty("eventConverter") DruidToTimelineMetricConverter druidToTimelineEventConverter,
- @JsonProperty("alertEmitters") List alertEmitters,
- @JsonProperty("emitWaitTime") Long emitWaitTime,
- @JsonProperty("waitForEventTime") Long waitForEventTime
- )
- {
- this.hostname = Preconditions.checkNotNull(hostname, "hostname can not be null");
- this.port = Preconditions.checkNotNull(port, "port can not be null");
- this.protocol = protocol == null ? DEFAULT_PROTOCOL : protocol;
- this.trustStorePath = trustStorePath;
- this.trustStoreType = trustStoreType;
- this.trustStorePassword = trustStorePassword;
- this.batchSize = (batchSize == null) ? DEFAULT_BATCH_SIZE : batchSize;
- this.flushPeriod = flushPeriod == null ? DEFAULT_FLUSH_PERIOD_MILLIS : flushPeriod;
- this.maxQueueSize = maxQueueSize == null ? Integer.MAX_VALUE : maxQueueSize;
- this.druidToTimelineEventConverter = Preconditions.checkNotNull(
- druidToTimelineEventConverter,
- "Event converter can not be null"
- );
- this.alertEmitters = alertEmitters == null ? Collections.emptyList() : alertEmitters;
- this.emitWaitTime = emitWaitTime == null ? 0 : emitWaitTime;
- this.waitForEventTime = waitForEventTime == null ? DEFAULT_GET_TIMEOUT_MILLIS : waitForEventTime;
- }
-
- @JsonProperty
- public String getHostname()
- {
- return hostname;
- }
-
- @JsonProperty
- public int getPort()
- {
- return port;
- }
-
- @JsonProperty
- public String getProtocol()
- {
- return protocol;
- }
-
- @JsonProperty
- public String getTrustStorePath()
- {
- return trustStorePath;
- }
-
- @JsonProperty
- public String getTrustStoreType()
- {
- return trustStoreType;
- }
-
- @JsonProperty
- public String getTrustStorePassword()
- {
- return trustStorePassword;
- }
-
- @JsonProperty
- public int getBatchSize()
- {
- return batchSize;
- }
-
- @JsonProperty
- public int getMaxQueueSize()
- {
- return maxQueueSize;
- }
-
- @JsonProperty
- public long getFlushPeriod()
- {
- return flushPeriod;
- }
-
- @JsonProperty
- public DruidToTimelineMetricConverter getDruidToTimelineEventConverter()
- {
- return druidToTimelineEventConverter;
- }
-
- @JsonProperty
- public List getAlertEmitters()
- {
- return alertEmitters;
- }
-
- @JsonProperty
- public long getEmitWaitTime()
- {
- return emitWaitTime;
- }
-
- @JsonProperty
- public long getWaitForEventTime()
- {
- return waitForEventTime;
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- AmbariMetricsEmitterConfig that = (AmbariMetricsEmitterConfig) o;
-
- if (port != that.port) {
- return false;
- }
- if (batchSize != that.batchSize) {
- return false;
- }
- if (flushPeriod != that.flushPeriod) {
- return false;
- }
- if (maxQueueSize != that.maxQueueSize) {
- return false;
- }
- if (emitWaitTime != that.emitWaitTime) {
- return false;
- }
- if (waitForEventTime != that.waitForEventTime) {
- return false;
- }
- if (!hostname.equals(that.hostname)) {
- return false;
- }
- if (!protocol.equals(that.protocol)) {
- return false;
- }
- if (trustStorePath != null ? !trustStorePath.equals(that.trustStorePath) : that.trustStorePath != null) {
- return false;
- }
- if (trustStoreType != null ? !trustStoreType.equals(that.trustStoreType) : that.trustStoreType != null) {
- return false;
- }
- if (trustStorePassword != null
- ? !trustStorePassword.equals(that.trustStorePassword)
- : that.trustStorePassword != null) {
- return false;
- }
- if (!druidToTimelineEventConverter.equals(that.druidToTimelineEventConverter)) {
- return false;
- }
- return alertEmitters.equals(that.alertEmitters);
-
- }
-
- @Override
- public int hashCode()
- {
- int result = hostname.hashCode();
- result = 31 * result + port;
- result = 31 * result + protocol.hashCode();
- result = 31 * result + (trustStorePath != null ? trustStorePath.hashCode() : 0);
- result = 31 * result + (trustStoreType != null ? trustStoreType.hashCode() : 0);
- result = 31 * result + (trustStorePassword != null ? trustStorePassword.hashCode() : 0);
- result = 31 * result + batchSize;
- result = 31 * result + (int) (flushPeriod ^ (flushPeriod >>> 32));
- result = 31 * result + maxQueueSize;
- result = 31 * result + druidToTimelineEventConverter.hashCode();
- result = 31 * result + alertEmitters.hashCode();
- result = 31 * result + (int) (emitWaitTime ^ (emitWaitTime >>> 32));
- result = 31 * result + (int) (waitForEventTime ^ (waitForEventTime >>> 32));
- return result;
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java
deleted file mode 100644
index bdb3bf17195a..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterModule.java
+++ /dev/null
@@ -1,66 +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.emitter.ambari.metrics;
-
-import com.fasterxml.jackson.databind.Module;
-import com.google.inject.Binder;
-import com.google.inject.Injector;
-import com.google.inject.Key;
-import com.google.inject.Provides;
-import com.google.inject.name.Named;
-import com.google.inject.name.Names;
-import org.apache.druid.guice.JsonConfigProvider;
-import org.apache.druid.guice.ManageLifecycle;
-import org.apache.druid.initialization.DruidModule;
-import org.apache.druid.java.util.emitter.core.Emitter;
-
-import java.util.Collections;
-import java.util.List;
-import java.util.stream.Collectors;
-
-public class AmbariMetricsEmitterModule implements DruidModule
-{
- private static final String EMITTER_TYPE = "ambari-metrics";
-
- @Override
- public List extends Module> getJacksonModules()
- {
- return Collections.emptyList();
- }
-
- @Override
- public void configure(Binder binder)
- {
- JsonConfigProvider.bind(binder, "druid.emitter." + EMITTER_TYPE, AmbariMetricsEmitterConfig.class);
- }
-
- @Provides
- @ManageLifecycle
- @Named(EMITTER_TYPE)
- public Emitter getEmitter(AmbariMetricsEmitterConfig emitterConfig, final Injector injector)
- {
- List emitters = emitterConfig
- .getAlertEmitters()
- .stream()
- .map((String name) -> injector.getInstance(Key.get(Emitter.class, Names.named(name))))
- .collect(Collectors.toList());
- return new AmbariMetricsEmitter(emitterConfig, emitters);
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/DruidToTimelineMetricConverter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/DruidToTimelineMetricConverter.java
deleted file mode 100644
index 10183e51370a..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/DruidToTimelineMetricConverter.java
+++ /dev/null
@@ -1,44 +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.emitter.ambari.metrics;
-
-import com.fasterxml.jackson.annotation.JsonSubTypes;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
-
-
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = WhiteListBasedDruidToTimelineEventConverter.class)
-@JsonSubTypes(value = {
- @JsonSubTypes.Type(name = "all", value = SendAllTimelineEventConverter.class),
- @JsonSubTypes.Type(name = "whiteList", value = WhiteListBasedDruidToTimelineEventConverter.class)
-})
-public interface DruidToTimelineMetricConverter
-{
- /**
- * This function acts as a filter. It returns null if the event is not suppose to be emitted to Ambari Server
- * Also This function will define the mapping between the druid event dimension's values and Ambari Metric Name
- *
- * @param serviceMetricEvent Druid event ot type {@link ServiceMetricEvent}
- *
- * @return {@link TimelineMetric} or null
- */
- TimelineMetric druidEventToTimelineMetric(ServiceMetricEvent serviceMetricEvent);
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/SendAllTimelineEventConverter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/SendAllTimelineEventConverter.java
deleted file mode 100644
index 6467418468b0..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/SendAllTimelineEventConverter.java
+++ /dev/null
@@ -1,129 +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.emitter.ambari.metrics;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Joiner;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSortedSet;
-import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
-
-/**
- * Emits all the events instance of {@link ServiceMetricEvent}.
- *
- * All the dimensions will be retained and lexicographically order using dimensions name.
- *
- * The metric name of the timeline event is:
- * ...
- *
- * Note that this name will be sanitized by replacing all the `.` or `space` to `_` {@link AmbariMetricsEmitter#sanitize(String)}
- */
-
-@JsonTypeName("all")
-public class SendAllTimelineEventConverter implements DruidToTimelineMetricConverter
-{
- public static final String DEFAULT_APP_NAME = "druid";
-
- @JsonProperty
- private final String namespacePrefix;
-
- @JsonProperty
- private final String appName;
-
- @JsonProperty
- public String getNamespacePrefix()
- {
- return namespacePrefix;
- }
-
- @JsonCreator
- public SendAllTimelineEventConverter(
- @JsonProperty("namespacePrefix") String namespacePrefix,
- @JsonProperty("appName") String appName
- )
- {
- this.namespacePrefix = namespacePrefix;
- this.appName = appName == null ? DEFAULT_APP_NAME : appName;
- }
-
- @Override
- public TimelineMetric druidEventToTimelineMetric(ServiceMetricEvent serviceMetricEvent)
- {
- ImmutableList.Builder metricNameBuilder = new ImmutableList.Builder<>();
- if (!Strings.isNullOrEmpty(namespacePrefix)) {
- metricNameBuilder.add(namespacePrefix);
- }
- metricNameBuilder.add(AmbariMetricsEmitter.sanitize(serviceMetricEvent.getService()));
- ImmutableSortedSet dimNames = ImmutableSortedSet.copyOf(serviceMetricEvent.getUserDims().keySet());
- for (String dimName : dimNames) {
- metricNameBuilder.add(
- AmbariMetricsEmitter.sanitize(
- String.valueOf(
- serviceMetricEvent.getUserDims().get(dimName)
- )
- )
- );
- }
- metricNameBuilder.add(AmbariMetricsEmitter.sanitize(serviceMetricEvent.getMetric()));
-
- TimelineMetric metric = new TimelineMetric();
- metric.setMetricName(Joiner.on(".").join(metricNameBuilder.build()));
- metric.setAppId(appName);
- metric.setHostName(serviceMetricEvent.getHost());
- metric.setType(serviceMetricEvent.getFeed());
- metric.setInstanceId(serviceMetricEvent.getService());
- long ts = serviceMetricEvent.getCreatedTime().getMillis();
- metric.setStartTime(ts);
- metric.setTimestamp(ts);
- metric.getMetricValues().put(ts, serviceMetricEvent.getValue().doubleValue());
- return metric;
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- SendAllTimelineEventConverter that = (SendAllTimelineEventConverter) o;
-
- if (namespacePrefix != null ? !namespacePrefix.equals(that.namespacePrefix) : that.namespacePrefix != null) {
- return false;
- }
- return appName.equals(that.appName);
-
- }
-
- @Override
- public int hashCode()
- {
- int result = namespacePrefix != null ? namespacePrefix.hashCode() : 0;
- result = 31 * result + appName.hashCode();
- return result;
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverter.java b/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverter.java
deleted file mode 100644
index aa0def755c1c..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverter.java
+++ /dev/null
@@ -1,259 +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.emitter.ambari.metrics;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.fasterxml.jackson.core.type.TypeReference;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Strings;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSortedMap;
-import com.google.common.io.Files;
-import com.google.common.io.Resources;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
-
-import java.io.File;
-import java.io.IOException;
-import java.net.URL;
-import java.nio.charset.StandardCharsets;
-import java.util.Collection;
-import java.util.Collections;
-import java.util.List;
-import java.util.SortedMap;
-
-@JsonTypeName("whiteList")
-public class WhiteListBasedDruidToTimelineEventConverter implements DruidToTimelineMetricConverter
-{
- private static final Logger LOGGER = new Logger(WhiteListBasedDruidToTimelineEventConverter.class);
- /**
- * @code whiteListDimsMapper is a white list of metric->dimensions mappings.
- * Key is the druid metric name or the metric's prefix.
- * Value is a list of metric's dimensions names.
- * The order of the dimension name is important, it will be used to build the ambari metrics name.
- * For instance if we have dimensions dataSource and queryType for metric query/time
- * the final metric name will be -> prefix.dataSource.queryType.metricName
- * Note that this name will be sanitized by replacing all the `.` or `space` to `_` {@link AmbariMetricsEmitter#sanitize(String)}
- */
- private final ImmutableSortedMap> whiteListDimsMapper;
-
- @JsonProperty
- private final String namespacePrefix;
-
- @JsonProperty
- private final String appName;
-
- @JsonProperty
- private final String mapPath;
-
- private final ObjectMapper mapper;
-
- @JsonCreator
- public WhiteListBasedDruidToTimelineEventConverter(
- @JsonProperty("namespacePrefix") String namespacePrefix,
- @JsonProperty("appName") String appName,
- @JsonProperty("mapPath") String mapPath,
- @JacksonInject ObjectMapper mapper
- )
- {
- this.mapper = Preconditions.checkNotNull(mapper);
- this.mapPath = mapPath;
- this.whiteListDimsMapper = readMap(this.mapPath);
- this.namespacePrefix = namespacePrefix;
- this.appName = appName == null ? SendAllTimelineEventConverter.DEFAULT_APP_NAME : appName;
-
- }
-
- @JsonProperty
- public String getNamespacePrefix()
- {
- return namespacePrefix;
- }
-
- /**
- * @param event Event subject to filtering
- *
- * @return true if and only if the event prefix key is in the {@code whiteListDimsMapper}
- */
- private boolean isInWhiteList(ServiceMetricEvent event)
- {
- return getPrefixKey(event.getMetric(), whiteListDimsMapper) != null;
- }
-
- /**
- * @param key the metric name to lookup
- * @param whiteList
- *
- * @return null if the key does not match with any of the prefixes keys in @code metricsWhiteList,
- * or the prefix in @code whiteListDimsMapper
- */
- private String getPrefixKey(String key, SortedMap whiteList)
- {
- String prefixKey = null;
- if (whiteList.containsKey(key)) {
- return key;
- }
- SortedMap headMap = whiteList.headMap(key);
- if (!headMap.isEmpty() && key.startsWith(headMap.lastKey())) {
- prefixKey = headMap.lastKey();
- }
- return prefixKey;
- }
-
- /**
- * Returns a {@link List} of the white-listed dimension's values to send.
- * The list is order is the same as the order of dimensions {@code whiteListDimsMapper}
- *
- * @param event the event for which will filter dimensions
- *
- * @return {@link List} of the filtered dimension values to send or null if the event is not in the white list
- */
- private List getOrderedDimValues(ServiceMetricEvent event)
- {
- String prefixKey = getPrefixKey(event.getMetric(), whiteListDimsMapper);
- if (prefixKey == null) {
- return null;
- }
- ImmutableList.Builder outputList = new ImmutableList.Builder<>();
- List dimensions = whiteListDimsMapper.get(prefixKey);
- if (dimensions == null) {
- return Collections.emptyList();
- }
-
- for (String dimKey : dimensions) {
- Object rawValue = event.getUserDims().get(dimKey);
- String value = null;
-
- if (rawValue instanceof String) {
- value = (String) rawValue;
- } else if (rawValue instanceof Collection) {
- Collection values = (Collection) rawValue;
- if (!values.isEmpty()) {
- value = (String) values.iterator().next();
- }
- }
-
- if (value != null) {
- outputList.add(AmbariMetricsEmitter.sanitize(value));
- }
- }
- return outputList.build();
- }
-
- /**
- * @param serviceMetricEvent druid metric event to convert
- *
- * @return null if the event is not white listed, otherwise return {@link TimelineMetric}
- *
- * The metric name of the ambari timeline metric event is:
- * ...
- *
- * The order of the dimension is the order returned by {@code getOrderedDimValues()}
- * Note that this name will be sanitized by replacing all the `.` or space by `_` {@link AmbariMetricsEmitter#sanitize(String)}
- *
- */
-
- @Override
- public TimelineMetric druidEventToTimelineMetric(ServiceMetricEvent serviceMetricEvent)
- {
- if (!this.isInWhiteList(serviceMetricEvent)) {
- return null;
- }
- final ImmutableList.Builder metricNameBuilder = new ImmutableList.Builder<>();
- if (!Strings.isNullOrEmpty(namespacePrefix)) {
- metricNameBuilder.add(namespacePrefix);
- }
- metricNameBuilder.add(AmbariMetricsEmitter.sanitize(serviceMetricEvent.getService()));
- metricNameBuilder.addAll(this.getOrderedDimValues(serviceMetricEvent));
- metricNameBuilder.add(AmbariMetricsEmitter.sanitize(serviceMetricEvent.getMetric()));
-
- TimelineMetric metric = new TimelineMetric();
- metric.setMetricName(Joiner.on(".").join(metricNameBuilder.build()));
- metric.setAppId(appName);
- metric.setHostName(serviceMetricEvent.getHost());
- metric.setType(serviceMetricEvent.getFeed());
- metric.setInstanceId(serviceMetricEvent.getService());
- long ts = serviceMetricEvent.getCreatedTime().getMillis();
- metric.setStartTime(ts);
- metric.setTimestamp(ts);
- metric.getMetricValues().put(ts, serviceMetricEvent.getValue().doubleValue());
- return metric;
- }
-
- private ImmutableSortedMap> readMap(final String mapPath)
- {
- String fileContent;
- String actualPath = mapPath;
- try {
- if (Strings.isNullOrEmpty(mapPath)) {
- URL defaultWhiteListMapUrl = this.getClass().getClassLoader().getResource("defaultWhiteListMap.json");
- actualPath = defaultWhiteListMapUrl.getFile();
- LOGGER.info("using default whiteList map located at [%s]", actualPath);
- fileContent = Resources.toString(defaultWhiteListMapUrl, StandardCharsets.UTF_8);
- } else {
- fileContent = Files.asCharSource(new File(mapPath), StandardCharsets.UTF_8).read();
- }
- return mapper.readerFor(new TypeReference>>()
- {
- }).readValue(fileContent);
- }
- catch (IOException e) {
- throw new ISE(e, "Got an exception while parsing file [%s]", actualPath);
- }
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- WhiteListBasedDruidToTimelineEventConverter that = (WhiteListBasedDruidToTimelineEventConverter) o;
-
- if (namespacePrefix != null ? !namespacePrefix.equals(that.namespacePrefix) : that.namespacePrefix != null) {
- return false;
- }
- if (!appName.equals(that.appName)) {
- return false;
- }
- return mapPath != null ? mapPath.equals(that.mapPath) : that.mapPath == null;
-
- }
-
- @Override
- public int hashCode()
- {
- int result = namespacePrefix != null ? namespacePrefix.hashCode() : 0;
- result = 31 * result + appName.hashCode();
- result = 31 * result + (mapPath != null ? mapPath.hashCode() : 0);
- return result;
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/ambari-metrics-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
deleted file mode 100644
index 78118ba31b84..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.druid.emitter.ambari.metrics.AmbariMetricsEmitterModule
diff --git a/extensions-contrib/ambari-metrics-emitter/src/main/resources/defaultWhiteListMap.json b/extensions-contrib/ambari-metrics-emitter/src/main/resources/defaultWhiteListMap.json
deleted file mode 100644
index ea31beed2a0f..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/main/resources/defaultWhiteListMap.json
+++ /dev/null
@@ -1,73 +0,0 @@
-{
- "ingest/events": [
- "dataSource"
- ],
- "ingest/handoff/failed": [
- "dataSource"
- ],
- "ingest/persists": [
- "dataSource"
- ],
- "ingest/rows/output": [
- "dataSource"
- ],
- "ingest/merge": [
- "dataSource"
- ],
- "jvm/gc": [],
- "jvm/mem": [
- "memKind"
- ],
- "query/cpu/time": [
- "dataSource",
- "type"
- ],
- "query/node/time": [
- "dataSource",
- "type"
- ],
- "query/node/ttfb": [
- "dataSource",
- "type"
- ],
- "query/partial/time": [
- "dataSource",
- "type"
- ],
- "query/segment/time": [
- "dataSource",
- "type"
- ],
- "query/segmentAndCache/time": [
- "dataSource",
- "type"
- ],
- "query/time": [
- "dataSource",
- "type"
- ],
- "query/wait/time": [
- "dataSource",
- "type"
- ],
- "segment/count": [
- "dataSource"
- ],
- "segment/dropQueue/count": [],
- "segment/loadQueue/count": [],
- "segment/loadQueue/failed": [],
- "segment/loadQueue/size": [],
- "segment/scan/pending": [],
- "segment/size": [
- "dataSource"
- ],
- "segment/usedPercent": [
- "dataSource"
- ],
- "segment/added/bytes" : [
- "dataSource"
- ],
- "segment/nuked/bytes" : [
- "dataSource"
- ]
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfigTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfigTest.java
deleted file mode 100644
index f2ea37a40fc7..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/AmbariMetricsEmitterConfigTest.java
+++ /dev/null
@@ -1,88 +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.emitter.ambari.metrics;
-
-import com.fasterxml.jackson.databind.InjectableValues;
-import com.fasterxml.jackson.databind.ObjectMapper;
-import org.apache.druid.jackson.DefaultObjectMapper;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Collections;
-
-public class AmbariMetricsEmitterConfigTest
-{
- private final ObjectMapper mapper = new DefaultObjectMapper();
-
- @Before
- public void setUp()
- {
- mapper.setInjectableValues(new InjectableValues.Std().addValue(
- ObjectMapper.class,
- new DefaultObjectMapper()
- ));
- }
-
- @Test
- public void testSerDeAmbariMetricsEmitterConfig() throws IOException
- {
- AmbariMetricsEmitterConfig config = new AmbariMetricsEmitterConfig(
- "hostname",
- 8080,
- "http",
- "truststore.path",
- "truststore.type",
- "truststore.password",
- 1000,
- 1000L,
- 100,
- new SendAllTimelineEventConverter("prefix", "druid"),
- Collections.emptyList(),
- 500L,
- 400L
- );
- AmbariMetricsEmitterConfig serde = mapper.readerFor(AmbariMetricsEmitterConfig.class).readValue(
- mapper.writeValueAsBytes(config)
- );
- Assert.assertEquals(config, serde);
- }
-
- @Test
- public void testSerDeDruidToTimelineEventConverter() throws IOException
- {
- SendAllTimelineEventConverter sendAllConverter = new SendAllTimelineEventConverter("prefix", "druid");
- DruidToTimelineMetricConverter serde = mapper.readerFor(DruidToTimelineMetricConverter.class)
- .readValue(mapper.writeValueAsBytes(sendAllConverter));
- Assert.assertEquals(sendAllConverter, serde);
-
- WhiteListBasedDruidToTimelineEventConverter whiteListBasedDruidToTimelineEventConverter = new WhiteListBasedDruidToTimelineEventConverter(
- "prefix",
- "druid",
- "",
- new DefaultObjectMapper()
- );
- serde = mapper.readerFor(DruidToTimelineMetricConverter.class)
- .readValue(mapper.writeValueAsBytes(
- whiteListBasedDruidToTimelineEventConverter));
- Assert.assertEquals(whiteListBasedDruidToTimelineEventConverter, serde);
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/DruidToWhiteListBasedConverterTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/DruidToWhiteListBasedConverterTest.java
deleted file mode 100644
index d4fe42290f19..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/DruidToWhiteListBasedConverterTest.java
+++ /dev/null
@@ -1,34 +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.emitter.ambari.metrics;
-
-import org.junit.Assert;
-import org.junit.Test;
-
-public class DruidToWhiteListBasedConverterTest
-{
-
- @Test
- public void testSanitize()
- {
- String test = "host name.yahoo.com:8080";
- Assert.assertEquals("host_name_yahoo_com:8080", AmbariMetricsEmitter.sanitize(test));
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java b/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
deleted file mode 100644
index d5ec9213501e..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/test/java/org/apache/druid/emitter/ambari/metrics/WhiteListBasedDruidToTimelineEventConverterTest.java
+++ /dev/null
@@ -1,195 +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.emitter.ambari.metrics;
-
-import junitparams.JUnitParamsRunner;
-import junitparams.Parameters;
-import org.apache.commons.io.IOUtils;
-import org.apache.druid.annotations.UsedByJUnitParamsRunner;
-import org.apache.druid.jackson.DefaultObjectMapper;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.emitter.service.ServiceMetricEvent;
-import org.apache.hadoop.metrics2.sink.timeline.TimelineMetric;
-import org.easymock.EasyMock;
-import org.joda.time.DateTime;
-import org.junit.Assert;
-import org.junit.Before;
-import org.junit.Test;
-import org.junit.runner.RunWith;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-import java.util.HashMap;
-
-
-@RunWith(JUnitParamsRunner.class)
-public class WhiteListBasedDruidToTimelineEventConverterTest
-{
- private final String prefix = "druid";
- private final WhiteListBasedDruidToTimelineEventConverter defaultWhiteListBasedDruidToTimelineEventConverter =
- new WhiteListBasedDruidToTimelineEventConverter(prefix, "druid", null, new DefaultObjectMapper());
- private ServiceMetricEvent event;
- private final DateTime createdTime = DateTimes.nowUtc();
- private final String hostname = "testHost:8080";
- private final String serviceName = "historical";
- private final String defaultNamespace = prefix + "." + serviceName;
-
- @Before
- public void setUp()
- {
- event = EasyMock.createMock(ServiceMetricEvent.class);
- EasyMock.expect(event.getHost()).andReturn(hostname).anyTimes();
- EasyMock.expect(event.getService()).andReturn(serviceName).anyTimes();
- EasyMock.expect(event.getCreatedTime()).andReturn(createdTime).anyTimes();
- EasyMock.expect(event.getUserDims()).andReturn(new HashMap<>()).anyTimes();
- EasyMock.expect(event.getValue()).andReturn(10).anyTimes();
- EasyMock.expect(event.getFeed()).andReturn("metrics").anyTimes();
- }
-
- @Test
- @Parameters(
- {
- "query/time, true",
- "query/node/ttfb, true",
- "query/segmentAndCache/time, true",
- "query/time/balaba, true",
- "query/tim, false",
- "segment/added/bytes, true",
- "segment/count, true",
- "segment/size, true",
- "segment/cost/raw, false",
- "coordinator/TIER_1 /cost/raw, false",
- "segment/Kost/raw, false",
- ", false",
- "word, false",
- "coordinator, false",
- "server/, false",
- "ingest/persists/time, true",
- "jvm/mem/init, true",
- "jvm/gc/count, true"
- }
- )
- public void testDefaultIsInWhiteList(String key, boolean expectedValue)
- {
- EasyMock.expect(event.getMetric()).andReturn(key).anyTimes();
- EasyMock.replay(event);
- boolean isIn = defaultWhiteListBasedDruidToTimelineEventConverter.druidEventToTimelineMetric(event) != null;
- Assert.assertEquals(expectedValue, isIn);
- }
-
- @Test
- @Parameters
- public void testGetName(ServiceMetricEvent serviceMetricEvent, String expectedPath)
- {
- TimelineMetric metric = defaultWhiteListBasedDruidToTimelineEventConverter.druidEventToTimelineMetric(serviceMetricEvent);
- String path = null;
- if (metric != null) {
- path = metric.getMetricName();
- }
- Assert.assertEquals(expectedPath, path);
- }
-
- @Test
- public void testWhiteListedStringArrayDimension() throws IOException
- {
- File mapFile = File.createTempFile("testing-" + System.nanoTime(), ".json");
- mapFile.deleteOnExit();
-
- try (OutputStream outputStream = new FileOutputStream(mapFile)) {
- IOUtils.copyLarge(
- getClass().getResourceAsStream("/testWhiteListedStringArrayDimension.json"),
- outputStream
- );
- }
-
- WhiteListBasedDruidToTimelineEventConverter converter = new WhiteListBasedDruidToTimelineEventConverter(
- prefix,
- "druid",
- mapFile.getAbsolutePath(),
- new DefaultObjectMapper()
- );
-
- ServiceMetricEvent event = new ServiceMetricEvent.Builder()
- .setDimension("gcName", new String[] {"g1"})
- .build(createdTime, "jvm/gc/cpu", 10)
- .build(serviceName, hostname);
-
- TimelineMetric metric = converter.druidEventToTimelineMetric(event);
-
- Assert.assertNotNull(metric);
- Assert.assertEquals(defaultNamespace + ".g1.jvm/gc/cpu", metric.getMetricName());
- }
-
- @UsedByJUnitParamsRunner
- private Object[] parametersForTestGetName()
- {
- return new Object[]{
- new Object[]{
- new ServiceMetricEvent.Builder().setDimension("id", "dummy_id")
- .setDimension("status", "some_status")
- .setDimension("numDimensions", "1")
- .setDimension("segment", "dummy_segment")
- .build(createdTime, "query/segment/time/balabla/more", 10)
- .build(serviceName, hostname),
- defaultNamespace + ".query/segment/time/balabla/more"
- },
- new Object[]{
- new ServiceMetricEvent.Builder().setDimension("dataSource", "some_data_source")
- .setDimension("tier", "_default_tier")
- .build(createdTime, "segment/max", 10)
- .build(serviceName, hostname),
- null
- },
- new Object[]{
- new ServiceMetricEvent.Builder().setDimension("dataSource", "data-source")
- .setDimension("type", "groupBy")
- .setDimension("interval", "2013/2015")
- .setDimension("some_random_dim1", "random_dim_value1")
- .setDimension("some_random_dim2", "random_dim_value2")
- .setDimension("hasFilters", "no")
- .setDimension("duration", "P1D")
- .setDimension("remoteAddress", "194.0.90.2")
- .setDimension("id", "ID")
- .setDimension("context", "{context}")
- .build(createdTime, "query/time", 10)
- .build(serviceName, hostname),
- defaultNamespace + ".data-source.groupBy.query/time"
- },
- new Object[]{
- new ServiceMetricEvent.Builder().setDimension("dataSource", "data-source")
- .setDimension("type", "groupBy")
- .setDimension("some_random_dim1", "random_dim_value1")
- .build(createdTime, "ingest/persists/count", 10)
- .build(serviceName, hostname),
- defaultNamespace + ".data-source.ingest/persists/count"
- },
- new Object[]{
- new ServiceMetricEvent.Builder().setDimension("bufferpoolName", "BufferPool")
- .setDimension("type", "groupBy")
- .setDimension("some_random_dim1", "random_dim_value1")
- .build(createdTime, "jvm/bufferpool/capacity", 10)
- .build(serviceName, hostname),
- null
- }
- };
- }
-}
diff --git a/extensions-contrib/ambari-metrics-emitter/src/test/resources/testWhiteListedStringArrayDimension.json b/extensions-contrib/ambari-metrics-emitter/src/test/resources/testWhiteListedStringArrayDimension.json
deleted file mode 100644
index 757742e1eed3..000000000000
--- a/extensions-contrib/ambari-metrics-emitter/src/test/resources/testWhiteListedStringArrayDimension.json
+++ /dev/null
@@ -1,3 +0,0 @@
-{
- "jvm/gc": ["gcName"]
-}
diff --git a/extensions-contrib/cassandra-storage/pom.xml b/extensions-contrib/cassandra-storage/pom.xml
deleted file mode 100644
index de610aaf4e29..000000000000
--- a/extensions-contrib/cassandra-storage/pom.xml
+++ /dev/null
@@ -1,176 +0,0 @@
-
-
-
-
- 4.0.0
-
- org.apache.druid.extensions.contrib
- druid-cassandra-storage
- druid-cassandra-storage
- druid-cassandra-storage
-
-
- org.apache.druid
- druid
- 0.19.0-iap2-SNAPSHOT
- ../../pom.xml
-
-
-
-
- org.apache.druid
- druid-core
- ${project.parent.version}
- provided
-
-
-
- com.netflix.astyanax
- astyanax
- 1.0.1
-
-
- jline
- jline
-
-
- joda-time
- joda-time
-
-
- org.mortbay.jetty
- jetty-util
-
-
- com.ning
- compress-lzf
-
-
- commons-lang
- commons-lang
-
-
- commons-cli
- commons-cli
-
-
- log4j
- log4j
-
-
- commons-codec
- commons-codec
-
-
- commons-logging
- commons-logging
-
-
- org.apache.httpcomponents
- httpclient
-
-
- org.apache.httpcomponents
- httpcore
-
-
- org.codehaus.jackson
- jackson-core-asl
-
-
- org.codehaus.jackson
- jackson-mapper-asl
-
-
- org.apache.zookeeper
- zookeeper
-
-
- org.slf4j
- slf4j-api
-
-
- org.slf4j
- slf4j-log4j12
-
-
- com.github.stephenc.high-scale-lib
- high-scale-lib
-
-
-
- org.mortbay.jetty
- jetty
-
-
-
-
- commons-io
- commons-io
- provided
-
-
- com.fasterxml.jackson.core
- jackson-annotations
- provided
-
-
- com.google.guava
- guava
-
-
- com.google.inject
- guice
- provided
-
-
- com.fasterxml.jackson.core
- jackson-databind
- provided
-
-
- javax.validation
- validation-api
- provided
-
-
- com.fasterxml.jackson.core
- jackson-core
- provided
-
-
- com.google.inject.extensions
- guice-multibindings
- provided
-
-
-
-
- junit
- junit
- test
-
-
-
-
diff --git a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentConfig.java b/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentConfig.java
deleted file mode 100644
index 6302e3dc222f..000000000000
--- a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentConfig.java
+++ /dev/null
@@ -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.storage.cassandra;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import javax.validation.constraints.NotNull;
-
-/**
- * Cassandra Config
- */
-public class CassandraDataSegmentConfig
-{
- @JsonProperty
- @NotNull
- public String host = null;
-
- @JsonProperty
- @NotNull
- public String keyspace = null;
-
- public String getKeyspace()
- {
- return keyspace;
- }
-
- public String getHost()
- {
- return host;
- }
-}
diff --git a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentPuller.java b/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentPuller.java
deleted file mode 100644
index 076a3306a1a2..000000000000
--- a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentPuller.java
+++ /dev/null
@@ -1,108 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cassandra;
-
-import com.google.common.base.Predicates;
-import com.google.inject.Inject;
-import com.netflix.astyanax.recipes.storage.ChunkedStorage;
-import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.RetryUtils;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-import org.apache.druid.utils.CompressionUtils;
-
-import java.io.File;
-import java.io.FileOutputStream;
-import java.io.IOException;
-import java.io.OutputStream;
-
-/**
- * Cassandra Segment Puller
- */
-public class CassandraDataSegmentPuller extends CassandraStorage
-{
- private static final Logger log = new Logger(CassandraDataSegmentPuller.class);
- private static final int CONCURRENCY = 10;
- private static final int BATCH_SIZE = 10;
-
- @Inject
- public CassandraDataSegmentPuller(CassandraDataSegmentConfig config)
- {
- super(config);
- }
-
- FileUtils.FileCopyResult getSegmentFiles(final String key, final File outDir) throws SegmentLoadingException
- {
- log.info("Pulling index from C* at path[%s] to outDir[%s]", key, outDir);
- try {
- org.apache.commons.io.FileUtils.forceMkdir(outDir);
- }
- catch (IOException e) {
- throw new SegmentLoadingException(e, "");
- }
-
- long startTime = System.currentTimeMillis();
- final File tmpFile = new File(outDir, "index.zip");
- log.info("Pulling to temporary local cache [%s]", tmpFile.getAbsolutePath());
-
- try {
- RetryUtils.retry(
- () -> {
- try (OutputStream os = new FileOutputStream(tmpFile)) {
- ChunkedStorage
- .newReader(indexStorage, key, os)
- .withBatchSize(BATCH_SIZE)
- .withConcurrencyLevel(CONCURRENCY)
- .call();
- }
- return new FileUtils.FileCopyResult(tmpFile);
- },
- Predicates.alwaysTrue(),
- 10
- );
- }
- catch (Exception e) {
- throw new SegmentLoadingException(e, "Unable to copy key [%s] to file [%s]", key, tmpFile.getAbsolutePath());
- }
- try {
- final FileUtils.FileCopyResult result = CompressionUtils.unzip(tmpFile, outDir);
- log.info(
- "Pull of file[%s] completed in %,d millis (%s bytes)", key, System.currentTimeMillis() - startTime,
- result.size()
- );
- return result;
- }
- catch (Exception e) {
- try {
- FileUtils.deleteDirectory(outDir);
- }
- catch (IOException e1) {
- log.error(e1, "Error clearing segment directory [%s]", outDir.getAbsolutePath());
- e.addSuppressed(e1);
- }
- throw new SegmentLoadingException(e, e.getMessage());
- }
- finally {
- if (!tmpFile.delete()) {
- log.warn("Could not delete cache file at [%s]", tmpFile.getAbsolutePath());
- }
- }
- }
-}
diff --git a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentPusher.java b/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentPusher.java
deleted file mode 100644
index c61443f924a9..000000000000
--- a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDataSegmentPusher.java
+++ /dev/null
@@ -1,132 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cassandra;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.base.Joiner;
-import com.google.common.collect.ImmutableMap;
-import com.google.inject.Inject;
-import com.netflix.astyanax.MutationBatch;
-import com.netflix.astyanax.connectionpool.exceptions.NotFoundException;
-import com.netflix.astyanax.recipes.storage.ChunkedStorage;
-import com.netflix.astyanax.recipes.storage.ChunkedStorageProvider;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.segment.SegmentUtils;
-import org.apache.druid.segment.loading.DataSegmentPusher;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.utils.CompressionUtils;
-
-import java.io.File;
-import java.io.IOException;
-import java.io.InputStream;
-import java.net.URI;
-import java.nio.file.Files;
-import java.util.Map;
-
-/**
- * Cassandra Segment Pusher
- */
-public class CassandraDataSegmentPusher extends CassandraStorage implements DataSegmentPusher
-{
- private static final Logger log = new Logger(CassandraDataSegmentPusher.class);
- private static final int CONCURRENCY = 10;
- private static final Joiner JOINER = Joiner.on("/").skipNulls();
- private final ObjectMapper jsonMapper;
-
- @Inject
- public CassandraDataSegmentPusher(
- CassandraDataSegmentConfig config,
- ObjectMapper jsonMapper
- )
- {
- super(config);
- this.jsonMapper = jsonMapper;
- }
-
- @Override
- public String getPathForHadoop()
- {
- throw new UnsupportedOperationException("Cassandra storage does not support indexing via Hadoop");
- }
-
- @Deprecated
- @Override
- public String getPathForHadoop(String dataSource)
- {
- return getPathForHadoop();
- }
-
- @Override
- public DataSegment push(final File indexFilesDir, DataSegment segment, final boolean useUniquePath) throws IOException
- {
- log.info("Writing [%s] to C*", indexFilesDir);
- String key = JOINER.join(
- config.getKeyspace().isEmpty() ? null : config.getKeyspace(),
- this.getStorageDir(segment, useUniquePath)
- );
-
- // Create index
- final File compressedIndexFile = File.createTempFile("druid", "index.zip");
- long indexSize = CompressionUtils.zip(indexFilesDir, compressedIndexFile);
- log.info("Wrote compressed file [%s] to [%s]", compressedIndexFile.getAbsolutePath(), key);
-
- int version = SegmentUtils.getVersionFromDir(indexFilesDir);
-
- try (final InputStream fileStream = Files.newInputStream(compressedIndexFile.toPath())) {
- long start = System.currentTimeMillis();
- ChunkedStorage.newWriter(indexStorage, key, fileStream)
- .withConcurrencyLevel(CONCURRENCY).call();
- byte[] json = jsonMapper.writeValueAsBytes(segment);
- MutationBatch mutation = this.keyspace.prepareMutationBatch();
- mutation.withRow(descriptorStorage, key)
- .putColumn("lastmodified", System.currentTimeMillis(), null)
- .putColumn("descriptor", json, null);
- mutation.execute();
- log.info("Wrote index to C* in [%s] ms", System.currentTimeMillis() - start);
- }
- catch (Exception e) {
- throw new IOException(e);
- }
-
- segment = segment.withSize(indexSize)
- .withLoadSpec(ImmutableMap.of("type", "c*", "key", key))
- .withBinaryVersion(version);
-
- log.info("Deleting zipped index File[%s]", compressedIndexFile);
- compressedIndexFile.delete();
- return segment;
- }
-
- @Override
- public Map makeLoadSpec(URI uri)
- {
- throw new UnsupportedOperationException("not supported");
- }
-
- private boolean doesObjectExist(ChunkedStorageProvider provider, String objectName) throws Exception
- {
- try {
- return ChunkedStorage.newInfoReader(provider, objectName).call().isValidForRead();
- }
- catch (NotFoundException e) {
- return false;
- }
- }
-}
diff --git a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDruidModule.java b/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDruidModule.java
deleted file mode 100644
index 26a8434afc38..000000000000
--- a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraDruidModule.java
+++ /dev/null
@@ -1,77 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cassandra;
-
-import com.fasterxml.jackson.core.Version;
-import com.fasterxml.jackson.databind.Module;
-import com.google.common.collect.ImmutableList;
-import com.google.inject.Binder;
-import com.google.inject.Key;
-import org.apache.druid.guice.JsonConfigProvider;
-import org.apache.druid.guice.LazySingleton;
-import org.apache.druid.guice.PolyBind;
-import org.apache.druid.initialization.DruidModule;
-import org.apache.druid.segment.loading.DataSegmentPusher;
-
-import java.util.List;
-
-/**
- */
-public class CassandraDruidModule implements DruidModule
-{
- public static final String SCHEME = "c*";
-
- @Override
- public void configure(Binder binder)
- {
- PolyBind.optionBinder(binder, Key.get(DataSegmentPusher.class))
- .addBinding(SCHEME)
- .to(CassandraDataSegmentPusher.class)
- .in(LazySingleton.class);
- JsonConfigProvider.bind(binder, "druid.storage", CassandraDataSegmentConfig.class);
- }
-
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new Module()
- {
- @Override
- public String getModuleName()
- {
- return "DruidCassandraStorage-" + System.identityHashCode(this);
- }
-
- @Override
- public Version version()
- {
- return Version.unknownVersion();
- }
-
- @Override
- public void setupModule(SetupContext context)
- {
- context.registerSubtypes(CassandraLoadSpec.class);
- }
- }
- );
- }
-}
diff --git a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraLoadSpec.java b/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraLoadSpec.java
deleted file mode 100644
index d17e5ab1f135..000000000000
--- a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraLoadSpec.java
+++ /dev/null
@@ -1,56 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cassandra;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import org.apache.druid.segment.loading.LoadSpec;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-
-import java.io.File;
-
-/**
- *
- */
-@JsonTypeName(CassandraDruidModule.SCHEME)
-public class CassandraLoadSpec implements LoadSpec
-{
- @JsonProperty
- private final String key;
- private final CassandraDataSegmentPuller puller;
-
- @JsonCreator
- public CassandraLoadSpec(
- @JacksonInject CassandraDataSegmentPuller puller,
- @JsonProperty("key") String key
- )
- {
- this.puller = puller;
- this.key = key;
- }
-
- @Override
- public LoadSpecResult loadSegment(File outDir) throws SegmentLoadingException
- {
- return new LoadSpecResult(puller.getSegmentFiles(key, outDir).size());
- }
-}
diff --git a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraStorage.java b/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraStorage.java
deleted file mode 100644
index 9588eb946fb5..000000000000
--- a/extensions-contrib/cassandra-storage/src/main/java/org/apache/druid/storage/cassandra/CassandraStorage.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cassandra;
-
-import com.netflix.astyanax.AstyanaxContext;
-import com.netflix.astyanax.Keyspace;
-import com.netflix.astyanax.connectionpool.NodeDiscoveryType;
-import com.netflix.astyanax.connectionpool.impl.ConnectionPoolConfigurationImpl;
-import com.netflix.astyanax.connectionpool.impl.CountingConnectionPoolMonitor;
-import com.netflix.astyanax.impl.AstyanaxConfigurationImpl;
-import com.netflix.astyanax.model.ColumnFamily;
-import com.netflix.astyanax.recipes.storage.CassandraChunkedStorageProvider;
-import com.netflix.astyanax.recipes.storage.ChunkedStorageProvider;
-import com.netflix.astyanax.serializers.StringSerializer;
-import com.netflix.astyanax.thrift.ThriftFamilyFactory;
-
-/**
- * Superclass for accessing Cassandra Storage.
- *
- * This is the schema used to support the index and descriptor storage:
- *
- * CREATE TABLE index_storage ( key text, chunk text, value blob, PRIMARY KEY (key, chunk)) WITH COMPACT STORAGE;
- * CREATE TABLE descriptor_storage ( key varchar, lastModified timestamp, descriptor varchar, PRIMARY KEY (key) ) WITH COMPACT STORAGE;
- */
-public class CassandraStorage
-{
- private static final String CLUSTER_NAME = "druid_cassandra_cluster";
- private static final String INDEX_TABLE_NAME = "index_storage";
- private static final String DESCRIPTOR_TABLE_NAME = "descriptor_storage";
-
- private AstyanaxContext astyanaxContext;
- final Keyspace keyspace;
- final ChunkedStorageProvider indexStorage;
- final ColumnFamily descriptorStorage;
- final CassandraDataSegmentConfig config;
-
- public CassandraStorage(CassandraDataSegmentConfig config)
- {
- this.astyanaxContext = new AstyanaxContext.Builder()
- .forCluster(CLUSTER_NAME)
- .forKeyspace(config.getKeyspace())
- .withAstyanaxConfiguration(new AstyanaxConfigurationImpl().setDiscoveryType(NodeDiscoveryType.NONE))
- .withConnectionPoolConfiguration(
- new ConnectionPoolConfigurationImpl("MyConnectionPool").setMaxConnsPerHost(10)
- .setSeeds(config.getHost())).withConnectionPoolMonitor(new CountingConnectionPoolMonitor())
- .buildKeyspace(ThriftFamilyFactory.getInstance());
- this.astyanaxContext.start();
- this.keyspace = this.astyanaxContext.getEntity();
- this.config = config;
- indexStorage = new CassandraChunkedStorageProvider(keyspace, INDEX_TABLE_NAME);
-
- descriptorStorage = new ColumnFamily(DESCRIPTOR_TABLE_NAME,
- StringSerializer.get(), StringSerializer.get());
- }
-}
diff --git a/extensions-contrib/cassandra-storage/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/cassandra-storage/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
deleted file mode 100644
index c0190a51f7a3..000000000000
--- a/extensions-contrib/cassandra-storage/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.druid.storage.cassandra.CassandraDruidModule
diff --git a/extensions-contrib/cloudfiles-extensions/pom.xml b/extensions-contrib/cloudfiles-extensions/pom.xml
deleted file mode 100644
index c3cbf94e0191..000000000000
--- a/extensions-contrib/cloudfiles-extensions/pom.xml
+++ /dev/null
@@ -1,187 +0,0 @@
-
-
-
-
- 4.0.0
-
- org.apache.druid.extensions.contrib
- druid-cloudfiles-extensions
- druid-cloudfiles-extensions
- druid-cloudfiles-extensions
-
-
- org.apache.druid
- druid
- 0.19.0-iap2-SNAPSHOT
- ../../pom.xml
-
-
-
- UTF-8
- 1.9.1
-
- 3.0
-
-
-
-
- org.apache.druid
- druid-core
- ${project.parent.version}
- provided
-
-
- com.google.inject
- guice
- ${guice.version}
-
-
- aopalliance
- aopalliance
-
-
-
-
-
- com.google.inject.extensions
- guice-multibindings
- ${guice.version}
-
-
-
- commons-io
- commons-io
- provided
-
-
- com.fasterxml.jackson.core
- jackson-annotations
- provided
-
-
- com.fasterxml.jackson.core
- jackson-databind
- provided
-
-
- com.fasterxml.jackson.core
- jackson-core
- provided
-
-
- com.google.guava
- guava
-
-
- javax.validation
- validation-api
- provided
-
-
-
- org.apache.jclouds
- jclouds-core
- ${jclouds.version}
-
-
- org.apache.jclouds.api
- openstack-swift
- ${jclouds.version}
-
-
- org.apache.jclouds.driver
- jclouds-slf4j
- ${jclouds.version}
-
-
- org.slf4j
- slf4j-api
-
-
- javax.ws.rs
- jsr311-api
-
-
-
-
- org.apache.jclouds.api
- openstack-keystone
- ${jclouds.version}
-
-
- org.apache.jclouds.api
- rackspace-cloudfiles
- ${jclouds.version}
-
-
-
- org.apache.jclouds.provider
- rackspace-cloudfiles-us
- ${jclouds.version}
-
-
-
- org.apache.jclouds.provider
- rackspace-cloudfiles-uk
- ${jclouds.version}
-
-
- com.fasterxml.jackson.module
- jackson-module-guice
- ${jackson.version}
-
-
- com.fasterxml.jackson.core
- jackson-databind
-
-
- com.fasterxml.jackson.core
- jackson-core
-
-
-
-
-
-
- junit
- junit
- test
-
-
- org.easymock
- easymock
- test
-
-
- org.apache.druid
- druid-server
- ${project.parent.version}
- test
-
-
- org.apache.druid
- druid-processing
- ${project.parent.version}
- test
-
-
-
-
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java
deleted file mode 100644
index edc0bbb65c06..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesBlob.java
+++ /dev/null
@@ -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);
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java
deleted file mode 100644
index 3531e7584e45..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/CloudFilesFirehoseDruidModule.java
+++ /dev/null
@@ -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)
- {
-
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java
deleted file mode 100644
index f0de9f7e98de..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactory.java
+++ /dev/null
@@ -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
-{
- private static final Logger log = new Logger(StaticCloudFilesFirehoseFactory.class);
-
- private final CloudFilesApi cloudFilesApi;
- private final List blobs;
-
- @JsonCreator
- public StaticCloudFilesFirehoseFactory(
- @JacksonInject CloudFilesApi cloudFilesApi,
- @JsonProperty("blobs") List 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 getBlobs()
- {
- return blobs;
- }
-
- @Override
- protected Collection 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 getRetryCondition()
- {
- return CloudFilesUtils.CLOUDFILESRETRY;
- }
-
- @Override
- public FiniteFirehoseFactory withSplit(InputSplit split)
- {
- return new StaticCloudFilesFirehoseFactory(
- cloudFilesApi,
- Collections.singletonList(split.get()),
- getMaxCacheCapacityBytes(),
- getMaxFetchCapacityBytes(),
- getPrefetchTriggerBytes(),
- getFetchTimeout(),
- getMaxFetchRetry()
- );
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesAccountConfig.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesAccountConfig.java
deleted file mode 100644
index 0317d6124996..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesAccountConfig.java
+++ /dev/null
@@ -1,65 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-
-import javax.validation.constraints.NotNull;
-
-public class CloudFilesAccountConfig
-{
-
- @JsonProperty
- @NotNull
- private String provider;
-
- @JsonProperty
- @NotNull
- private String userName;
-
- @JsonProperty
- @NotNull
- private String apiKey;
-
- @JsonProperty
- @NotNull
- private boolean useServiceNet = true;
-
- public String getProvider()
- {
- return provider;
- }
-
- public String getUserName()
- {
- return userName;
- }
-
- public String getApiKey()
- {
- return apiKey;
- }
-
- public boolean getUseServiceNet()
- {
- return useServiceNet;
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSource.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSource.java
deleted file mode 100644
index eaa602a5f46f..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSource.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.google.common.io.ByteSource;
-import org.jclouds.io.Payload;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-public class CloudFilesByteSource extends ByteSource
-{
-
- private final CloudFilesObjectApiProxy objectApi;
- private final String path;
- private Payload payload;
-
- public CloudFilesByteSource(CloudFilesObjectApiProxy objectApi, String path)
- {
- this.objectApi = objectApi;
- this.path = path;
- this.payload = null;
- }
-
- public void closeStream() throws IOException
- {
- if (payload != null) {
- payload.close();
- payload = null;
- }
- }
-
- @Override
- public InputStream openStream() throws IOException
- {
- return openStream(0);
- }
-
- public InputStream openStream(long start) throws IOException
- {
- payload = (payload == null) ? objectApi.get(path, start).getPayload() : payload;
-
- try {
- return payload.openStream();
- }
- catch (IOException e) {
- if (CloudFilesUtils.CLOUDFILESRETRY.apply(e)) {
- throw new IOException("Recoverable exception", e);
- }
- throw new RuntimeException(e);
- }
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPuller.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPuller.java
deleted file mode 100644
index 55399fa78aac..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPuller.java
+++ /dev/null
@@ -1,96 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.google.inject.Inject;
-import org.apache.druid.java.util.common.FileUtils;
-import org.apache.druid.java.util.common.ISE;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-import org.apache.druid.utils.CompressionUtils;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-
-import java.io.File;
-import java.io.IOException;
-
-public class CloudFilesDataSegmentPuller
-{
-
- private static final Logger log = new Logger(CloudFilesDataSegmentPuller.class);
- private final CloudFilesApi cloudFilesApi;
-
- @Inject
- public CloudFilesDataSegmentPuller(final CloudFilesApi cloudFilesApi)
- {
- this.cloudFilesApi = cloudFilesApi;
- }
-
- FileUtils.FileCopyResult getSegmentFiles(String region, String container, String path, File outDir)
- throws SegmentLoadingException
- {
- CloudFilesObjectApiProxy objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
- final CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path);
-
- try {
- final FileUtils.FileCopyResult result = CompressionUtils.unzip(
- byteSource,
- outDir,
- CloudFilesUtils.CLOUDFILESRETRY,
- false
- );
- log.info("Loaded %d bytes from [%s] to [%s]", result.size(), path, outDir.getAbsolutePath());
- return result;
- }
- catch (Exception e) {
- try {
- FileUtils.deleteDirectory(outDir);
- }
- catch (IOException ioe) {
- log.warn(
- ioe,
- "Failed to remove output directory [%s] for segment pulled from [%s]",
- outDir.getAbsolutePath(),
- path
- );
- }
- throw new SegmentLoadingException(e, e.getMessage());
- }
- finally {
- try {
- byteSource.closeStream();
- }
- catch (IOException ioe) {
- log.warn(ioe, "Failed to close payload for segmente pulled from [%s]", path);
- }
- }
- }
-
- private void prepareOutDir(final File outDir) throws ISE
- {
- if (!outDir.exists()) {
- outDir.mkdirs();
- }
-
- if (!outDir.isDirectory()) {
- throw new ISE("outDir[%s] must be a directory.", outDir);
- }
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
deleted file mode 100644
index 42fe23f94318..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusher.java
+++ /dev/null
@@ -1,153 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.fasterxml.jackson.databind.ObjectMapper;
-import com.google.common.collect.ImmutableMap;
-import com.google.inject.Inject;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.apache.druid.segment.SegmentUtils;
-import org.apache.druid.segment.loading.DataSegmentPusher;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.utils.CompressionUtils;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-
-import java.io.File;
-import java.net.URI;
-import java.nio.file.Files;
-import java.util.Map;
-
-public class CloudFilesDataSegmentPusher implements DataSegmentPusher
-{
-
- private static final Logger log = new Logger(CloudFilesDataSegmentPusher.class);
- private final CloudFilesObjectApiProxy objectApi;
- private final CloudFilesDataSegmentPusherConfig config;
- private final ObjectMapper jsonMapper;
-
- @Inject
- public CloudFilesDataSegmentPusher(
- final CloudFilesApi cloudFilesApi,
- final CloudFilesDataSegmentPusherConfig config, final ObjectMapper jsonMapper
- )
- {
- this.config = config;
- String region = this.config.getRegion();
- String container = this.config.getContainer();
- this.objectApi = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
- this.jsonMapper = jsonMapper;
- }
-
- @Override
- public String getPathForHadoop()
- {
- return null;
- }
-
- @Deprecated
- @Override
- public String getPathForHadoop(final String dataSource)
- {
- return getPathForHadoop();
- }
-
- @Override
- public DataSegment push(final File indexFilesDir, final DataSegment inSegment, final boolean useUniquePath)
- {
- final String segmentPath = CloudFilesUtils.buildCloudFilesPath(
- this.config.getBasePath(),
- getStorageDir(inSegment, useUniquePath)
- );
-
- File descriptorFile = null;
- File zipOutFile = null;
-
- try {
- final File descFile = descriptorFile = File.createTempFile("descriptor", ".json");
- final File outFile = zipOutFile = File.createTempFile("druid", "index.zip");
-
- final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile);
-
- log.info("Copying segment[%s] to CloudFiles at location[%s]", inSegment.getId(), segmentPath);
- return CloudFilesUtils.retryCloudFilesOperation(
- () -> {
- CloudFilesObject segmentData = new CloudFilesObject(
- segmentPath,
- outFile,
- objectApi.getRegion(),
- objectApi.getContainer()
- );
-
- log.info("Pushing %s.", segmentData.getPath());
- objectApi.put(segmentData);
-
- // Avoid using Guava in DataSegmentPushers because they might be used with very diverse Guava versions in
- // runtime, and because Guava deletes methods over time, that causes incompatibilities.
- Files.write(descFile.toPath(), jsonMapper.writeValueAsBytes(inSegment));
- CloudFilesObject descriptorData = new CloudFilesObject(
- segmentPath,
- descFile,
- objectApi.getRegion(),
- objectApi.getContainer()
- );
- log.info("Pushing %s.", descriptorData.getPath());
- objectApi.put(descriptorData);
-
- final DataSegment outSegment = inSegment
- .withSize(indexSize)
- .withLoadSpec(makeLoadSpec(new URI(segmentData.getPath())))
- .withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir));
-
- return outSegment;
- },
- this.config.getOperationMaxRetries()
- );
- }
- catch (Exception e) {
- throw new RuntimeException(e);
- }
- finally {
- if (zipOutFile != null) {
- log.info("Deleting zipped index File[%s]", zipOutFile);
- zipOutFile.delete();
- }
-
- if (descriptorFile != null) {
- log.info("Deleting descriptor file[%s]", descriptorFile);
- descriptorFile.delete();
- }
- }
- }
-
- @Override
- public Map makeLoadSpec(URI uri)
- {
- return ImmutableMap.of(
- "type",
- CloudFilesStorageDruidModule.SCHEME,
- "region",
- objectApi.getRegion(),
- "container",
- objectApi.getContainer(),
- "path",
- uri.toString()
- );
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherConfig.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherConfig.java
deleted file mode 100644
index 68619737ab93..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherConfig.java
+++ /dev/null
@@ -1,90 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-
-import javax.validation.constraints.NotNull;
-
-/**
- */
-public class CloudFilesDataSegmentPusherConfig
-{
- @JsonProperty
- @NotNull
- private String region;
-
- @JsonProperty
- @NotNull
- private String container;
-
- @JsonProperty
- @NotNull
- private String basePath;
-
- @JsonProperty
- private int operationMaxRetries = 10;
-
- public void setRegion(String region)
- {
- this.region = region;
- }
-
- public void setContainer(String container)
- {
- this.container = container;
- }
-
- public void setBasePath(String basePath)
- {
- this.basePath = basePath;
- }
-
- @SuppressWarnings("unused") // Used by Jackson deserialization?
- public void setOperationMaxRetries(int operationMaxRetries)
- {
- this.operationMaxRetries = operationMaxRetries;
- }
-
- public String getRegion()
- {
- Preconditions.checkNotNull(region);
- return region;
- }
-
- public String getContainer()
- {
- Preconditions.checkNotNull(container);
- return container;
- }
-
- public String getBasePath()
- {
- Preconditions.checkNotNull(basePath);
- return basePath;
- }
-
- public int getOperationMaxRetries()
- {
- return operationMaxRetries;
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesLoadSpec.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesLoadSpec.java
deleted file mode 100644
index d06261dd9e68..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesLoadSpec.java
+++ /dev/null
@@ -1,67 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.fasterxml.jackson.annotation.JacksonInject;
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.fasterxml.jackson.annotation.JsonTypeName;
-import com.google.common.base.Preconditions;
-import org.apache.druid.segment.loading.LoadSpec;
-import org.apache.druid.segment.loading.SegmentLoadingException;
-
-import java.io.File;
-
-@JsonTypeName(CloudFilesStorageDruidModule.SCHEME)
-public class CloudFilesLoadSpec implements LoadSpec
-{
-
- @JsonProperty
- private final String region;
-
- @JsonProperty
- private final String container;
-
- @JsonProperty
- private final String path;
-
- private final CloudFilesDataSegmentPuller puller;
-
- @JsonCreator
- public CloudFilesLoadSpec(
- @JsonProperty("region") String region, @JsonProperty("container") String container,
- @JsonProperty("path") String path, @JacksonInject CloudFilesDataSegmentPuller puller
- )
- {
- Preconditions.checkNotNull(region);
- Preconditions.checkNotNull(container);
- Preconditions.checkNotNull(path);
- this.container = container;
- this.region = region;
- this.path = path;
- this.puller = puller;
- }
-
- @Override
- public LoadSpecResult loadSegment(File file) throws SegmentLoadingException
- {
- return new LoadSpecResult(puller.getSegmentFiles(region, container, path, file).size());
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesObject.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesObject.java
deleted file mode 100644
index 5f44856051c8..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesObject.java
+++ /dev/null
@@ -1,83 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.google.common.io.ByteSource;
-import com.google.common.io.Files;
-import org.jclouds.io.Payload;
-import org.jclouds.io.Payloads;
-
-import java.io.File;
-
-public class CloudFilesObject
-{
-
- private Payload payload;
- private String path;
- private final String region;
- private final String container;
-
- public CloudFilesObject(final String basePath, final File file, final String region, final String container)
- {
- this(region, container);
- ByteSource byteSource = Files.asByteSource(file);
- this.payload = Payloads.newByteSourcePayload(byteSource);
- this.path = CloudFilesUtils.buildCloudFilesPath(basePath, file.getName());
- }
-
- public CloudFilesObject(final Payload payload, final String region, final String container, final String path)
- {
- this(region, container, path);
- this.payload = payload;
- }
-
- private CloudFilesObject(final String region, final String container, final String path)
- {
- this(region, container);
- this.path = path;
- }
-
- private CloudFilesObject(final String region, final String container)
- {
- this.region = region;
- this.container = container;
- }
-
- public String getRegion()
- {
- return region;
- }
-
- public String getContainer()
- {
- return container;
- }
-
- public String getPath()
- {
- return path;
- }
-
- public Payload getPayload()
- {
- return payload;
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxy.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxy.java
deleted file mode 100644
index 07d8363fada9..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxy.java
+++ /dev/null
@@ -1,72 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import org.jclouds.http.options.GetOptions;
-import org.jclouds.io.Payload;
-import org.jclouds.openstack.swift.v1.domain.SwiftObject;
-import org.jclouds.openstack.swift.v1.features.ObjectApi;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-
-public class CloudFilesObjectApiProxy
-{
- private final ObjectApi objectApi;
- private final String region;
- private final String container;
-
- public CloudFilesObjectApiProxy(final CloudFilesApi cloudFilesApi, final String region, final String container)
- {
- this.region = region;
- this.container = container;
- this.objectApi = cloudFilesApi.getObjectApi(region, container);
- }
-
- public String getRegion()
- {
- return region;
- }
-
- public String getContainer()
- {
- return container;
- }
-
- public String put(final CloudFilesObject cloudFilesObject)
- {
- return objectApi.put(cloudFilesObject.getPath(), cloudFilesObject.getPayload());
- }
-
- public CloudFilesObject get(String path, long start)
- {
- final SwiftObject swiftObject;
- if (start == 0) {
- swiftObject = objectApi.get(path);
- } else {
- swiftObject = objectApi.get(path, new GetOptions().startAt(start));
- }
- Payload payload = swiftObject.getPayload();
- return new CloudFilesObject(payload, this.region, this.container, path);
- }
-
- public boolean exists(String path)
- {
- return objectApi.getWithoutBody(path) != null;
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
deleted file mode 100644
index c54342fd32de..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesStorageDruidModule.java
+++ /dev/null
@@ -1,117 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.fasterxml.jackson.core.Version;
-import com.fasterxml.jackson.databind.Module;
-import com.google.common.collect.ImmutableList;
-import com.google.common.collect.ImmutableSet;
-import com.google.inject.Binder;
-import com.google.inject.Provides;
-import org.apache.druid.guice.Binders;
-import org.apache.druid.guice.JsonConfigProvider;
-import org.apache.druid.guice.LazySingleton;
-import org.apache.druid.initialization.DruidModule;
-import org.apache.druid.java.util.common.logger.Logger;
-import org.jclouds.ContextBuilder;
-import org.jclouds.logging.slf4j.config.SLF4JLoggingModule;
-import org.jclouds.openstack.v2_0.config.InternalUrlModule;
-import org.jclouds.osgi.ProviderRegistry;
-import org.jclouds.rackspace.cloudfiles.uk.CloudFilesUKProviderMetadata;
-import org.jclouds.rackspace.cloudfiles.us.CloudFilesUSProviderMetadata;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-
-import java.util.List;
-
-/**
- */
-public class CloudFilesStorageDruidModule implements DruidModule
-{
-
- private static final Logger log = new Logger(CloudFilesStorageDruidModule.class);
- public static final String SCHEME = "cloudfiles";
-
- @Override
- public List extends Module> getJacksonModules()
- {
- log.info("Getting jackson modules...");
-
- return ImmutableList.of(
- new Module()
- {
- @Override
- public String getModuleName()
- {
- return "CloudFiles-" + System.identityHashCode(this);
- }
-
- @Override
- public Version version()
- {
- return Version.unknownVersion();
- }
-
- @Override
- public void setupModule(SetupContext context)
- {
- context.registerSubtypes(CloudFilesLoadSpec.class);
- }
- }
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
- log.info("Configuring CloudFilesStorageDruidModule...");
- JsonConfigProvider.bind(binder, "druid.storage", CloudFilesDataSegmentPusherConfig.class);
- JsonConfigProvider.bind(binder, "druid.cloudfiles", CloudFilesAccountConfig.class);
-
- Binders.dataSegmentPusherBinder(binder).addBinding(SCHEME).to(CloudFilesDataSegmentPusher.class)
- .in(LazySingleton.class);
-
- log.info("Configured CloudFilesStorageDruidModule.");
- }
-
- @Provides
- @LazySingleton
- public CloudFilesApi getCloudFilesApi(final CloudFilesAccountConfig config)
- {
- log.info("Building Cloud Files Api...");
-
- Iterable modules;
- if (config.getUseServiceNet()) {
- log.info("Configuring Cloud Files Api to use the internal service network...");
- modules = ImmutableSet.of(new SLF4JLoggingModule(), new InternalUrlModule());
- } else {
- log.info("Configuring Cloud Files Api to use the public network...");
- modules = ImmutableSet.of(new SLF4JLoggingModule());
- }
-
- ProviderRegistry.registerProvider(CloudFilesUSProviderMetadata.builder().build());
- ProviderRegistry.registerProvider(CloudFilesUKProviderMetadata.builder().build());
- ContextBuilder cb = ContextBuilder.newBuilder(config.getProvider())
- .credentials(config.getUserName(), config.getApiKey()).modules(modules);
- CloudFilesApi cfa = cb.buildApi(CloudFilesApi.class);
- log.info("Cloud Files Api built.");
- return cfa;
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesUtils.java b/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesUtils.java
deleted file mode 100644
index bc883b28fd84..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/java/org/apache/druid/storage/cloudfiles/CloudFilesUtils.java
+++ /dev/null
@@ -1,70 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.google.common.base.Predicate;
-import org.apache.druid.java.util.common.RetryUtils;
-import org.apache.druid.java.util.common.RetryUtils.Task;
-
-import java.io.IOException;
-
-/**
- *
- */
-public class CloudFilesUtils
-{
-
- public static final Predicate CLOUDFILESRETRY = new Predicate()
- {
- @Override
- public boolean apply(Throwable e)
- {
- if (e == null) {
- return false;
- } else if (e instanceof IOException) {
- return true;
- } else {
- return apply(e.getCause());
- }
- }
- };
-
- /**
- * Retries CloudFiles operations that fail due to io-related exceptions.
- */
- public static T retryCloudFilesOperation(Task f, final int maxTries) throws Exception
- {
- return RetryUtils.retry(f, CLOUDFILESRETRY, maxTries);
- }
-
- public static String buildCloudFilesPath(String basePath, final String fileName)
- {
- String path = fileName;
- if (!basePath.isEmpty()) {
- int lastSlashIndex = basePath.lastIndexOf('/');
- if (lastSlashIndex != -1) {
- basePath = basePath.substring(0, lastSlashIndex);
- }
- path = basePath + "/" + fileName;
- }
- return path;
- }
-
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
deleted file mode 100644
index 81a1411f89b6..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ /dev/null
@@ -1,19 +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.
-
-org.apache.druid.storage.cloudfiles.CloudFilesStorageDruidModule
-org.apache.druid.firehose.cloudfiles.CloudFilesFirehoseDruidModule
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java
deleted file mode 100644
index 48f3ca694fa3..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/firehose/cloudfiles/StaticCloudFilesFirehoseFactoryTest.java
+++ /dev/null
@@ -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 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;
- }
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSourceTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSourceTest.java
deleted file mode 100644
index 6a07063c722e..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesByteSourceTest.java
+++ /dev/null
@@ -1,87 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import org.easymock.EasyMock;
-import org.easymock.EasyMockSupport;
-import org.jclouds.io.Payload;
-import org.junit.Assert;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.io.InputStream;
-
-public class CloudFilesByteSourceTest extends EasyMockSupport
-{
- @Test
- public void openStreamTest() throws IOException
- {
- final String path = "path";
-
- CloudFilesObjectApiProxy objectApi = createMock(CloudFilesObjectApiProxy.class);
- CloudFilesObject cloudFilesObject = createMock(CloudFilesObject.class);
- Payload payload = createMock(Payload.class);
- InputStream stream = createMock(InputStream.class);
-
- EasyMock.expect(objectApi.get(path, 0)).andReturn(cloudFilesObject);
- EasyMock.expect(cloudFilesObject.getPayload()).andReturn(payload);
- EasyMock.expect(payload.openStream()).andReturn(stream);
- payload.close();
-
- replayAll();
-
- CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path);
- Assert.assertEquals(stream, byteSource.openStream());
- byteSource.closeStream();
-
- verifyAll();
- }
-
- @Test()
- public void openStreamWithRecoverableErrorTest() throws IOException
- {
- final String path = "path";
-
- CloudFilesObjectApiProxy objectApi = createMock(CloudFilesObjectApiProxy.class);
- CloudFilesObject cloudFilesObject = createMock(CloudFilesObject.class);
- Payload payload = createMock(Payload.class);
- InputStream stream = createMock(InputStream.class);
-
- EasyMock.expect(objectApi.get(path, 0)).andReturn(cloudFilesObject);
- EasyMock.expect(cloudFilesObject.getPayload()).andReturn(payload);
- EasyMock.expect(payload.openStream()).andThrow(new IOException()).andReturn(stream);
- payload.close();
-
- replayAll();
-
- CloudFilesByteSource byteSource = new CloudFilesByteSource(objectApi, path);
- try {
- byteSource.openStream();
- }
- catch (Exception e) {
- Assert.assertEquals("Recoverable exception", e.getMessage());
- }
-
- Assert.assertEquals(stream, byteSource.openStream());
- byteSource.closeStream();
-
- verifyAll();
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
deleted file mode 100644
index 8a2e3f697eb3..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesDataSegmentPusherTest.java
+++ /dev/null
@@ -1,92 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import com.google.common.io.Files;
-import org.apache.druid.jackson.DefaultObjectMapper;
-import org.apache.druid.java.util.common.Intervals;
-import org.apache.druid.timeline.DataSegment;
-import org.apache.druid.timeline.partition.NoneShardSpec;
-import org.easymock.EasyMock;
-import org.jclouds.openstack.swift.v1.features.ObjectApi;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-import org.junit.Assert;
-import org.junit.Rule;
-import org.junit.Test;
-import org.junit.rules.TemporaryFolder;
-
-import java.io.File;
-import java.util.ArrayList;
-import java.util.HashMap;
-
-/**
- */
-public class CloudFilesDataSegmentPusherTest
-{
- @Rule
- public final TemporaryFolder tempFolder = new TemporaryFolder();
-
- @Test
- public void testPush() throws Exception
- {
- ObjectApi objectApi = EasyMock.createStrictMock(ObjectApi.class);
- EasyMock.expect(objectApi.put(EasyMock.anyString(), EasyMock.anyObject())).andReturn(null).atLeastOnce();
- EasyMock.replay(objectApi);
-
- CloudFilesApi api = EasyMock.createStrictMock(CloudFilesApi.class);
- EasyMock.expect(api.getObjectApi(EasyMock.anyString(), EasyMock.anyString()))
- .andReturn(objectApi)
- .atLeastOnce();
- EasyMock.replay(api);
-
-
- CloudFilesDataSegmentPusherConfig config = new CloudFilesDataSegmentPusherConfig();
- config.setRegion("region");
- config.setContainer("container");
- config.setBasePath("basePath");
-
- CloudFilesDataSegmentPusher pusher = new CloudFilesDataSegmentPusher(api, config, new DefaultObjectMapper());
-
- // Create a mock segment on disk
- File tmp = tempFolder.newFile("version.bin");
-
- final byte[] data = new byte[]{0x0, 0x0, 0x0, 0x1};
- Files.write(data, tmp);
- final long size = data.length;
-
- DataSegment segmentToPush = new DataSegment(
- "foo",
- Intervals.of("2015/2016"),
- "0",
- new HashMap<>(),
- new ArrayList<>(),
- new ArrayList<>(),
- NoneShardSpec.instance(),
- 0,
- size
- );
-
- DataSegment segment = pusher.push(tempFolder.getRoot(), segmentToPush, false);
-
- Assert.assertEquals(segmentToPush.getSize(), segment.getSize());
-
- EasyMock.verify(api);
- }
-}
diff --git a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxyTest.java b/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxyTest.java
deleted file mode 100644
index eb3b61c21494..000000000000
--- a/extensions-contrib/cloudfiles-extensions/src/test/java/org/apache/druid/storage/cloudfiles/CloudFilesObjectApiProxyTest.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/*
- * Licensed to the Apache Software Foundation (ASF) under one
- * or more contributor license agreements. See the NOTICE file
- * distributed with this work for additional information
- * regarding copyright ownership. The ASF licenses this file
- * to you under the Apache License, Version 2.0 (the
- * "License"); you may not use this file except in compliance
- * with the License. You may obtain a copy of the License at
- *
- * http://www.apache.org/licenses/LICENSE-2.0
- *
- * Unless required by applicable law or agreed to in writing,
- * software distributed under the License is distributed on an
- * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
- * KIND, either express or implied. See the License for the
- * specific language governing permissions and limitations
- * under the License.
- */
-
-package org.apache.druid.storage.cloudfiles;
-
-import org.easymock.EasyMock;
-import org.easymock.EasyMockSupport;
-import org.jclouds.io.Payload;
-import org.jclouds.openstack.swift.v1.domain.SwiftObject;
-import org.jclouds.openstack.swift.v1.features.ObjectApi;
-import org.jclouds.rackspace.cloudfiles.v1.CloudFilesApi;
-import org.junit.Assert;
-import org.junit.Test;
-
-public class CloudFilesObjectApiProxyTest extends EasyMockSupport
-{
- @Test
- public void getTest()
- {
- final String path = "path";
- final String region = "region";
- final String container = "container";
-
- CloudFilesApi cloudFilesApi = createMock(CloudFilesApi.class);
- ObjectApi objectApi = createMock(ObjectApi.class);
- SwiftObject swiftObject = createMock(SwiftObject.class);
- Payload payload = createMock(Payload.class);
-
- EasyMock.expect(cloudFilesApi.getObjectApi(region, container)).andReturn(objectApi);
- EasyMock.expect(objectApi.get(path)).andReturn(swiftObject);
- EasyMock.expect(swiftObject.getPayload()).andReturn(payload);
-
- replayAll();
-
- CloudFilesObjectApiProxy cfoApiProxy = new CloudFilesObjectApiProxy(cloudFilesApi, region, container);
- CloudFilesObject cloudFilesObject = cfoApiProxy.get(path, 0);
-
- Assert.assertEquals(cloudFilesObject.getPayload(), payload);
- Assert.assertEquals(cloudFilesObject.getRegion(), region);
- Assert.assertEquals(cloudFilesObject.getContainer(), container);
- Assert.assertEquals(cloudFilesObject.getPath(), path);
-
- verifyAll();
- }
-}
diff --git a/extensions-contrib/distinctcount/pom.xml b/extensions-contrib/distinctcount/pom.xml
deleted file mode 100644
index 5933e1ce51b0..000000000000
--- a/extensions-contrib/distinctcount/pom.xml
+++ /dev/null
@@ -1,117 +0,0 @@
-
-
-
-
- 4.0.0
-
- org.apache.druid.extensions.contrib
- druid-distinctcount
- druid-distinctcount
- druid-distinctcount
-
-
- org.apache.druid
- druid
- 0.19.0-iap2-SNAPSHOT
- ../../pom.xml
-
-
-
-
- org.apache.druid
- druid-core
- ${project.parent.version}
- provided
-
-
- org.apache.druid
- druid-processing
- ${project.parent.version}
- provided
-
-
- com.google.code.findbugs
- jsr305
- provided
-
-
- com.fasterxml.jackson.core
- jackson-annotations
- provided
-
-
- com.google.guava
- guava
- provided
-
-
- joda-time
- joda-time
- provided
-
-
- com.google.inject
- guice
- provided
-
-
- com.fasterxml.jackson.core
- jackson-databind
- provided
-
-
- it.unimi.dsi
- fastutil
- provided
-
-
-
-
- org.apache.druid
- druid-core
- ${project.parent.version}
- test
- test-jar
-
-
- org.apache.druid
- druid-processing
- ${project.parent.version}
- test
- test-jar
-
-
- junit
- junit
- test
-
-
- org.easymock
- easymock
- test
-
-
- org.hamcrest
- hamcrest-core
- test
-
-
-
-
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/BitMapFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/BitMapFactory.java
deleted file mode 100644
index 5bf148d063f4..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/BitMapFactory.java
+++ /dev/null
@@ -1,42 +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.query.aggregation.distinctcount;
-
-import com.fasterxml.jackson.annotation.JsonSubTypes;
-import com.fasterxml.jackson.annotation.JsonTypeInfo;
-import org.apache.druid.collections.bitmap.MutableBitmap;
-
-/**
- */
-@JsonTypeInfo(use = JsonTypeInfo.Id.NAME, property = "type", defaultImpl = RoaringBitMapFactory.class)
-@JsonSubTypes(value = {
- @JsonSubTypes.Type(name = "java", value = JavaBitMapFactory.class),
- @JsonSubTypes.Type(name = "concise", value = ConciseBitMapFactory.class),
- @JsonSubTypes.Type(name = "roaring", value = RoaringBitMapFactory.class)
-})
-public interface BitMapFactory
-{
- /**
- * Create a new empty bitmap
- *
- * @return the new bitmap
- */
- MutableBitmap makeEmptyMutableBitmap();
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/ConciseBitMapFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/ConciseBitMapFactory.java
deleted file mode 100644
index aa0833ce9f21..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/ConciseBitMapFactory.java
+++ /dev/null
@@ -1,57 +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.query.aggregation.distinctcount;
-
-import org.apache.druid.collections.bitmap.BitmapFactory;
-import org.apache.druid.collections.bitmap.ConciseBitmapFactory;
-import org.apache.druid.collections.bitmap.MutableBitmap;
-
-public class ConciseBitMapFactory implements BitMapFactory
-{
- private static final BitmapFactory BITMAP_FACTORY = new ConciseBitmapFactory();
-
- public ConciseBitMapFactory()
- {
- }
-
- @Override
- public MutableBitmap makeEmptyMutableBitmap()
- {
- return BITMAP_FACTORY.makeEmptyMutableBitmap();
- }
-
- @Override
- public String toString()
- {
- return "ConciseBitMapFactory";
- }
-
- @Override
- public boolean equals(Object o)
- {
- return this == o || o instanceof ConciseBitMapFactory;
- }
-
- @Override
- public int hashCode()
- {
- return 0;
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregator.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregator.java
deleted file mode 100644
index d7099e368914..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregator.java
+++ /dev/null
@@ -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.query.aggregation.distinctcount;
-
-import org.apache.druid.collections.bitmap.MutableBitmap;
-import org.apache.druid.query.aggregation.Aggregator;
-import org.apache.druid.segment.DimensionSelector;
-import org.apache.druid.segment.data.IndexedInts;
-
-public class DistinctCountAggregator implements Aggregator
-{
-
- private final DimensionSelector selector;
- private final MutableBitmap mutableBitmap;
-
- public DistinctCountAggregator(
- DimensionSelector selector,
- MutableBitmap mutableBitmap
- )
- {
- this.selector = selector;
- this.mutableBitmap = mutableBitmap;
- }
-
- @Override
- public void aggregate()
- {
- IndexedInts row = selector.getRow();
- for (int i = 0, rowSize = row.size(); i < rowSize; i++) {
- int index = row.get(i);
- mutableBitmap.add(index);
- }
- }
-
- @Override
- public Object get()
- {
- return mutableBitmap.size();
- }
-
- @Override
- public float getFloat()
- {
- return (float) mutableBitmap.size();
- }
-
- @Override
- public void close()
- {
- mutableBitmap.clear();
- }
-
- @Override
- public long getLong()
- {
- return (long) mutableBitmap.size();
- }
-
- @Override
- public double getDouble()
- {
- return (double) mutableBitmap.size();
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java
deleted file mode 100644
index b74aba1e24bd..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountAggregatorFactory.java
+++ /dev/null
@@ -1,246 +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.query.aggregation.distinctcount;
-
-import com.fasterxml.jackson.annotation.JsonCreator;
-import com.fasterxml.jackson.annotation.JsonProperty;
-import com.google.common.base.Preconditions;
-import com.google.common.primitives.Longs;
-import org.apache.druid.java.util.common.StringUtils;
-import org.apache.druid.query.aggregation.AggregateCombiner;
-import org.apache.druid.query.aggregation.Aggregator;
-import org.apache.druid.query.aggregation.AggregatorFactory;
-import org.apache.druid.query.aggregation.AggregatorUtil;
-import org.apache.druid.query.aggregation.BufferAggregator;
-import org.apache.druid.query.aggregation.LongSumAggregateCombiner;
-import org.apache.druid.query.aggregation.LongSumAggregatorFactory;
-import org.apache.druid.query.dimension.DefaultDimensionSpec;
-import org.apache.druid.segment.ColumnSelectorFactory;
-import org.apache.druid.segment.DimensionSelector;
-
-import javax.annotation.Nullable;
-import java.nio.ByteBuffer;
-import java.util.Collections;
-import java.util.Comparator;
-import java.util.List;
-
-public class DistinctCountAggregatorFactory extends AggregatorFactory
-{
- private static final BitMapFactory DEFAULT_BITMAP_FACTORY = new RoaringBitMapFactory();
-
- private final String name;
- private final String fieldName;
- private final BitMapFactory bitMapFactory;
-
- @JsonCreator
- public DistinctCountAggregatorFactory(
- @JsonProperty("name") String name,
- @JsonProperty("fieldName") String fieldName,
- @JsonProperty("bitmapFactory") BitMapFactory bitMapFactory
- )
- {
- Preconditions.checkNotNull(name);
- Preconditions.checkNotNull(fieldName);
- this.name = name;
- this.fieldName = fieldName;
- this.bitMapFactory = bitMapFactory == null ? DEFAULT_BITMAP_FACTORY : bitMapFactory;
- }
-
- @Override
- public Aggregator factorize(ColumnSelectorFactory columnFactory)
- {
- DimensionSelector selector = makeDimensionSelector(columnFactory);
- if (selector == null) {
- return new NoopDistinctCountAggregator();
- } else {
- return new DistinctCountAggregator(
- selector,
- bitMapFactory.makeEmptyMutableBitmap()
- );
- }
- }
-
- @Override
- public BufferAggregator factorizeBuffered(ColumnSelectorFactory columnFactory)
- {
- DimensionSelector selector = makeDimensionSelector(columnFactory);
- if (selector == null) {
- return NoopDistinctCountBufferAggregator.instance();
- } else {
- return new DistinctCountBufferAggregator(makeDimensionSelector(columnFactory));
- }
- }
-
- private DimensionSelector makeDimensionSelector(final ColumnSelectorFactory columnFactory)
- {
- return columnFactory.makeDimensionSelector(new DefaultDimensionSpec(fieldName, fieldName));
- }
-
- @Override
- public Comparator getComparator()
- {
- return new Comparator()
- {
- @Override
- public int compare(Object o, Object o1)
- {
- return Longs.compare(((Number) o).longValue(), ((Number) o1).longValue());
- }
- };
- }
-
- @Override
- public Object combine(Object lhs, Object rhs)
- {
- if (lhs == null && rhs == null) {
- return 0L;
- }
- if (rhs == null) {
- return ((Number) lhs).longValue();
- }
- if (lhs == null) {
- return ((Number) rhs).longValue();
- }
- return ((Number) lhs).longValue() + ((Number) rhs).longValue();
- }
-
- @Override
- public AggregateCombiner makeAggregateCombiner()
- {
- // This is likely wrong as well as combine(), see https://github.com/apache/druid/pull/2602#issuecomment-321224202
- return new LongSumAggregateCombiner();
- }
-
- @Override
- public AggregatorFactory getCombiningFactory()
- {
- return new LongSumAggregatorFactory(name, name);
- }
-
- @Override
- public List getRequiredColumns()
- {
- return Collections.singletonList(
- new DistinctCountAggregatorFactory(fieldName, fieldName, bitMapFactory)
- );
- }
-
- @Override
- public Object deserialize(Object object)
- {
- return object;
- }
-
- @Nullable
- @Override
- public Object finalizeComputation(@Nullable Object object)
- {
- return object;
- }
-
- @JsonProperty
- public String getFieldName()
- {
- return fieldName;
- }
-
- @JsonProperty("bitmapFactory")
- public BitMapFactory getBitMapFactory()
- {
- return bitMapFactory;
- }
-
- @Override
- @JsonProperty
- public String getName()
- {
- return name;
- }
-
- @Override
- public List requiredFields()
- {
- return Collections.singletonList(fieldName);
- }
-
- @Override
- public byte[] getCacheKey()
- {
- byte[] fieldNameBytes = StringUtils.toUtf8(fieldName);
- byte[] bitMapFactoryCacheKey = StringUtils.toUtf8(bitMapFactory.toString());
- return ByteBuffer.allocate(2 + fieldNameBytes.length + bitMapFactoryCacheKey.length)
- .put(AggregatorUtil.DISTINCT_COUNT_CACHE_KEY)
- .put(fieldNameBytes)
- .put(AggregatorUtil.STRING_SEPARATOR)
- .put(bitMapFactoryCacheKey)
- .array();
- }
-
- @Override
- public String getTypeName()
- {
- return "distinctCount";
- }
-
- @Override
- public int getMaxIntermediateSize()
- {
- return Long.BYTES;
- }
-
- @Override
- public boolean equals(Object o)
- {
- if (this == o) {
- return true;
- }
- if (o == null || getClass() != o.getClass()) {
- return false;
- }
-
- DistinctCountAggregatorFactory that = (DistinctCountAggregatorFactory) o;
-
- if (!fieldName.equals(that.fieldName)) {
- return false;
- }
- if (!name.equals(that.name)) {
- return false;
- }
-
- return true;
- }
-
- @Override
- public int hashCode()
- {
- int result = name.hashCode();
- result = 31 * result + fieldName.hashCode();
- return result;
- }
-
- @Override
- public String toString()
- {
- return "DistinctCountAggregatorFactory{" +
- "name='" + name + '\'' +
- ", fieldName='" + fieldName + '\'' +
- '}';
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java
deleted file mode 100644
index 0b1ebf556d60..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountBufferAggregator.java
+++ /dev/null
@@ -1,108 +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.query.aggregation.distinctcount;
-
-import it.unimi.dsi.fastutil.ints.Int2ObjectMap;
-import it.unimi.dsi.fastutil.ints.Int2ObjectOpenHashMap;
-import org.apache.druid.collections.bitmap.MutableBitmap;
-import org.apache.druid.collections.bitmap.WrappedRoaringBitmap;
-import org.apache.druid.query.aggregation.BufferAggregator;
-import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
-import org.apache.druid.segment.DimensionSelector;
-import org.apache.druid.segment.data.IndexedInts;
-
-import java.nio.ByteBuffer;
-
-public class DistinctCountBufferAggregator implements BufferAggregator
-{
- private final DimensionSelector selector;
- private final Int2ObjectMap mutableBitmapCollection = new Int2ObjectOpenHashMap<>();
-
- public DistinctCountBufferAggregator(
- DimensionSelector selector
- )
- {
- this.selector = selector;
- }
-
- @Override
- public void init(ByteBuffer buf, int position)
- {
- buf.putLong(position, 0L);
- }
-
- @Override
- public void aggregate(ByteBuffer buf, int position)
- {
- MutableBitmap mutableBitmap = getMutableBitmap(position);
- IndexedInts row = selector.getRow();
- for (int i = 0, rowSize = row.size(); i < rowSize; i++) {
- int index = row.get(i);
- mutableBitmap.add(index);
- }
- buf.putLong(position, mutableBitmap.size());
- }
-
- private MutableBitmap getMutableBitmap(int position)
- {
- MutableBitmap mutableBitmap = mutableBitmapCollection.get(position);
- if (mutableBitmap == null) {
- mutableBitmap = new WrappedRoaringBitmap();
- mutableBitmapCollection.put(position, mutableBitmap);
- }
- return mutableBitmap;
- }
-
- @Override
- public Object get(ByteBuffer buf, int position)
- {
- return buf.getLong(position);
- }
-
- @Override
- public float getFloat(ByteBuffer buf, int position)
- {
- return (float) buf.getLong(position);
- }
-
- @Override
- public long getLong(ByteBuffer buf, int position)
- {
- return buf.getLong(position);
- }
-
- @Override
- public double getDouble(ByteBuffer buf, int position)
- {
- return (double) buf.getLong(position);
- }
-
- @Override
- public void close()
- {
- mutableBitmapCollection.clear();
- }
-
- @Override
- public void inspectRuntimeShape(RuntimeShapeInspector inspector)
- {
- inspector.visit("selector", selector);
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountDruidModule.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountDruidModule.java
deleted file mode 100644
index 4e2cb12a239c..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountDruidModule.java
+++ /dev/null
@@ -1,49 +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.query.aggregation.distinctcount;
-
-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 DistinctCountDruidModule implements DruidModule
-{
- public static final String DISTINCT_COUNT = "distinctCount";
-
- @Override
- public List extends Module> getJacksonModules()
- {
- return ImmutableList.of(
- new SimpleModule("DistinctCountModule").registerSubtypes(
- new NamedType(DistinctCountAggregatorFactory.class, DISTINCT_COUNT)
- )
- );
- }
-
- @Override
- public void configure(Binder binder)
- {
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/JavaBitMapFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/JavaBitMapFactory.java
deleted file mode 100644
index 2b4992c81805..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/JavaBitMapFactory.java
+++ /dev/null
@@ -1,57 +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.query.aggregation.distinctcount;
-
-import org.apache.druid.collections.bitmap.BitSetBitmapFactory;
-import org.apache.druid.collections.bitmap.BitmapFactory;
-import org.apache.druid.collections.bitmap.MutableBitmap;
-
-public class JavaBitMapFactory implements BitMapFactory
-{
- private static final BitmapFactory BITMAP_FACTORY = new BitSetBitmapFactory();
-
- public JavaBitMapFactory()
- {
- }
-
- @Override
- public MutableBitmap makeEmptyMutableBitmap()
- {
- return BITMAP_FACTORY.makeEmptyMutableBitmap();
- }
-
- @Override
- public String toString()
- {
- return "JavaBitMapFactory";
- }
-
- @Override
- public boolean equals(Object o)
- {
- return this == o || o instanceof JavaBitMapFactory;
- }
-
- @Override
- public int hashCode()
- {
- return 0;
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/NoopDistinctCountAggregator.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/NoopDistinctCountAggregator.java
deleted file mode 100644
index d6b2a1be06b8..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/NoopDistinctCountAggregator.java
+++ /dev/null
@@ -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.query.aggregation.distinctcount;
-
-import org.apache.druid.query.aggregation.Aggregator;
-
-public class NoopDistinctCountAggregator implements Aggregator
-{
- public NoopDistinctCountAggregator()
- {
- }
-
- @Override
- public void aggregate()
- {
- }
-
- @Override
- public Object get()
- {
- return 0L;
- }
-
- @Override
- public float getFloat()
- {
- return 0.0f;
- }
-
- @Override
- public long getLong()
- {
- return 0L;
- }
-
- @Override
- public double getDouble()
- {
- return 0.0;
- }
-
- @Override
- public void close()
- {
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/NoopDistinctCountBufferAggregator.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/NoopDistinctCountBufferAggregator.java
deleted file mode 100644
index 1ac10183a99d..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/NoopDistinctCountBufferAggregator.java
+++ /dev/null
@@ -1,88 +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.query.aggregation.distinctcount;
-
-import org.apache.druid.query.aggregation.BufferAggregator;
-import org.apache.druid.query.monomorphicprocessing.RuntimeShapeInspector;
-
-import java.nio.ByteBuffer;
-
-/**
- * The difference from {@link org.apache.druid.query.aggregation.NoopBufferAggregator} is that
- * NoopDistinctCountBufferAggregator returns 0 instead of null from {@link #get(ByteBuffer, int)}.
- */
-public final class NoopDistinctCountBufferAggregator implements BufferAggregator
-{
- private static final NoopDistinctCountBufferAggregator INSTANCE = new NoopDistinctCountBufferAggregator();
-
- static NoopDistinctCountBufferAggregator instance()
- {
- return INSTANCE;
- }
-
- private NoopDistinctCountBufferAggregator()
- {
- }
-
- @Override
- public void init(ByteBuffer buf, int position)
- {
- }
-
- @Override
- public void aggregate(ByteBuffer buf, int position)
- {
- }
-
- @Override
- public Object get(ByteBuffer buf, int position)
- {
- return 0L;
- }
-
- @Override
- public float getFloat(ByteBuffer buf, int position)
- {
- return (float) 0;
- }
-
- @Override
- public long getLong(ByteBuffer buf, int position)
- {
- return (long) 0;
- }
-
- @Override
- public double getDouble(ByteBuffer buf, int position)
- {
- return 0;
- }
-
- @Override
- public void close()
- {
- }
-
- @Override
- public void inspectRuntimeShape(RuntimeShapeInspector inspector)
- {
- // nothing to inspect
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/RoaringBitMapFactory.java b/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/RoaringBitMapFactory.java
deleted file mode 100644
index 48b9db19531a..000000000000
--- a/extensions-contrib/distinctcount/src/main/java/org/apache/druid/query/aggregation/distinctcount/RoaringBitMapFactory.java
+++ /dev/null
@@ -1,57 +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.query.aggregation.distinctcount;
-
-import org.apache.druid.collections.bitmap.BitmapFactory;
-import org.apache.druid.collections.bitmap.MutableBitmap;
-import org.apache.druid.collections.bitmap.RoaringBitmapFactory;
-
-public class RoaringBitMapFactory implements BitMapFactory
-{
- private static final BitmapFactory BITMAP_FACTORY = new RoaringBitmapFactory();
-
- public RoaringBitMapFactory()
- {
- }
-
- @Override
- public MutableBitmap makeEmptyMutableBitmap()
- {
- return BITMAP_FACTORY.makeEmptyMutableBitmap();
- }
-
- @Override
- public String toString()
- {
- return "RoaringBitMapFactory";
- }
-
- @Override
- public boolean equals(Object o)
- {
- return this == o || o instanceof RoaringBitMapFactory;
- }
-
- @Override
- public int hashCode()
- {
- return 0;
- }
-}
diff --git a/extensions-contrib/distinctcount/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule b/extensions-contrib/distinctcount/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
deleted file mode 100644
index e83dbd1f2b29..000000000000
--- a/extensions-contrib/distinctcount/src/main/resources/META-INF/services/org.apache.druid.initialization.DruidModule
+++ /dev/null
@@ -1,16 +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.
-
-org.apache.druid.query.aggregation.distinctcount.DistinctCountDruidModule
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
deleted file mode 100644
index 6c7db8eb3638..000000000000
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountGroupByQueryTest.java
+++ /dev/null
@@ -1,159 +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.query.aggregation.distinctcount;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.druid.data.input.MapBasedInputRow;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.Pair;
-import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.java.util.common.io.Closer;
-import org.apache.druid.query.QueryRunnerTestHelper;
-import org.apache.druid.query.aggregation.CountAggregatorFactory;
-import org.apache.druid.query.dimension.DefaultDimensionSpec;
-import org.apache.druid.query.groupby.GroupByQuery;
-import org.apache.druid.query.groupby.GroupByQueryConfig;
-import org.apache.druid.query.groupby.GroupByQueryRunnerFactory;
-import org.apache.druid.query.groupby.GroupByQueryRunnerTest;
-import org.apache.druid.query.groupby.GroupByQueryRunnerTestHelper;
-import org.apache.druid.query.groupby.ResultRow;
-import org.apache.druid.query.groupby.orderby.DefaultLimitSpec;
-import org.apache.druid.query.groupby.orderby.OrderByColumnSpec;
-import org.apache.druid.segment.IncrementalIndexSegment;
-import org.apache.druid.segment.Segment;
-import org.apache.druid.segment.TestHelper;
-import org.apache.druid.segment.incremental.IncrementalIndex;
-import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.testing.InitializedNullHandlingTest;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-
-public class DistinctCountGroupByQueryTest extends InitializedNullHandlingTest
-{
- private GroupByQueryRunnerFactory factory;
- private Closer resourceCloser;
-
- @Before
- public void setup()
- {
- final GroupByQueryConfig config = new GroupByQueryConfig();
- config.setMaxIntermediateRows(10000);
- final Pair factoryCloserPair = GroupByQueryRunnerTest.makeQueryRunnerFactory(
- config
- );
- factory = factoryCloserPair.lhs;
- resourceCloser = factoryCloserPair.rhs;
- }
-
- @After
- public void teardown() throws IOException
- {
- resourceCloser.close();
- }
-
- @Test
- public void testGroupByWithDistinctCountAgg() throws Exception
- {
- IncrementalIndex index = new IncrementalIndex.Builder()
- .setIndexSchema(
- new IncrementalIndexSchema.Builder()
- .withQueryGranularity(Granularities.SECOND)
- .withMetrics(new CountAggregatorFactory("cnt"))
- .build()
- )
- .setConcurrentEventAdd(true)
- .setMaxRowCount(1000)
- .buildOnheap();
-
- String visitor_id = "visitor_id";
- String client_type = "client_type";
- long timestamp = DateTimes.of("2010-01-01").getMillis();
- index.add(
- new MapBasedInputRow(
- timestamp,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "0", client_type, "iphone")
- )
- );
- index.add(
- new MapBasedInputRow(
- timestamp + 1,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "1", client_type, "iphone")
- )
- );
- index.add(
- new MapBasedInputRow(
- timestamp + 2,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "2", client_type, "android")
- )
- );
-
- GroupByQuery query = new GroupByQuery.Builder()
- .setDataSource(QueryRunnerTestHelper.DATA_SOURCE)
- .setGranularity(QueryRunnerTestHelper.ALL_GRAN)
- .setDimensions(new DefaultDimensionSpec(
- client_type,
- client_type
- ))
- .setInterval(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
- .setLimitSpec(
- new DefaultLimitSpec(
- Collections.singletonList(new OrderByColumnSpec(client_type, OrderByColumnSpec.Direction.DESCENDING)),
- 10
- )
- )
- .setAggregatorSpecs(QueryRunnerTestHelper.ROWS_COUNT, new DistinctCountAggregatorFactory("UV", visitor_id, null))
- .build();
- final Segment incrementalIndexSegment = new IncrementalIndexSegment(index, null);
-
- Iterable results = GroupByQueryRunnerTestHelper.runQuery(
- factory,
- factory.createRunner(incrementalIndexSegment),
- query
- );
-
- List expectedResults = Arrays.asList(
- GroupByQueryRunnerTestHelper.createExpectedRow(
- query,
- "1970-01-01T00:00:00.000Z",
- client_type, "iphone",
- "UV", 2L,
- "rows", 2L
- ),
- GroupByQueryRunnerTestHelper.createExpectedRow(
- query,
- "1970-01-01T00:00:00.000Z",
- client_type, "android",
- "UV", 1L,
- "rows", 1L
- )
- );
- TestHelper.assertExpectedObjects(expectedResults, results, "distinct-count");
- }
-}
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
deleted file mode 100644
index 2cc0526480bf..000000000000
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTimeseriesQueryTest.java
+++ /dev/null
@@ -1,114 +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.query.aggregation.distinctcount;
-
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.druid.data.input.MapBasedInputRow;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.query.Druids;
-import org.apache.druid.query.QueryRunnerTestHelper;
-import org.apache.druid.query.Result;
-import org.apache.druid.query.aggregation.CountAggregatorFactory;
-import org.apache.druid.query.timeseries.TimeseriesQuery;
-import org.apache.druid.query.timeseries.TimeseriesQueryEngine;
-import org.apache.druid.query.timeseries.TimeseriesResultValue;
-import org.apache.druid.segment.TestHelper;
-import org.apache.druid.segment.incremental.IncrementalIndex;
-import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
-import org.apache.druid.testing.InitializedNullHandlingTest;
-import org.joda.time.DateTime;
-import org.junit.Test;
-
-import java.util.Collections;
-import java.util.List;
-
-public class DistinctCountTimeseriesQueryTest extends InitializedNullHandlingTest
-{
-
- @Test
- public void testTopNWithDistinctCountAgg() throws Exception
- {
- TimeseriesQueryEngine engine = new TimeseriesQueryEngine();
-
- IncrementalIndex index = new IncrementalIndex.Builder()
- .setIndexSchema(
- new IncrementalIndexSchema.Builder()
- .withQueryGranularity(Granularities.SECOND)
- .withMetrics(new CountAggregatorFactory("cnt"))
- .build()
- )
- .setMaxRowCount(1000)
- .buildOnheap();
-
- String visitor_id = "visitor_id";
- String client_type = "client_type";
- DateTime time = DateTimes.of("2016-03-04T00:00:00.000Z");
- long timestamp = time.getMillis();
- index.add(
- new MapBasedInputRow(
- timestamp,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "0", client_type, "iphone")
- )
- );
- index.add(
- new MapBasedInputRow(
- timestamp,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "1", client_type, "iphone")
- )
- );
- index.add(
- new MapBasedInputRow(
- timestamp,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "2", client_type, "android")
- )
- );
-
- TimeseriesQuery query = Druids.newTimeseriesQueryBuilder()
- .dataSource(QueryRunnerTestHelper.DATA_SOURCE)
- .granularity(QueryRunnerTestHelper.ALL_GRAN)
- .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
- .aggregators(
- Lists.newArrayList(
- QueryRunnerTestHelper.ROWS_COUNT,
- new DistinctCountAggregatorFactory("UV", visitor_id, null)
- )
- )
- .build();
-
- final Iterable> results =
- engine.process(query, new IncrementalIndexStorageAdapter(index)).toList();
-
- List> expectedResults = Collections.singletonList(
- new Result<>(
- time,
- new TimeseriesResultValue(
- ImmutableMap.of("UV", 3, "rows", 3L)
- )
- )
- );
- TestHelper.assertExpectedResults(expectedResults, results);
- }
-}
diff --git a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java b/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
deleted file mode 100644
index 7b14fba07f11..000000000000
--- a/extensions-contrib/distinctcount/src/test/java/org/apache/druid/query/aggregation/distinctcount/DistinctCountTopNQueryTest.java
+++ /dev/null
@@ -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.query.aggregation.distinctcount;
-
-import com.google.common.base.Supplier;
-import com.google.common.collect.ImmutableMap;
-import com.google.common.collect.Lists;
-import org.apache.druid.collections.CloseableStupidPool;
-import org.apache.druid.data.input.MapBasedInputRow;
-import org.apache.druid.java.util.common.DateTimes;
-import org.apache.druid.java.util.common.granularity.Granularities;
-import org.apache.druid.query.QueryRunnerTestHelper;
-import org.apache.druid.query.Result;
-import org.apache.druid.query.aggregation.CountAggregatorFactory;
-import org.apache.druid.query.topn.TopNQuery;
-import org.apache.druid.query.topn.TopNQueryBuilder;
-import org.apache.druid.query.topn.TopNQueryEngine;
-import org.apache.druid.query.topn.TopNResultValue;
-import org.apache.druid.segment.TestHelper;
-import org.apache.druid.segment.incremental.IncrementalIndex;
-import org.apache.druid.segment.incremental.IncrementalIndexSchema;
-import org.apache.druid.segment.incremental.IncrementalIndexStorageAdapter;
-import org.apache.druid.testing.InitializedNullHandlingTest;
-import org.joda.time.DateTime;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-import java.util.Arrays;
-import java.util.Collections;
-import java.util.List;
-import java.util.Map;
-
-public class DistinctCountTopNQueryTest extends InitializedNullHandlingTest
-{
- private CloseableStupidPool pool;
-
- @Before
- public void setup()
- {
- pool = new CloseableStupidPool<>(
- "TopNQueryEngine-bufferPool",
- new Supplier()
- {
- @Override
- public ByteBuffer get()
- {
- return ByteBuffer.allocate(1024 * 1024);
- }
- }
- );
- }
-
- @After
- public void teardown()
- {
- pool.close();
- }
-
- @Test
- public void testTopNWithDistinctCountAgg() throws Exception
- {
- TopNQueryEngine engine = new TopNQueryEngine(pool);
-
- IncrementalIndex index = new IncrementalIndex.Builder()
- .setIndexSchema(
- new IncrementalIndexSchema.Builder()
- .withQueryGranularity(Granularities.SECOND)
- .withMetrics(new CountAggregatorFactory("cnt"))
- .build()
- )
- .setMaxRowCount(1000)
- .buildOnheap();
-
- String visitor_id = "visitor_id";
- String client_type = "client_type";
- DateTime time = DateTimes.of("2016-03-04T00:00:00.000Z");
- long timestamp = time.getMillis();
- index.add(
- new MapBasedInputRow(
- timestamp,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "0", client_type, "iphone")
- )
- );
- index.add(
- new MapBasedInputRow(
- timestamp,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "1", client_type, "iphone")
- )
- );
- index.add(
- new MapBasedInputRow(
- timestamp,
- Lists.newArrayList(visitor_id, client_type),
- ImmutableMap.of(visitor_id, "2", client_type, "android")
- )
- );
-
- TopNQuery query = new TopNQueryBuilder().dataSource(QueryRunnerTestHelper.DATA_SOURCE)
- .granularity(QueryRunnerTestHelper.ALL_GRAN)
- .intervals(QueryRunnerTestHelper.FULL_ON_INTERVAL_SPEC)
- .dimension(client_type)
- .metric("UV")
- .threshold(10)
- .aggregators(
- QueryRunnerTestHelper.ROWS_COUNT,
- new DistinctCountAggregatorFactory("UV", visitor_id, null)
- )
- .build();
-
- final Iterable> results =
- engine.query(query, new IncrementalIndexStorageAdapter(index), null).toList();
-
- List> expectedResults = Collections.singletonList(
- new Result<>(
- time,
- new TopNResultValue(
- Arrays.