mirror of https://github.com/apache/druid.git
support Aliyun OSS service as deep storage (#9898)
* init commit, all tests passed * fix format Signed-off-by: frank chen <frank.chen021@outlook.com> * data stored successfully * modify config path * add doc * add aliyun-oss extension to project * remove descriptor deletion code to avoid warning message output by aliyun client * fix warnings reported by lgtm-com * fix ci warnings Signed-off-by: frank chen <frank.chen021@outlook.com> * fix errors reported by intellj inspection check Signed-off-by: frank chen <frank.chen021@outlook.com> * fix doc spelling check Signed-off-by: frank chen <frank.chen021@outlook.com> * fix dependency warnings reported by ci Signed-off-by: frank chen <frank.chen021@outlook.com> * fix warnings reported by CI Signed-off-by: frank chen <frank.chen021@outlook.com> * add package configuration to support showing extension info Signed-off-by: frank chen <frank.chen021@outlook.com> * add IT test cases and fix bugs Signed-off-by: frank chen <frank.chen021@outlook.com> * 1. code review comments adopted 2. change schema from 'aliyun-oss' to 'oss' Signed-off-by: frank chen <frank.chen021@outlook.com> * add license info Signed-off-by: frank chen <frank.chen021@outlook.com> * fix doc Signed-off-by: frank chen <frank.chen021@outlook.com> * exclude execution of IT testcases of OSS extension from CI Signed-off-by: frank chen <frank.chen021@outlook.com> * put the extensions under contrib group and add to distribution * fix names in test cases * add unit test to cover OssInputSource * fix names in test cases * fix dependency problem reported by CI Signed-off-by: frank chen <frank.chen021@outlook.com>
This commit is contained in:
parent
e2c5bcc22d
commit
60c6bd5b4c
|
@ -389,7 +389,7 @@ jobs:
|
|||
name: "(Compile=openjdk8, Run=openjdk8) other integration test"
|
||||
jdk: openjdk8
|
||||
services: *integration_test_services
|
||||
env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage' JVM_RUNTIME='-Djvm.runtime=8'
|
||||
env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage' JVM_RUNTIME='-Djvm.runtime=8'
|
||||
script: *run_integration_test
|
||||
after_failure: *integration_test_diags
|
||||
# END - Integration tests for Compile with Java 8 and Run with Java 8
|
||||
|
@ -423,7 +423,7 @@ jobs:
|
|||
- <<: *integration_tests
|
||||
name: "(Compile=openjdk8, Run=openjdk11) other integration test"
|
||||
jdk: openjdk8
|
||||
env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage' JVM_RUNTIME='-Djvm.runtime=11'
|
||||
env: TESTNG_GROUPS='-DexcludedGroups=batch-index,perfect-rollup-parallel-batch-index,kafka-index,query,realtime-index,security,s3-deep-storage,gcs-deep-storage,azure-deep-storage,hdfs-deep-storage,s3-ingestion,kinesis-index,kinesis-data-format,kafka-transactional-index,kafka-index-slow,kafka-transactional-index-slow,kafka-data-format,hadoop-s3-to-s3-deep-storage,hadoop-s3-to-hdfs-deep-storage,hadoop-azure-to-azure-deep-storage,hadoop-azure-to-hdfs-deep-storage,hadoop-gcs-to-gcs-deep-storage,hadoop-gcs-to-hdfs-deep-storage,aliyun-oss-deep-storage' JVM_RUNTIME='-Djvm.runtime=11'
|
||||
# END - Integration tests for Compile with Java 8 and Run with Java 11
|
||||
|
||||
- name: "security vulnerabilities"
|
||||
|
|
1
LICENSE
1
LICENSE
|
@ -274,6 +274,7 @@ SOURCE/JAVA-CORE
|
|||
|
||||
This product contains s3 directory place holder check code adapted from JetS3t (https://bitbucket.org/jmurty/jets3t/wiki/Home).
|
||||
* extensions-core/s3-extensions/src/main/java/org/apache/druid/storage/s3/S3Utils.java
|
||||
* extensions-contrib/aliyun-oss-extensions/src/main/java/org/apache/druid/storage/aliyun/OssObjectSummaryIterator.java
|
||||
|
||||
This product contains lpad and rpad methods adapted from Apache Flink.
|
||||
* core/src/main/java/org/apache/druid/java/util/common/StringUtils.java
|
||||
|
|
|
@ -425,6 +425,8 @@
|
|||
<argument>org.apache.druid.extensions.contrib:druid-tdigestsketch</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:gce-extensions</argument>
|
||||
<argument>-c</argument>
|
||||
<argument>org.apache.druid.extensions.contrib:aliyun-oss-extensions</argument>
|
||||
</arguments>
|
||||
</configuration>
|
||||
</execution>
|
||||
|
|
|
@ -0,0 +1,54 @@
|
|||
---
|
||||
id: aliyun-oss
|
||||
title: "Aliyun OSS"
|
||||
---
|
||||
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
|
||||
To use this Apache Druid extension, make sure to [include](../../development/extensions.md#loading-extensions) `aliyun-oss-extensions` extension.
|
||||
|
||||
## Deep Storage
|
||||
|
||||
[Aliyun](https://www.aliyun.com) is the 3rd largest cloud infrastructure provider in the world. It provides its own storage solution known as OSS, [Object Storage Service](https://www.aliyun.com/product/oss).
|
||||
|
||||
To use aliyun OSS as deep storage, first config as below
|
||||
|
||||
|Property|Description|Possible Values|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.oss.accessKey`|the `AccessKey ID` of your account which can be used to access the bucket| |Must be set.|
|
||||
|`druid.oss.secretKey`|the `AccessKey Secret` of your account which can be used to access the bucket| |Must be set. |
|
||||
|`druid.oss.endpoint`|the endpoint url of your OSS storage| |Must be set.|
|
||||
|
||||
if you want to use OSS as deep storage, use the configurations below
|
||||
|
||||
|Property|Description|Possible Values|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.storage.type`| Global deep storage provider. Must be set to `oss` to make use of this extension. | oss |Must be set.|
|
||||
|`druid.storage.oss.bucket`|storage bucket name.| | Must be set.|
|
||||
|`druid.storage.oss.prefix`|a prefix string prepended to the file names for the segments published to aliyun OSS deep storage| druid/segments | |
|
||||
|
||||
To save index logs to OSS, apply the configurations below:
|
||||
|
||||
|Property|Description|Possible Values|Default|
|
||||
|--------|---------------|-----------|-------|
|
||||
|`druid.indexer.logs.type`| Global deep storage provider. Must be set to `oss` to make use of this extension. | oss |Must be set.|
|
||||
|`druid.indexer.logs.oss.bucket`|the bucket used to keep logs| |Must be set.|
|
||||
|`druid.indexer.logs.oss.prefix`|a prefix string prepended to the log files.| | |
|
|
@ -73,6 +73,7 @@ All of these community extensions can be downloaded using [pull-deps](../operati
|
|||
|
||||
|Name|Description|Docs|
|
||||
|----|-----------|----|
|
||||
|aliyun-oss-extensions|Aliyun OSS deep storage |[link](../development/extensions-contrib/aliyun-oss-extensions.md)|
|
||||
|ambari-metrics-emitter|Ambari Metrics Emitter |[link](../development/extensions-contrib/ambari-metrics-emitter.md)|
|
||||
|druid-cassandra-storage|Apache Cassandra deep storage.|[link](../development/extensions-contrib/cassandra.md)|
|
||||
|druid-cloudfiles-extensions|Rackspace Cloudfiles deep storage and firehose.|[link](../development/extensions-contrib/cloudfiles.md)|
|
||||
|
|
|
@ -0,0 +1,181 @@
|
|||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0" xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/maven-v4_0_0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<groupId>org.apache.druid.extensions.contrib</groupId>
|
||||
<artifactId>aliyun-oss-extensions</artifactId>
|
||||
<name>aliyun-oss-extensions</name>
|
||||
<description>aliyun-oss-extensions</description>
|
||||
|
||||
<parent>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid</artifactId>
|
||||
<version>0.19.0-SNAPSHOT</version>
|
||||
<relativePath>../../pom.xml</relativePath>
|
||||
</parent>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency> <!-- aliyun oss -->
|
||||
<groupId>com.aliyun.oss</groupId>
|
||||
<artifactId>aliyun-sdk-oss</artifactId>
|
||||
<version>3.3.0</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.module</groupId>
|
||||
<artifactId>jackson-module-guice</artifactId>
|
||||
<version>${jackson.version}</version>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-io</groupId>
|
||||
<artifactId>commons-io</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-annotations</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>joda-time</groupId>
|
||||
<artifactId>joda-time</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject</groupId>
|
||||
<artifactId>guice</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-databind</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.fasterxml.jackson.core</groupId>
|
||||
<artifactId>jackson-core</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.inject.extensions</groupId>
|
||||
<artifactId>guice-multibindings</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.guava</groupId>
|
||||
<artifactId>guava</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>javax.validation</groupId>
|
||||
<artifactId>validation-api</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.google.code.findbugs</groupId>
|
||||
<artifactId>jsr305</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>commons-lang</groupId>
|
||||
<artifactId>commons-lang</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<!-- Tests -->
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-core</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<type>test-jar</type>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-server</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.druid</groupId>
|
||||
<artifactId>druid-processing</artifactId>
|
||||
<version>${project.parent.version}</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.easymock</groupId>
|
||||
<artifactId>easymock</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>nl.jqno.equalsverifier</groupId>
|
||||
<artifactId>equalsverifier</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.hamcrest</groupId>
|
||||
<artifactId>hamcrest-core</artifactId>
|
||||
<version>1.3</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin><!--package all dependencies together-->
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-assembly-plugin</artifactId>
|
||||
<configuration>
|
||||
<appendAssemblyId>false</appendAssemblyId>
|
||||
<descriptorRefs>
|
||||
<descriptorRef>jar-with-dependencies</descriptorRef>
|
||||
</descriptorRefs>
|
||||
<archive><!--add entries like Implementation-Title to MANIFEST.MF-->
|
||||
<manifest>
|
||||
<addDefaultImplementationEntries>true</addDefaultImplementationEntries>
|
||||
</manifest>
|
||||
</archive>
|
||||
</configuration>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>make-assembly</id>
|
||||
<phase>package</phase>
|
||||
<goals>
|
||||
<goal>single</goal>
|
||||
</goals>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
</project>
|
|
@ -0,0 +1,122 @@
|
|||
/*
|
||||
* 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.OSSClientBuilder;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonIgnore;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.druid.metadata.PasswordProvider;
|
||||
|
||||
import java.util.Objects;
|
||||
|
||||
/**
|
||||
* Contains properties for aliyun OSS input source.
|
||||
* Properties can be specified by ingestionSpec which will override system default.
|
||||
*/
|
||||
public class OssClientConfig
|
||||
{
|
||||
@JsonCreator
|
||||
public OssClientConfig(
|
||||
@JsonProperty("endpoint") String endpoint,
|
||||
@JsonProperty("accessKey") PasswordProvider accessKey,
|
||||
@JsonProperty("secretKey") PasswordProvider secretKey
|
||||
)
|
||||
{
|
||||
this.accessKey = Preconditions.checkNotNull(
|
||||
accessKey,
|
||||
"accessKey cannot be null"
|
||||
);
|
||||
this.secretKey = Preconditions.checkNotNull(
|
||||
secretKey,
|
||||
"secretKey cannot be null"
|
||||
);
|
||||
this.endpoint = endpoint;
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
private String endpoint;
|
||||
|
||||
@JsonProperty
|
||||
private PasswordProvider accessKey;
|
||||
|
||||
@JsonProperty
|
||||
private PasswordProvider secretKey;
|
||||
|
||||
public String getEndpoint()
|
||||
{
|
||||
return endpoint;
|
||||
}
|
||||
|
||||
public PasswordProvider getAccessKey()
|
||||
{
|
||||
return accessKey;
|
||||
}
|
||||
|
||||
public PasswordProvider getSecretKey()
|
||||
{
|
||||
return secretKey;
|
||||
}
|
||||
|
||||
@JsonIgnore
|
||||
public boolean isCredentialsConfigured()
|
||||
{
|
||||
return accessKey != null &&
|
||||
secretKey != null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "OssInputSourceConfig{" +
|
||||
"endpoint=" + endpoint +
|
||||
"accessKeyId=" + accessKey +
|
||||
", secretAccessKey=" + secretKey +
|
||||
'}';
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
OssClientConfig that = (OssClientConfig) o;
|
||||
return Objects.equals(accessKey, that.accessKey) &&
|
||||
Objects.equals(secretKey, that.secretKey) &&
|
||||
Objects.equals(endpoint, that.endpoint);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(accessKey, secretKey, endpoint);
|
||||
}
|
||||
|
||||
public OSS buildClient()
|
||||
{
|
||||
return new OSSClientBuilder().build(endpoint, accessKey.getPassword(), secretKey.getPassword());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,88 @@
|
|||
/*
|
||||
* 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.OSSException;
|
||||
import com.aliyun.oss.model.GetObjectRequest;
|
||||
import com.aliyun.oss.model.OSSObject;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.data.input.RetryingInputEntity;
|
||||
import org.apache.druid.data.input.impl.CloudObjectLocation;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.storage.aliyun.OssStorageDruidModule;
|
||||
import org.apache.druid.storage.aliyun.OssUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
|
||||
public class OssEntity extends RetryingInputEntity
|
||||
{
|
||||
private final OSS ossClient;
|
||||
private final CloudObjectLocation object;
|
||||
|
||||
OssEntity(OSS ossClient, CloudObjectLocation coords)
|
||||
{
|
||||
this.ossClient = ossClient;
|
||||
this.object = coords;
|
||||
}
|
||||
|
||||
@Override
|
||||
public URI getUri()
|
||||
{
|
||||
return object.toUri(OssStorageDruidModule.SCHEME);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputStream readFrom(long offset) throws IOException
|
||||
{
|
||||
final GetObjectRequest request = new GetObjectRequest(object.getBucket(), object.getPath());
|
||||
request.setRange(offset, -1 /*from offset to end*/);
|
||||
|
||||
try {
|
||||
final OSSObject ossObject = ossClient.getObject(request);
|
||||
if (ossObject == null) {
|
||||
throw new ISE(
|
||||
"Failed to get an Aliyun OSS object for bucket[%s], key[%s], and start[%d]",
|
||||
object.getBucket(),
|
||||
object.getPath(),
|
||||
offset
|
||||
);
|
||||
}
|
||||
return ossObject.getObjectContent();
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected String getPath()
|
||||
{
|
||||
return object.getPath();
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Throwable> getRetryCondition()
|
||||
{
|
||||
return OssUtils.RETRYABLE;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,178 @@
|
|||
/*
|
||||
* 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.model.OSSObjectSummary;
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Supplier;
|
||||
import com.google.common.base.Suppliers;
|
||||
import org.apache.druid.data.input.InputEntity;
|
||||
import org.apache.druid.data.input.InputFileAttribute;
|
||||
import org.apache.druid.data.input.InputSplit;
|
||||
import org.apache.druid.data.input.SplitHintSpec;
|
||||
import org.apache.druid.data.input.impl.CloudObjectInputSource;
|
||||
import org.apache.druid.data.input.impl.CloudObjectLocation;
|
||||
import org.apache.druid.data.input.impl.SplittableInputSource;
|
||||
import org.apache.druid.storage.aliyun.OssInputDataConfig;
|
||||
import org.apache.druid.storage.aliyun.OssStorageDruidModule;
|
||||
import org.apache.druid.storage.aliyun.OssUtils;
|
||||
import org.apache.druid.utils.Streams;
|
||||
|
||||
import javax.annotation.Nonnull;
|
||||
import javax.annotation.Nullable;
|
||||
import java.net.URI;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
import java.util.stream.Stream;
|
||||
|
||||
public class OssInputSource extends CloudObjectInputSource
|
||||
{
|
||||
private final Supplier<OSS> clientSupplier;
|
||||
@JsonProperty("properties")
|
||||
private final OssClientConfig inputSourceConfig;
|
||||
private final OssInputDataConfig inputDataConfig;
|
||||
|
||||
/**
|
||||
* Constructor for OssInputSource
|
||||
*
|
||||
* @param client The default client built with all default configs
|
||||
* from Guice. This injected singleton client is used when {@param inputSourceConfig}
|
||||
* is not provided and hence
|
||||
* @param inputDataConfig Stores the configuration for options related to reading input data
|
||||
* @param uris User provided uris to read input data
|
||||
* @param prefixes User provided prefixes to read input data
|
||||
* @param objects User provided cloud objects values to read input data
|
||||
* @param inputSourceConfig User provided properties for overriding the default aliyun OSS configuration
|
||||
*/
|
||||
@JsonCreator
|
||||
public OssInputSource(
|
||||
@JacksonInject OSS client,
|
||||
@JacksonInject OssInputDataConfig inputDataConfig,
|
||||
@JsonProperty("uris") @Nullable List<URI> uris,
|
||||
@JsonProperty("prefixes") @Nullable List<URI> prefixes,
|
||||
@JsonProperty("objects") @Nullable List<CloudObjectLocation> objects,
|
||||
@JsonProperty("properties") @Nullable OssClientConfig inputSourceConfig
|
||||
)
|
||||
{
|
||||
super(OssStorageDruidModule.SCHEME, uris, prefixes, objects);
|
||||
this.inputDataConfig = Preconditions.checkNotNull(inputDataConfig, "inputDataConfig");
|
||||
Preconditions.checkNotNull(client, "client");
|
||||
this.inputSourceConfig = inputSourceConfig;
|
||||
this.clientSupplier = Suppliers.memoize(
|
||||
() -> {
|
||||
if (inputSourceConfig != null) {
|
||||
return inputSourceConfig.buildClient();
|
||||
} else {
|
||||
return client;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
|
||||
|
||||
@Nullable
|
||||
@JsonProperty("properties")
|
||||
public OssClientConfig getOssInputSourceConfig()
|
||||
{
|
||||
return inputSourceConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputEntity createEntity(CloudObjectLocation location)
|
||||
{
|
||||
return new OssEntity(clientSupplier.get(), location);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Stream<InputSplit<List<CloudObjectLocation>>> getPrefixesSplitStream(@Nonnull SplitHintSpec splitHintSpec)
|
||||
{
|
||||
final Iterator<List<OSSObjectSummary>> splitIterator = splitHintSpec.split(
|
||||
getIterableObjectsFromPrefixes().iterator(),
|
||||
object -> new InputFileAttribute(object.getSize())
|
||||
);
|
||||
|
||||
return Streams.sequentialStreamFrom(splitIterator)
|
||||
.map(objects -> objects.stream()
|
||||
.map(OssUtils::summaryToCloudObjectLocation)
|
||||
.collect(Collectors.toList()))
|
||||
.map(InputSplit::new);
|
||||
}
|
||||
|
||||
@Override
|
||||
public SplittableInputSource<List<CloudObjectLocation>> withSplit(InputSplit<List<CloudObjectLocation>> split)
|
||||
{
|
||||
return new OssInputSource(
|
||||
clientSupplier.get(),
|
||||
inputDataConfig,
|
||||
null,
|
||||
null,
|
||||
split.get(),
|
||||
getOssInputSourceConfig()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(super.hashCode(), inputSourceConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
if (!super.equals(o)) {
|
||||
return false;
|
||||
}
|
||||
OssInputSource that = (OssInputSource) o;
|
||||
return Objects.equals(inputSourceConfig, that.inputSourceConfig);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString()
|
||||
{
|
||||
return "OssInputSource{" +
|
||||
"uris=" + getUris() +
|
||||
", prefixes=" + getPrefixes() +
|
||||
", objects=" + getObjects() +
|
||||
", ossInputSourceConfig=" + getOssInputSourceConfig() +
|
||||
'}';
|
||||
}
|
||||
|
||||
private Iterable<OSSObjectSummary> getIterableObjectsFromPrefixes()
|
||||
{
|
||||
return () -> OssUtils.objectSummaryIterator(
|
||||
clientSupplier.get(),
|
||||
getPrefixes(),
|
||||
inputDataConfig.getMaxListingLength()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.data.input.aliyun;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
import org.apache.druid.storage.aliyun.OssStorageDruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* Druid module to wire up native batch support for aliyun OSS input
|
||||
*/
|
||||
public class OssInputSourceDruidModule implements DruidModule
|
||||
{
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new SimpleModule().registerSubtypes(new NamedType(OssInputSource.class, OssStorageDruidModule.SCHEME))
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
}
|
||||
}
|
|
@ -0,0 +1,46 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.firehose.aliyun;
|
||||
|
||||
import com.fasterxml.jackson.databind.Module;
|
||||
import com.fasterxml.jackson.databind.jsontype.NamedType;
|
||||
import com.fasterxml.jackson.databind.module.SimpleModule;
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.inject.Binder;
|
||||
import org.apache.druid.initialization.DruidModule;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
public class OssFirehoseDruidModule implements DruidModule
|
||||
{
|
||||
@Override
|
||||
public List<? extends Module> getJacksonModules()
|
||||
{
|
||||
return ImmutableList.of(
|
||||
new SimpleModule().registerSubtypes(new NamedType(StaticOssFirehoseFactory.class, "static-aliyun-oss"))
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
|
||||
}
|
||||
}
|
|
@ -0,0 +1,243 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.firehose.aliyun;
|
||||
|
||||
import com.aliyun.oss.OSS;
|
||||
import com.aliyun.oss.OSSException;
|
||||
import com.aliyun.oss.model.GetObjectRequest;
|
||||
import com.aliyun.oss.model.OSSObject;
|
||||
import com.aliyun.oss.model.OSSObjectSummary;
|
||||
import com.fasterxml.jackson.annotation.JacksonInject;
|
||||
import com.fasterxml.jackson.annotation.JsonCreator;
|
||||
import com.fasterxml.jackson.annotation.JsonProperty;
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.google.common.base.Predicate;
|
||||
import org.apache.druid.data.input.FiniteFirehoseFactory;
|
||||
import org.apache.druid.data.input.InputSplit;
|
||||
import org.apache.druid.data.input.impl.StringInputRowParser;
|
||||
import org.apache.druid.data.input.impl.prefetch.PrefetchableTextFilesFirehoseFactory;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.storage.aliyun.OssStorageDruidModule;
|
||||
import org.apache.druid.storage.aliyun.OssUtils;
|
||||
import org.apache.druid.utils.CompressionUtils;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.Collections;
|
||||
import java.util.Iterator;
|
||||
import java.util.List;
|
||||
import java.util.Objects;
|
||||
import java.util.stream.Collectors;
|
||||
|
||||
/**
|
||||
* Builds firehoses that read from a predefined list of aliyun OSS objects and then dry up.
|
||||
*/
|
||||
public class StaticOssFirehoseFactory extends PrefetchableTextFilesFirehoseFactory<URI>
|
||||
{
|
||||
private static final Logger log = new Logger(StaticOssFirehoseFactory.class);
|
||||
|
||||
private final OSS client;
|
||||
private final List<URI> uris;
|
||||
private final List<URI> prefixes;
|
||||
|
||||
@JsonCreator
|
||||
public StaticOssFirehoseFactory(
|
||||
@JacksonInject OSS client,
|
||||
@JsonProperty("uris") List<URI> uris,
|
||||
@JsonProperty("prefixes") List<URI> prefixes,
|
||||
@JsonProperty("maxCacheCapacityBytes") Long maxCacheCapacityBytes,
|
||||
@JsonProperty("maxFetchCapacityBytes") Long maxFetchCapacityBytes,
|
||||
@JsonProperty("prefetchTriggerBytes") Long prefetchTriggerBytes,
|
||||
@JsonProperty("fetchTimeout") Long fetchTimeout,
|
||||
@JsonProperty("maxFetchRetry") Integer maxFetchRetry
|
||||
)
|
||||
{
|
||||
super(maxCacheCapacityBytes, maxFetchCapacityBytes, prefetchTriggerBytes, fetchTimeout, maxFetchRetry);
|
||||
this.client = Preconditions.checkNotNull(client, "client");
|
||||
this.uris = uris == null ? new ArrayList<>() : uris;
|
||||
this.prefixes = prefixes == null ? new ArrayList<>() : prefixes;
|
||||
|
||||
if (!this.uris.isEmpty() && !this.prefixes.isEmpty()) {
|
||||
throw new IAE("uris and prefixes cannot be used together");
|
||||
}
|
||||
|
||||
if (this.uris.isEmpty() && this.prefixes.isEmpty()) {
|
||||
throw new IAE("uris or prefixes must be specified");
|
||||
}
|
||||
|
||||
for (final URI inputURI : this.uris) {
|
||||
Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()),
|
||||
"input uri scheme == %s (%s)",
|
||||
OssStorageDruidModule.SCHEME,
|
||||
inputURI);
|
||||
}
|
||||
|
||||
for (final URI inputURI : this.prefixes) {
|
||||
Preconditions.checkArgument(OssStorageDruidModule.SCHEME.equals(inputURI.getScheme()),
|
||||
"input uri scheme == %s (%s)",
|
||||
OssStorageDruidModule.SCHEME,
|
||||
inputURI);
|
||||
}
|
||||
}
|
||||
|
||||
@JsonProperty
|
||||
public List<URI> getUris()
|
||||
{
|
||||
return uris;
|
||||
}
|
||||
|
||||
@JsonProperty("prefixes")
|
||||
public List<URI> getPrefixes()
|
||||
{
|
||||
return prefixes;
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Collection<URI> initObjects()
|
||||
{
|
||||
if (!uris.isEmpty()) {
|
||||
return uris;
|
||||
} else {
|
||||
final List<OSSObjectSummary> objects = new ArrayList<>();
|
||||
for (final URI prefix : prefixes) {
|
||||
final Iterator<OSSObjectSummary> objectSummaryIterator = OssUtils.objectSummaryIterator(
|
||||
client,
|
||||
Collections.singletonList(prefix),
|
||||
OssUtils.MAX_LISTING_LENGTH
|
||||
);
|
||||
|
||||
objectSummaryIterator.forEachRemaining(objects::add);
|
||||
}
|
||||
return objects.stream().map(OssUtils::summaryToUri).collect(Collectors.toList());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputStream openObjectStream(URI object) throws IOException
|
||||
{
|
||||
try {
|
||||
// Get data of the given object and open an input stream
|
||||
final String bucket = object.getAuthority();
|
||||
final String key = OssUtils.extractKey(object);
|
||||
|
||||
final OSSObject ossObject = client.getObject(bucket, key);
|
||||
if (ossObject == null) {
|
||||
throw new ISE("Failed to get an Aliyun OSS object for bucket[%s] and key[%s]", bucket, key);
|
||||
}
|
||||
return ossObject.getObjectContent();
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputStream openObjectStream(URI object, long start) throws IOException
|
||||
{
|
||||
final String bucket = object.getAuthority();
|
||||
final String key = OssUtils.extractKey(object);
|
||||
|
||||
final GetObjectRequest request = new GetObjectRequest(bucket, key);
|
||||
try {
|
||||
final OSSObject ossObject = client.getObject(request);
|
||||
if (ossObject == null) {
|
||||
throw new ISE(
|
||||
"Failed to get an Aliyun OSS object for bucket[%s], key[%s], and start[%d]",
|
||||
bucket,
|
||||
key,
|
||||
start
|
||||
);
|
||||
}
|
||||
InputStream is = ossObject.getObjectContent();
|
||||
is.skip(start);
|
||||
return is;
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
protected InputStream wrapObjectStream(URI object, InputStream stream) throws IOException
|
||||
{
|
||||
return CompressionUtils.decompress(stream, OssUtils.extractKey(object));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean equals(Object o)
|
||||
{
|
||||
if (this == o) {
|
||||
return true;
|
||||
}
|
||||
if (o == null || getClass() != o.getClass()) {
|
||||
return false;
|
||||
}
|
||||
|
||||
StaticOssFirehoseFactory that = (StaticOssFirehoseFactory) o;
|
||||
|
||||
return Objects.equals(uris, that.uris) &&
|
||||
Objects.equals(prefixes, that.prefixes) &&
|
||||
getMaxCacheCapacityBytes() == that.getMaxCacheCapacityBytes() &&
|
||||
getMaxFetchCapacityBytes() == that.getMaxFetchCapacityBytes() &&
|
||||
getPrefetchTriggerBytes() == that.getPrefetchTriggerBytes() &&
|
||||
getFetchTimeout() == that.getFetchTimeout() &&
|
||||
getMaxFetchRetry() == that.getMaxFetchRetry();
|
||||
}
|
||||
|
||||
@Override
|
||||
public int hashCode()
|
||||
{
|
||||
return Objects.hash(
|
||||
uris,
|
||||
prefixes,
|
||||
getMaxCacheCapacityBytes(),
|
||||
getMaxFetchCapacityBytes(),
|
||||
getPrefetchTriggerBytes(),
|
||||
getFetchTimeout(),
|
||||
getMaxFetchRetry()
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Predicate<Throwable> getRetryCondition()
|
||||
{
|
||||
return OssUtils.RETRYABLE;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FiniteFirehoseFactory<StringInputRowParser, URI> withSplit(InputSplit<URI> split)
|
||||
{
|
||||
return new StaticOssFirehoseFactory(
|
||||
client,
|
||||
Collections.singletonList(split.get()),
|
||||
null,
|
||||
getMaxCacheCapacityBytes(),
|
||||
getMaxFetchCapacityBytes(),
|
||||
getPrefetchTriggerBytes(),
|
||||
getFetchTimeout(),
|
||||
getMaxFetchRetry()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,101 @@
|
|||
/*
|
||||
* 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.databind.ObjectMapper;
|
||||
import com.google.common.base.Objects;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.guice.annotations.Json;
|
||||
import org.apache.druid.segment.loading.DataSegmentArchiver;
|
||||
import org.apache.druid.segment.loading.LoadSpec;
|
||||
import org.apache.druid.segment.loading.SegmentLoadingException;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
||||
public class OssDataSegmentArchiver extends OssDataSegmentMover implements DataSegmentArchiver
|
||||
{
|
||||
private final OssDataSegmentArchiverConfig archiveConfig;
|
||||
private final OssStorageConfig restoreConfig;
|
||||
private final ObjectMapper mapper;
|
||||
|
||||
@Inject
|
||||
public OssDataSegmentArchiver(
|
||||
@Json ObjectMapper mapper,
|
||||
OSS client,
|
||||
OssDataSegmentArchiverConfig archiveConfig,
|
||||
OssStorageConfig restoreConfig
|
||||
)
|
||||
{
|
||||
super(client, restoreConfig);
|
||||
this.mapper = mapper;
|
||||
this.archiveConfig = archiveConfig;
|
||||
this.restoreConfig = restoreConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment archive(DataSegment segment) throws SegmentLoadingException
|
||||
{
|
||||
String targetBucket = archiveConfig.getArchiveBucket();
|
||||
String targetKey = archiveConfig.getArchiveBaseKey();
|
||||
|
||||
final DataSegment archived = move(
|
||||
segment,
|
||||
ImmutableMap.of(
|
||||
"bucket", targetBucket,
|
||||
"baseKey", targetKey
|
||||
)
|
||||
);
|
||||
if (sameLoadSpec(segment, archived)) {
|
||||
return null;
|
||||
}
|
||||
return archived;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment restore(DataSegment segment) throws SegmentLoadingException
|
||||
{
|
||||
String targetBucket = restoreConfig.getBucket();
|
||||
String targetKey = restoreConfig.getPrefix();
|
||||
|
||||
final DataSegment restored = move(
|
||||
segment,
|
||||
ImmutableMap.of(
|
||||
"bucket", targetBucket,
|
||||
"baseKey", targetKey
|
||||
)
|
||||
);
|
||||
|
||||
if (sameLoadSpec(segment, restored)) {
|
||||
return null;
|
||||
}
|
||||
return restored;
|
||||
}
|
||||
|
||||
boolean sameLoadSpec(DataSegment s1, DataSegment s2)
|
||||
{
|
||||
final OssLoadSpec s1LoadSpec = (OssLoadSpec) mapper.convertValue(s1.getLoadSpec(), LoadSpec.class);
|
||||
final OssLoadSpec s2LoadSpec = (OssLoadSpec) mapper.convertValue(s2.getLoadSpec(), LoadSpec.class);
|
||||
return Objects.equal(s1LoadSpec.getBucket(), s2LoadSpec.getBucket()) && Objects.equal(
|
||||
s1LoadSpec.getKey(),
|
||||
s2LoadSpec.getKey()
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,41 @@
|
|||
/*
|
||||
* 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 OssDataSegmentArchiverConfig
|
||||
{
|
||||
@JsonProperty
|
||||
private String archiveBucket = "";
|
||||
|
||||
@JsonProperty
|
||||
private String archiveBaseKey = "";
|
||||
|
||||
public String getArchiveBucket()
|
||||
{
|
||||
return archiveBucket;
|
||||
}
|
||||
|
||||
public String getArchiveBaseKey()
|
||||
{
|
||||
return archiveBaseKey;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,98 @@
|
|||
/*
|
||||
* 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.google.common.base.Predicates;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.MapUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.segment.loading.DataSegmentKiller;
|
||||
import org.apache.druid.segment.loading.SegmentLoadingException;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
public class OssDataSegmentKiller implements DataSegmentKiller
|
||||
{
|
||||
private static final Logger log = new Logger(OssDataSegmentKiller.class);
|
||||
|
||||
private final OSS client;
|
||||
private final OssStorageConfig segmentPusherConfig;
|
||||
private final OssInputDataConfig inputDataConfig;
|
||||
|
||||
@Inject
|
||||
public OssDataSegmentKiller(
|
||||
OSS client,
|
||||
OssStorageConfig segmentPusherConfig,
|
||||
OssInputDataConfig inputDataConfig
|
||||
)
|
||||
{
|
||||
this.client = client;
|
||||
this.segmentPusherConfig = segmentPusherConfig;
|
||||
this.inputDataConfig = inputDataConfig;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void kill(DataSegment segment) throws SegmentLoadingException
|
||||
{
|
||||
try {
|
||||
Map<String, Object> loadSpec = segment.getLoadSpec();
|
||||
String bucket = MapUtils.getString(loadSpec, "bucket");
|
||||
String path = MapUtils.getString(loadSpec, "key");
|
||||
|
||||
if (client.doesObjectExist(bucket, path)) {
|
||||
log.info("Removing index file[%s://%s/%s] from aliyun OSS!", OssStorageDruidModule.SCHEME, bucket, path);
|
||||
client.deleteObject(bucket, path);
|
||||
}
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new SegmentLoadingException(e, "Couldn't kill segment[%s]: [%s]", segment.getId(), e);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void killAll() throws IOException
|
||||
{
|
||||
if (segmentPusherConfig.getBucket() == null || segmentPusherConfig.getPrefix() == null) {
|
||||
throw new ISE(
|
||||
"Cannot delete all segment from aliyun OSS Deep Storage since druid.storage.bucket and druid.storage.baseKey are not both set.");
|
||||
}
|
||||
log.info("Deleting all segment files from aliyun OSS location [bucket: '%s' prefix: '%s']",
|
||||
segmentPusherConfig.getBucket(), segmentPusherConfig.getPrefix()
|
||||
);
|
||||
try {
|
||||
OssUtils.deleteObjectsInPath(
|
||||
client,
|
||||
inputDataConfig,
|
||||
segmentPusherConfig.getBucket(),
|
||||
segmentPusherConfig.getPrefix(),
|
||||
Predicates.alwaysTrue()
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error("Error occurred while deleting segment files from aliyun OSS. Error: %s", e.getMessage());
|
||||
throw new IOException(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,253 @@
|
|||
/*
|
||||
* 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.CopyObjectRequest;
|
||||
import com.aliyun.oss.model.ListObjectsRequest;
|
||||
import com.aliyun.oss.model.OSSObjectSummary;
|
||||
import com.aliyun.oss.model.ObjectListing;
|
||||
import com.aliyun.oss.model.StorageClass;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Throwables;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.common.collect.Maps;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.IOE;
|
||||
import org.apache.druid.java.util.common.ISE;
|
||||
import org.apache.druid.java.util.common.MapUtils;
|
||||
import org.apache.druid.java.util.common.RetryUtils;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.segment.loading.DataSegmentMover;
|
||||
import org.apache.druid.segment.loading.DataSegmentPusher;
|
||||
import org.apache.druid.segment.loading.SegmentLoadingException;
|
||||
import org.apache.druid.timeline.DataSegment;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Map;
|
||||
|
||||
public class OssDataSegmentMover implements DataSegmentMover
|
||||
{
|
||||
private static final Logger log = new Logger(OssDataSegmentMover.class);
|
||||
|
||||
private final OSS client;
|
||||
private final OssStorageConfig config;
|
||||
|
||||
@Inject
|
||||
public OssDataSegmentMover(
|
||||
OSS client,
|
||||
OssStorageConfig config
|
||||
)
|
||||
{
|
||||
this.client = client;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment move(DataSegment segment, Map<String, Object> targetLoadSpec) throws SegmentLoadingException
|
||||
{
|
||||
try {
|
||||
Map<String, Object> loadSpec = segment.getLoadSpec();
|
||||
String bucket = MapUtils.getString(loadSpec, "bucket");
|
||||
String key = MapUtils.getString(loadSpec, "key");
|
||||
|
||||
final String targetBucket = MapUtils.getString(targetLoadSpec, "bucket");
|
||||
final String targetKey = MapUtils.getString(targetLoadSpec, "baseKey");
|
||||
|
||||
final String targetPath = OssUtils.constructSegmentPath(
|
||||
targetKey,
|
||||
DataSegmentPusher.getDefaultStorageDir(segment, false)
|
||||
);
|
||||
|
||||
if (targetBucket.isEmpty()) {
|
||||
throw new SegmentLoadingException("Target OSS bucket is not specified");
|
||||
}
|
||||
if (targetPath.isEmpty()) {
|
||||
throw new SegmentLoadingException("Target OSS baseKey is not specified");
|
||||
}
|
||||
|
||||
safeMove(bucket, key, targetBucket, targetPath);
|
||||
|
||||
return segment.withLoadSpec(
|
||||
ImmutableMap.<String, Object>builder()
|
||||
.putAll(
|
||||
Maps.filterKeys(
|
||||
loadSpec,
|
||||
new Predicate<String>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(String input)
|
||||
{
|
||||
return !("bucket".equals(input) || "key".equals(input));
|
||||
}
|
||||
}
|
||||
)
|
||||
)
|
||||
.put("bucket", targetBucket)
|
||||
.put("key", targetPath)
|
||||
.build()
|
||||
);
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new SegmentLoadingException(e, "Unable to move segment[%s]: [%s]", segment.getId(), e);
|
||||
}
|
||||
}
|
||||
|
||||
private void safeMove(
|
||||
final String srcBucket,
|
||||
final String srcPath,
|
||||
final String targetBucket,
|
||||
final String targetPath
|
||||
) throws SegmentLoadingException
|
||||
{
|
||||
try {
|
||||
OssUtils.retry(
|
||||
() -> {
|
||||
final String copyMsg = StringUtils.format(
|
||||
"[%s://%s/%s] to [%s://%s/%s]",
|
||||
OssStorageDruidModule.SCHEME,
|
||||
srcBucket,
|
||||
srcPath,
|
||||
OssStorageDruidModule.SCHEME,
|
||||
targetBucket,
|
||||
targetPath
|
||||
);
|
||||
try {
|
||||
selfCheckingMove(srcBucket, targetBucket, srcPath, targetPath, copyMsg);
|
||||
return null;
|
||||
}
|
||||
catch (OSSException | IOException | SegmentLoadingException e) {
|
||||
log.info(e, "Error while trying to move " + copyMsg);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
Throwables.propagateIfInstanceOf(e, OSSException.class);
|
||||
Throwables.propagateIfInstanceOf(e, SegmentLoadingException.class);
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Copies an object and after that checks that the object is present at the target location, via a separate API call.
|
||||
* If it is not, an exception is thrown, and the object is not deleted at the old location. This "paranoic" check
|
||||
* is added after it was observed that oss may report a successful move, and the object is not found at the target
|
||||
* location.
|
||||
*/
|
||||
private void selfCheckingMove(
|
||||
String srcBucket,
|
||||
String dstBucket,
|
||||
String srcPath,
|
||||
String dstPath,
|
||||
String copyMsg
|
||||
) throws IOException, SegmentLoadingException
|
||||
{
|
||||
if (srcBucket.equals(dstBucket) && srcPath.equals(dstPath)) {
|
||||
log.info("No need to move file[%s://%s/%s] onto itself", OssStorageDruidModule.SCHEME, srcBucket, srcPath);
|
||||
return;
|
||||
}
|
||||
if (client.doesObjectExist(srcBucket, srcPath)) {
|
||||
final ObjectListing listResult = client.listObjects(
|
||||
new ListObjectsRequest(srcBucket, srcPath, null, null, 1)
|
||||
);
|
||||
// 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 (listResult.getObjectSummaries().size() == 0) {
|
||||
// should never happen
|
||||
throw new ISE("Unable to list object [%s://%s/%s]", OssStorageDruidModule.SCHEME, srcBucket, srcPath);
|
||||
}
|
||||
final OSSObjectSummary objectSummary = listResult.getObjectSummaries().get(0);
|
||||
if (objectSummary.getStorageClass() != null &&
|
||||
objectSummary.getStorageClass().equals(StorageClass.IA.name())) {
|
||||
throw new OSSException(
|
||||
StringUtils.format(
|
||||
"Cannot move file[%s://%s/%s] of storage class glacier, skipping.",
|
||||
OssStorageDruidModule.SCHEME,
|
||||
srcBucket,
|
||||
srcPath
|
||||
)
|
||||
);
|
||||
} else {
|
||||
log.info("Moving file %s", copyMsg);
|
||||
final CopyObjectRequest copyRequest = new CopyObjectRequest(srcBucket, srcPath, dstBucket, dstPath);
|
||||
client.copyObject(copyRequest);
|
||||
if (!client.doesObjectExist(dstBucket, dstPath)) {
|
||||
throw new IOE(
|
||||
"After copy was reported as successful the file doesn't exist in the target location [%s]",
|
||||
copyMsg
|
||||
);
|
||||
}
|
||||
deleteWithRetriesSilent(srcBucket, srcPath);
|
||||
log.debug("Finished moving file %s", copyMsg);
|
||||
}
|
||||
} else {
|
||||
// ensure object exists in target location
|
||||
if (client.doesObjectExist(dstBucket, dstPath)) {
|
||||
log.info(
|
||||
"Not moving file [%s://%s/%s], already present in target location [%s://%s/%s]",
|
||||
OssStorageDruidModule.SCHEME,
|
||||
srcBucket,
|
||||
srcPath,
|
||||
OssStorageDruidModule.SCHEME,
|
||||
dstBucket,
|
||||
dstPath
|
||||
);
|
||||
} else {
|
||||
throw new SegmentLoadingException(
|
||||
"Unable to move file %s, not present in either source or target location",
|
||||
copyMsg
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private void deleteWithRetriesSilent(final String bucket, final String path)
|
||||
{
|
||||
try {
|
||||
deleteWithRetries(bucket, path);
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.error(e, "Failed to delete file [%s://%s/%s], giving up", OssStorageDruidModule.SCHEME, bucket, path);
|
||||
}
|
||||
}
|
||||
|
||||
private void deleteWithRetries(final String bucket, final String path) throws Exception
|
||||
{
|
||||
RetryUtils.retry(
|
||||
() -> {
|
||||
try {
|
||||
client.deleteObject(bucket, path);
|
||||
return null;
|
||||
}
|
||||
catch (Exception e) {
|
||||
log.info(e, "Error while trying to delete [%s://%s/%s]", OssStorageDruidModule.SCHEME, bucket, path);
|
||||
throw e;
|
||||
}
|
||||
},
|
||||
OssUtils.RETRYABLE,
|
||||
3
|
||||
);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,308 @@
|
|||
/*
|
||||
* 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.OSSObject;
|
||||
import com.aliyun.oss.model.OSSObjectSummary;
|
||||
import com.google.common.base.Predicate;
|
||||
import com.google.common.base.Strings;
|
||||
import com.google.common.io.ByteSource;
|
||||
import com.google.common.io.Files;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.data.input.impl.CloudObjectLocation;
|
||||
import org.apache.druid.java.util.common.FileUtils;
|
||||
import org.apache.druid.java.util.common.IAE;
|
||||
import org.apache.druid.java.util.common.IOE;
|
||||
import org.apache.druid.java.util.common.RE;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.common.UOE;
|
||||
import org.apache.druid.java.util.common.io.Closer;
|
||||
import org.apache.druid.java.util.common.logger.Logger;
|
||||
import org.apache.druid.segment.loading.SegmentLoadingException;
|
||||
import org.apache.druid.segment.loading.URIDataPuller;
|
||||
import org.apache.druid.utils.CompressionUtils;
|
||||
|
||||
import javax.tools.FileObject;
|
||||
import java.io.File;
|
||||
import java.io.FilterInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.io.OutputStream;
|
||||
import java.io.Reader;
|
||||
import java.io.Writer;
|
||||
import java.net.URI;
|
||||
|
||||
/**
|
||||
* A data segment puller that also hanldes URI data pulls.
|
||||
*/
|
||||
public class OssDataSegmentPuller implements URIDataPuller
|
||||
{
|
||||
private static final Logger log = new Logger(OssDataSegmentPuller.class);
|
||||
|
||||
static final String BUCKET = "bucket";
|
||||
protected static final String KEY = "key";
|
||||
|
||||
protected final OSS client;
|
||||
|
||||
@Inject
|
||||
public OssDataSegmentPuller(OSS client)
|
||||
{
|
||||
this.client = client;
|
||||
}
|
||||
|
||||
FileUtils.FileCopyResult getSegmentFiles(final CloudObjectLocation ossCoords, final File outDir)
|
||||
throws SegmentLoadingException
|
||||
{
|
||||
|
||||
log.info("Pulling index at path[%s] to outDir[%s]", ossCoords, outDir);
|
||||
|
||||
if (!isObjectInBucket(ossCoords)) {
|
||||
throw new SegmentLoadingException("IndexFile[%s] does not exist.", ossCoords);
|
||||
}
|
||||
|
||||
try {
|
||||
org.apache.commons.io.FileUtils.forceMkdir(outDir);
|
||||
|
||||
final URI uri = ossCoords.toUri(OssStorageDruidModule.SCHEME);
|
||||
final ByteSource byteSource = new ByteSource()
|
||||
{
|
||||
@Override
|
||||
public InputStream openStream() throws IOException
|
||||
{
|
||||
try {
|
||||
return buildFileObject(uri).openInputStream();
|
||||
}
|
||||
catch (OSSException e) {
|
||||
if (e.getCause() != null) {
|
||||
if (OssUtils.RETRYABLE.apply(e)) {
|
||||
throw new IOException("Recoverable exception", e);
|
||||
}
|
||||
}
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
};
|
||||
if (CompressionUtils.isZip(ossCoords.getPath())) {
|
||||
final FileUtils.FileCopyResult result = CompressionUtils.unzip(
|
||||
byteSource,
|
||||
outDir,
|
||||
OssUtils.RETRYABLE,
|
||||
false
|
||||
);
|
||||
log.info("Loaded %d bytes from [%s] to [%s]", result.size(), ossCoords.toString(), outDir.getAbsolutePath());
|
||||
return result;
|
||||
}
|
||||
if (CompressionUtils.isGz(ossCoords.getPath())) {
|
||||
final String fname = Files.getNameWithoutExtension(uri.getPath());
|
||||
final File outFile = new File(outDir, fname);
|
||||
|
||||
final FileUtils.FileCopyResult result = CompressionUtils.gunzip(byteSource, outFile, OssUtils.RETRYABLE);
|
||||
log.info("Loaded %d bytes from [%s] to [%s]", result.size(), ossCoords.toString(), outFile.getAbsolutePath());
|
||||
return result;
|
||||
}
|
||||
throw new IAE("Do not know how to load file type at [%s]", uri.toString());
|
||||
}
|
||||
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(),
|
||||
ossCoords.toString()
|
||||
);
|
||||
}
|
||||
throw new SegmentLoadingException(e, e.getMessage());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public InputStream getInputStream(URI uri) throws IOException
|
||||
{
|
||||
try {
|
||||
return buildFileObject(uri).openInputStream();
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new IOE(e, "Could not load URI [%s]", uri);
|
||||
}
|
||||
}
|
||||
|
||||
private FileObject buildFileObject(final URI uri) throws OSSException
|
||||
{
|
||||
final CloudObjectLocation coords = new CloudObjectLocation(OssUtils.checkURI(uri));
|
||||
final OSSObjectSummary objectSummary =
|
||||
OssUtils.getSingleObjectSummary(client, coords.getBucket(), coords.getPath());
|
||||
final String path = uri.getPath();
|
||||
|
||||
return new FileObject()
|
||||
{
|
||||
OSSObject ossObject = null;
|
||||
|
||||
@Override
|
||||
public URI toUri()
|
||||
{
|
||||
return uri;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getName()
|
||||
{
|
||||
final String ext = Files.getFileExtension(path);
|
||||
return Files.getNameWithoutExtension(path) + (Strings.isNullOrEmpty(ext) ? "" : ("." + ext));
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns an input stream for an OSS object. The returned input stream is not thread-safe.
|
||||
*/
|
||||
@Override
|
||||
public InputStream openInputStream() throws IOException
|
||||
{
|
||||
try {
|
||||
if (ossObject == null) {
|
||||
// lazily promote to full GET
|
||||
ossObject = client.getObject(objectSummary.getBucketName(), objectSummary.getKey());
|
||||
}
|
||||
|
||||
final InputStream in = ossObject.getObjectContent();
|
||||
final Closer closer = Closer.create();
|
||||
closer.register(in);
|
||||
closer.register(ossObject);
|
||||
|
||||
return new FilterInputStream(in)
|
||||
{
|
||||
@Override
|
||||
public void close() throws IOException
|
||||
{
|
||||
closer.close();
|
||||
}
|
||||
};
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new IOE(e, "Could not load OSS URI [%s]", uri);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public OutputStream openOutputStream()
|
||||
{
|
||||
throw new UOE("Cannot stream OSS output");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Reader openReader(boolean ignoreEncodingErrors)
|
||||
{
|
||||
throw new UOE("Cannot open reader");
|
||||
}
|
||||
|
||||
@Override
|
||||
public CharSequence getCharContent(boolean ignoreEncodingErrors)
|
||||
{
|
||||
throw new UOE("Cannot open character sequence");
|
||||
}
|
||||
|
||||
@Override
|
||||
public Writer openWriter()
|
||||
{
|
||||
throw new UOE("Cannot open writer");
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getLastModified()
|
||||
{
|
||||
return objectSummary.getLastModified().getTime();
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean delete()
|
||||
{
|
||||
throw new UOE("Cannot delete OSS items anonymously. jetS3t doesn't support authenticated deletes easily.");
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@Override
|
||||
public Predicate<Throwable> shouldRetryPredicate()
|
||||
{
|
||||
// Yay! smart retries!
|
||||
return new Predicate<Throwable>()
|
||||
{
|
||||
@Override
|
||||
public boolean apply(Throwable e)
|
||||
{
|
||||
if (e == null) {
|
||||
return false;
|
||||
}
|
||||
if (e instanceof OSSException) {
|
||||
return OssUtils.isServiceExceptionRecoverable((OSSException) e);
|
||||
}
|
||||
if (OssUtils.RETRYABLE.apply(e)) {
|
||||
return true;
|
||||
}
|
||||
// Look all the way down the cause chain, just in case something wraps it deep.
|
||||
return apply(e.getCause());
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
/**
|
||||
* Returns the "version" (aka last modified timestamp) of the URI
|
||||
*
|
||||
* @param uri The URI to check the last timestamp
|
||||
* @return The time in ms of the last modification of the URI in String format
|
||||
* @throws IOException
|
||||
*/
|
||||
@Override
|
||||
public String getVersion(URI uri) throws IOException
|
||||
{
|
||||
try {
|
||||
final CloudObjectLocation coords = new CloudObjectLocation(OssUtils.checkURI(uri));
|
||||
final OSSObjectSummary objectSummary =
|
||||
OssUtils.getSingleObjectSummary(client, coords.getBucket(), coords.getPath());
|
||||
return StringUtils.format("%d", objectSummary.getLastModified().getTime());
|
||||
}
|
||||
catch (OSSException e) {
|
||||
if (OssUtils.isServiceExceptionRecoverable(e)) {
|
||||
// The recoverable logic is always true for IOException, so we want to only pass IOException if it is recoverable
|
||||
throw new IOE(e, "Could not fetch last modified timestamp from URI [%s]", uri);
|
||||
} else {
|
||||
throw new RE(e, "Error fetching last modified timestamp from URI [%s]", uri);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
private boolean isObjectInBucket(final CloudObjectLocation coords) throws SegmentLoadingException
|
||||
{
|
||||
try {
|
||||
return OssUtils.retry(
|
||||
() -> OssUtils.isObjectInBucketIgnoringPermission(client, coords.getBucket(), coords.getPath())
|
||||
);
|
||||
}
|
||||
catch (OSSException | IOException e) {
|
||||
throw new SegmentLoadingException(e, "fail! Key[%s]", coords);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,131 @@
|
|||
/*
|
||||
* 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.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import com.google.inject.Inject;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.apache.druid.java.util.emitter.EmittingLogger;
|
||||
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.net.URI;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
|
||||
public class OssDataSegmentPusher implements DataSegmentPusher
|
||||
{
|
||||
private static final EmittingLogger log = new EmittingLogger(OssDataSegmentPusher.class);
|
||||
|
||||
private final OSS client;
|
||||
private final OssStorageConfig config;
|
||||
|
||||
@Inject
|
||||
public OssDataSegmentPusher(
|
||||
OSS client,
|
||||
OssStorageConfig config
|
||||
)
|
||||
{
|
||||
this.client = client;
|
||||
this.config = config;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getPathForHadoop()
|
||||
{
|
||||
return StringUtils.format("%s/%s", config.getBucket(), config.getPrefix());
|
||||
}
|
||||
|
||||
@Deprecated
|
||||
@Override
|
||||
public String getPathForHadoop(String dataSource)
|
||||
{
|
||||
return getPathForHadoop();
|
||||
}
|
||||
|
||||
@Override
|
||||
public List<String> getAllowedPropertyPrefixesForHadoop()
|
||||
{
|
||||
return ImmutableList.of("druid.oss");
|
||||
}
|
||||
|
||||
@Override
|
||||
public DataSegment push(final File indexFilesDir, final DataSegment inSegment, final boolean useUniquePath)
|
||||
throws IOException
|
||||
{
|
||||
final String path = OssUtils.constructSegmentPath(config.getPrefix(), getStorageDir(inSegment, useUniquePath));
|
||||
|
||||
log.debug("Copying segment[%s] to OSS at location[%s]", inSegment.getId(), path);
|
||||
|
||||
final File zipOutFile = File.createTempFile("druid", "index.zip");
|
||||
final long indexSize = CompressionUtils.zip(indexFilesDir, zipOutFile);
|
||||
|
||||
final DataSegment outSegment = inSegment.withSize(indexSize)
|
||||
.withLoadSpec(makeLoadSpec(config.getBucket(), path))
|
||||
.withBinaryVersion(SegmentUtils.getVersionFromDir(indexFilesDir));
|
||||
|
||||
try {
|
||||
return OssUtils.retry(
|
||||
() -> {
|
||||
OssUtils.uploadFileIfPossible(client, config.getBucket(), path, zipOutFile);
|
||||
|
||||
return outSegment;
|
||||
}
|
||||
);
|
||||
}
|
||||
catch (OSSException e) {
|
||||
throw new IOException(e);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
finally {
|
||||
log.debug("Deleting temporary cached index.zip");
|
||||
zipOutFile.delete();
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public Map<String, Object> makeLoadSpec(URI finalIndexZipFilePath)
|
||||
{
|
||||
// remove the leading "/"
|
||||
return makeLoadSpec(finalIndexZipFilePath.getHost(), finalIndexZipFilePath.getPath().substring(1));
|
||||
}
|
||||
|
||||
private Map<String, Object> makeLoadSpec(String bucket, String key)
|
||||
{
|
||||
return ImmutableMap.of(
|
||||
"type",
|
||||
OssStorageDruidModule.SCHEME_ZIP,
|
||||
"bucket",
|
||||
bucket,
|
||||
"key",
|
||||
key
|
||||
);
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,52 @@
|
|||
/*
|
||||
* 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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,72 @@
|
|||
/*
|
||||
* 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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,156 @@
|
|||
/*
|
||||
* 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}.
|
||||
* <p>
|
||||
* 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<OSSObjectSummary>
|
||||
{
|
||||
private final OSS client;
|
||||
private final Iterator<URI> prefixesIterator;
|
||||
private final int maxListingLength;
|
||||
|
||||
private ListObjectsRequest request;
|
||||
private ObjectListing result;
|
||||
private Iterator<OSSObjectSummary> objectSummaryIterator;
|
||||
private OSSObjectSummary currentObjectSummary;
|
||||
|
||||
OssObjectSummaryIterator(
|
||||
final OSS client,
|
||||
final Iterable<URI> 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("/");
|
||||
}
|
||||
}
|
|
@ -0,0 +1,50 @@
|
|||
/*
|
||||
* 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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,106 @@
|
|||
/*
|
||||
* 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();
|
||||
}
|
||||
}
|
|
@ -0,0 +1,201 @@
|
|||
/*
|
||||
* 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<ByteSource> streamTaskLog(final String taskid, final long offset) throws IOException
|
||||
{
|
||||
final String taskKey = getTaskLogKey(taskid, "log");
|
||||
return streamTaskFile(offset, taskKey);
|
||||
}
|
||||
|
||||
@Override
|
||||
public Optional<ByteSource> streamTaskReports(String taskid) throws IOException
|
||||
{
|
||||
final String taskKey = getTaskLogKey(taskid, "report.json");
|
||||
return streamTaskFile(0, taskKey);
|
||||
}
|
||||
|
||||
private Optional<ByteSource> 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);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,73 @@
|
|||
/*
|
||||
* 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;
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,90 @@
|
|||
/*
|
||||
* 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<URI>
|
||||
{
|
||||
@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<OSSObjectSummary> 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);
|
||||
}
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,271 @@
|
|||
/*
|
||||
* 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<Throwable> RETRYABLE = new Predicate<Throwable>()
|
||||
{
|
||||
@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> T retry(Task<T> 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.
|
||||
* <p>
|
||||
* 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<OSSObjectSummary> objectSummaryIterator(
|
||||
final OSS client,
|
||||
final Iterable<URI> 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.
|
||||
*
|
||||
* <pre>
|
||||
* {@code oss://{BUCKET_NAME}/{OBJECT_KEY}}
|
||||
* </pre>
|
||||
*/
|
||||
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<OSSObjectSummary> filter
|
||||
)
|
||||
throws Exception
|
||||
{
|
||||
final List<String> 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<String> 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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,18 @@
|
|||
# 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
|
|
@ -0,0 +1,660 @@
|
|||
/*
|
||||
* 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<URI> EXPECTED_URIS = Arrays.asList(
|
||||
URI.create("oss://foo/bar/file.csv"),
|
||||
URI.create("oss://bar/foo/file2.csv")
|
||||
);
|
||||
|
||||
private static final List<URI> 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<List<CloudObjectLocation>> EXPECTED_COORDS =
|
||||
EXPECTED_URIS.stream()
|
||||
.map(uri -> Collections.singletonList(new CloudObjectLocation(uri)))
|
||||
.collect(Collectors.toList());
|
||||
|
||||
private static final List<URI> 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<CloudObjectLocation> 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<InputSplit<List<CloudObjectLocation>>> 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<InputSplit<List<CloudObjectLocation>>> 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<InputSplit<List<CloudObjectLocation>>> 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<InputSplit<List<CloudObjectLocation>>> 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<InputRow> 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<InputRow> 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<URI> 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("<request for prefix [").append(prefixUri).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<OSSClient>())
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void configure(Binder binder)
|
||||
{
|
||||
}
|
||||
|
||||
@Provides
|
||||
public OSS getOssClient()
|
||||
{
|
||||
return OSSCLIENT;
|
||||
}
|
||||
}
|
||||
|
||||
public static class ItemDeserializer<T> extends StdDeserializer<T>
|
||||
{
|
||||
ItemDeserializer()
|
||||
{
|
||||
this(null);
|
||||
}
|
||||
|
||||
ItemDeserializer(Class<?> vc)
|
||||
{
|
||||
super(vc);
|
||||
}
|
||||
|
||||
@Override
|
||||
public T deserialize(JsonParser jp, DeserializationContext ctxt)
|
||||
{
|
||||
throw new UnsupportedOperationException();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,195 @@
|
|||
/*
|
||||
* 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<String, Object> 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<String, Object> 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<String, Object> 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<String, Object> targetLoadSpec)
|
||||
{
|
||||
return SOURCE_SEGMENT;
|
||||
}
|
||||
};
|
||||
Assert.assertNull(archiver.restore(SOURCE_SEGMENT));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,205 @@
|
|||
/*
|
||||
* 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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,266 @@
|
|||
/*
|
||||
* 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<String, Object> 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<String, Object> 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<String, Set<String>> 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<String> 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();
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,205 @@
|
|||
/*
|
||||
* 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());
|
||||
}
|
||||
|
||||
}
|
|
@ -0,0 +1,51 @@
|
|||
/*
|
||||
* 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));
|
||||
}
|
||||
}
|
|
@ -0,0 +1,125 @@
|
|||
/*
|
||||
* 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<T>
|
||||
{
|
||||
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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,276 @@
|
|||
/*
|
||||
* 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<OSSObjectSummary> 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<String> expectedUris,
|
||||
final List<String> prefixes,
|
||||
final int maxListingLength
|
||||
)
|
||||
{
|
||||
final List<OSSObjectSummary> expectedObjects = new ArrayList<>();
|
||||
|
||||
// O(N^2) but who cares -- the list is short.
|
||||
for (final String uri : expectedUris) {
|
||||
final List<OSSObjectSummary> matches = TEST_OBJECTS.stream()
|
||||
.filter(
|
||||
summary ->
|
||||
OssUtils.summaryToUri(summary).toString().equals(uri)
|
||||
)
|
||||
.collect(Collectors.toList());
|
||||
|
||||
expectedObjects.add(Iterables.getOnlyElement(matches));
|
||||
}
|
||||
|
||||
final List<OSSObjectSummary> 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<OSSObjectSummary> 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<OSSObjectSummary> 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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,336 @@
|
|||
/*
|
||||
* 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<Grant> 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<Grant> 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());
|
||||
}
|
||||
}
|
|
@ -0,0 +1,177 @@
|
|||
/*
|
||||
* 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<String> expectedKeysAndVersions = deleteObjectsRequest.getKeys();
|
||||
List<String> 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<OSSObjectSummary> 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<DeleteObjectsRequest> deleteRequestsExpected,
|
||||
Map<DeleteObjectsRequest, Exception> requestToException
|
||||
)
|
||||
{
|
||||
Map<DeleteObjectsRequest, IExpectationSetters<DeleteObjectsResult>> requestToResultExpectationSetter = new HashMap<>();
|
||||
|
||||
for (Map.Entry<DeleteObjectsRequest, Exception> requestsAndErrors : requestToException.entrySet()) {
|
||||
DeleteObjectsRequest request = requestsAndErrors.getKey();
|
||||
Exception exception = requestsAndErrors.getValue();
|
||||
IExpectationSetters<DeleteObjectsResult> resultExpectationSetter = requestToResultExpectationSetter.get(request);
|
||||
if (resultExpectationSetter == null) {
|
||||
client.deleteObjects(
|
||||
OssTestUtils.deleteObjectsRequestArgumentMatcher(request));
|
||||
resultExpectationSetter = EasyMock.<DeleteObjectsResult>expectLastCall().andThrow(exception);
|
||||
requestToResultExpectationSetter.put(request, resultExpectationSetter);
|
||||
} else {
|
||||
resultExpectationSetter.andThrow(exception);
|
||||
}
|
||||
}
|
||||
|
||||
for (DeleteObjectsRequest request : deleteRequestsExpected) {
|
||||
IExpectationSetters<DeleteObjectsResult> 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("<request for prefix [").append(prefixUri).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;
|
||||
}
|
||||
}
|
|
@ -0,0 +1,178 @@
|
|||
/*
|
||||
* 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);
|
||||
}
|
||||
}
|
|
@ -0,0 +1,30 @@
|
|||
#
|
||||
# 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.
|
||||
#
|
||||
|
||||
#
|
||||
# Example of override config file to provide.
|
||||
# Please replace <OVERRIDE_THIS> with your cloud configs/credentials
|
||||
#
|
||||
druid_storage_type=oss
|
||||
druid_storage_oss_bucket=<OVERRIDE_THIS>
|
||||
druid_storage_oss_prefix=<OVERRIDE_THIS>
|
||||
druid_oss_accessKey=<OVERRIDE_THIS>
|
||||
druid_oss_secretKey=<OVERRIDE_THIS>
|
||||
druid_oss_endpoint=<OVERRIDE_THIS>
|
||||
druid_extensions_loadList=["aliyun-oss-extensions"]
|
|
@ -76,6 +76,13 @@ public class TestNGGroup
|
|||
*/
|
||||
public static final String AZURE_DEEP_STORAGE = "azure-deep-storage";
|
||||
|
||||
/**
|
||||
* This group is not part of CI. To run this group, azure configs/credentials for your oss must be provided in a file.
|
||||
* The path of the file must then be pass to mvn with -Doverride.config.path=<PATH_TO_FILE>
|
||||
* See integration-tests/docker/environment-configs/override-examples/oss for env vars to provide.
|
||||
*/
|
||||
public static final String ALIYUN_OSS_DEEP_STORAGE = "aliyun-oss-deep-storage";
|
||||
|
||||
/**
|
||||
* This group is not part of CI. To run this group, hadoop configs must be provided in a file. The path of the file
|
||||
* must then be pass to mvn with -Doverride.config.path=<PATH_TO_FILE>
|
||||
|
|
|
@ -0,0 +1,132 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.tests.indexer;
|
||||
|
||||
import com.google.common.collect.ImmutableList;
|
||||
import com.google.common.collect.ImmutableMap;
|
||||
import org.apache.druid.indexer.partitions.DynamicPartitionsSpec;
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.java.util.common.StringUtils;
|
||||
import org.testng.annotations.DataProvider;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.List;
|
||||
import java.util.UUID;
|
||||
import java.util.function.Function;
|
||||
|
||||
public abstract class AbstractOssInputSourceParallelIndexTest extends AbstractITBatchIndexTest
|
||||
{
|
||||
private static final String INDEX_TASK = "/indexer/wikipedia_cloud_index_task.json";
|
||||
private static final String INDEX_QUERIES_RESOURCE = "/indexer/wikipedia_index_queries.json";
|
||||
private static final String INPUT_SOURCE_URIS_KEY = "uris";
|
||||
private static final String INPUT_SOURCE_PREFIXES_KEY = "prefixes";
|
||||
private static final String INPUT_SOURCE_OBJECTS_KEY = "objects";
|
||||
private static final String WIKIPEDIA_DATA_1 = "wikipedia_index_data1.json";
|
||||
private static final String WIKIPEDIA_DATA_2 = "wikipedia_index_data2.json";
|
||||
private static final String WIKIPEDIA_DATA_3 = "wikipedia_index_data3.json";
|
||||
|
||||
@DataProvider
|
||||
public static Object[][] resources()
|
||||
{
|
||||
return new Object[][]{
|
||||
{new Pair<>(INPUT_SOURCE_URIS_KEY,
|
||||
ImmutableList.of(
|
||||
"oss://%%BUCKET%%/%%PATH%%" + WIKIPEDIA_DATA_1,
|
||||
"oss://%%BUCKET%%/%%PATH%%" + WIKIPEDIA_DATA_2,
|
||||
"oss://%%BUCKET%%/%%PATH%%" + WIKIPEDIA_DATA_3
|
||||
)
|
||||
)},
|
||||
{new Pair<>(INPUT_SOURCE_PREFIXES_KEY,
|
||||
ImmutableList.of(
|
||||
"oss://%%BUCKET%%/%%PATH%%"
|
||||
)
|
||||
)},
|
||||
{new Pair<>(INPUT_SOURCE_OBJECTS_KEY,
|
||||
ImmutableList.of(
|
||||
ImmutableMap.of("bucket", "%%BUCKET%%", "path", "%%PATH%%" + WIKIPEDIA_DATA_1),
|
||||
ImmutableMap.of("bucket", "%%BUCKET%%", "path", "%%PATH%%" + WIKIPEDIA_DATA_2),
|
||||
ImmutableMap.of("bucket", "%%BUCKET%%", "path", "%%PATH%%" + WIKIPEDIA_DATA_3)
|
||||
)
|
||||
)}
|
||||
};
|
||||
}
|
||||
|
||||
void doTest(Pair<String, List> inputSource) throws Exception
|
||||
{
|
||||
final String indexDatasource = "wikipedia_index_test_" + UUID.randomUUID();
|
||||
try (
|
||||
final Closeable ignored1 = unloader(indexDatasource + config.getExtraDatasourceNameSuffix());
|
||||
) {
|
||||
final Function<String, String> propsTransform = spec -> {
|
||||
try {
|
||||
String inputSourceValue = jsonMapper.writeValueAsString(inputSource.rhs);
|
||||
inputSourceValue = StringUtils.replace(
|
||||
inputSourceValue,
|
||||
"%%BUCKET%%",
|
||||
config.getCloudBucket()
|
||||
);
|
||||
inputSourceValue = StringUtils.replace(
|
||||
inputSourceValue,
|
||||
"%%PATH%%",
|
||||
config.getCloudPath()
|
||||
);
|
||||
spec = StringUtils.replace(
|
||||
spec,
|
||||
"%%INPUT_FORMAT_TYPE%%",
|
||||
InputFormatDetails.JSON.getInputFormatType()
|
||||
);
|
||||
spec = StringUtils.replace(
|
||||
spec,
|
||||
"%%PARTITIONS_SPEC%%",
|
||||
jsonMapper.writeValueAsString(new DynamicPartitionsSpec(null, null))
|
||||
);
|
||||
spec = StringUtils.replace(
|
||||
spec,
|
||||
"%%INPUT_SOURCE_TYPE%%",
|
||||
"oss"
|
||||
);
|
||||
spec = StringUtils.replace(
|
||||
spec,
|
||||
"%%INPUT_SOURCE_PROPERTY_KEY%%",
|
||||
inputSource.lhs
|
||||
);
|
||||
return StringUtils.replace(
|
||||
spec,
|
||||
"%%INPUT_SOURCE_PROPERTY_VALUE%%",
|
||||
inputSourceValue
|
||||
);
|
||||
}
|
||||
catch (Exception e) {
|
||||
throw new RuntimeException(e);
|
||||
}
|
||||
};
|
||||
|
||||
doIndexTest(
|
||||
indexDatasource,
|
||||
INDEX_TASK,
|
||||
propsTransform,
|
||||
INDEX_QUERIES_RESOURCE,
|
||||
false,
|
||||
true,
|
||||
true
|
||||
);
|
||||
}
|
||||
}
|
||||
}
|
|
@ -0,0 +1,49 @@
|
|||
/*
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
*
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
*
|
||||
* Unless required by applicable law or agreed to in writing,
|
||||
* software distributed under the License is distributed on an
|
||||
* "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
|
||||
* KIND, either express or implied. See the License for the
|
||||
* specific language governing permissions and limitations
|
||||
* under the License.
|
||||
*/
|
||||
|
||||
package org.apache.druid.tests.indexer;
|
||||
|
||||
import org.apache.druid.java.util.common.Pair;
|
||||
import org.apache.druid.testing.guice.DruidTestModuleFactory;
|
||||
import org.apache.druid.tests.TestNGGroup;
|
||||
import org.testng.annotations.Guice;
|
||||
import org.testng.annotations.Test;
|
||||
|
||||
import java.util.List;
|
||||
|
||||
/**
|
||||
* IMPORTANT:
|
||||
* To run this test, you must:
|
||||
* 1) Set the bucket and path for your data. This can be done by setting -Ddruid.test.config.cloudBucket and
|
||||
* -Ddruid.test.config.cloudPath or setting "cloud_bucket" and "cloud_path" in the config file.
|
||||
* 2) Copy wikipedia_index_data1.json, wikipedia_index_data2.json, and wikipedia_index_data3.json
|
||||
* located in integration-tests/src/test/resources/data/batch_index/json to your Aliyun OSS at the location set in step 1.
|
||||
* 3) Provide -Doverride.config.path=<PATH_TO_FILE> with Aliyun OSS credentials/configs set. See
|
||||
* integration-tests/docker/environment-configs/override-examples/oss for env vars to provide.
|
||||
*/
|
||||
@Test(groups = TestNGGroup.ALIYUN_OSS_DEEP_STORAGE)
|
||||
@Guice(moduleFactory = DruidTestModuleFactory.class)
|
||||
public class ITOssToOssParallelIndexTest extends AbstractOssInputSourceParallelIndexTest
|
||||
{
|
||||
@Test(dataProvider = "resources")
|
||||
public void testAliyunOssIndexData(Pair<String, List> ossInputSource) throws Exception
|
||||
{
|
||||
doTest(ossInputSource);
|
||||
}
|
||||
}
|
1
pom.xml
1
pom.xml
|
@ -193,6 +193,7 @@
|
|||
<module>extensions-contrib/tdigestsketch</module>
|
||||
<module>extensions-contrib/influxdb-emitter</module>
|
||||
<module>extensions-contrib/gce-extensions</module>
|
||||
<module>extensions-contrib/aliyun-oss-extensions</module>
|
||||
<!-- distribution packaging -->
|
||||
<module>distribution</module>
|
||||
</modules>
|
||||
|
|
|
@ -574,6 +574,14 @@ thriftJar
|
|||
- ../docs/development/extensions-contrib/time-min-max.md
|
||||
timeMax
|
||||
timeMin
|
||||
- ../docs/development/extensions-contrib/aliyun-oss-extensions.md
|
||||
Aliyun
|
||||
aliyun
|
||||
OSS
|
||||
AccessKey
|
||||
aliyun-oss
|
||||
oss
|
||||
url
|
||||
- ../docs/development/extensions-core/approximate-histograms.md
|
||||
approxHistogram
|
||||
approxHistogramFold
|
||||
|
|
Loading…
Reference in New Issue