From 6e4ddd9728bde9781e8884ae88596ca732d7cc40 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 16 May 2016 16:38:09 +0900 Subject: [PATCH 01/27] TAJO-2069: Implement finding the total size of all objects in a bucket with AWS SDK. --- .../java/org/apache/tajo/conf/TajoConf.java | 9 + .../apache/tajo/master/exec/DDLExecutor.java | 16 +- .../org/apache/tajo/querymaster/Query.java | 28 +- tajo-dist/pom.xml | 7 +- tajo-dist/src/main/bin/tajo | 11 + .../apache/tajo/storage/FileTablespace.java | 6 +- tajo-storage/tajo-storage-s3/pom.xml | 487 +++++++++++++- .../apache/tajo/storage/s3/S3TableSpace.java | 133 ++++ .../tajo/storage/s3/TajoS3Credentials.java | 90 +++ .../apache/tajo/storage/s3/MockAmazonS3.java | 598 ++++++++++++++++++ .../tajo/storage/s3/TestS3TableSpace.java | 54 +- 11 files changed, 1394 insertions(+), 45 deletions(-) create mode 100644 tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java create mode 100644 tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 6e3eaeae8a..0618b8dc57 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -232,6 +232,15 @@ public static enum ConfVars implements ConfigKey { // for RCFile HIVEUSEEXPLICITRCFILEHEADER("tajo.exec.rcfile.use.explicit.header", true, Validators.bool()), + // S3 Configuration -------------------------------------------------- + S3_MAX_ERROR_RETRIES("tajo.s3.max-error-retries", 10), + S3_SSL_ENABLED("tajo.s3.ssl.enabled", true), + S3_CONNECT_TIMEOUT("tajo.s3.connect-timeout", "5m"), + S3_SOCKET_TIMEOUT("tajo.s3.socket-timeout", "5m"), + S3_MAX_CONNECTIONS("tajo.s3.max-connections", 500), + S3_USE_INSTANCE_CREDENTIALS("tajo.s3.use-instance-credentials", true), + S3_PIN_CLIENT_TO_CURRENT_REGION("tajo.s3.pin-client-to-current-region", false), + // RPC -------------------------------------------------------------------- // Internal RPC Client INTERNAL_RPC_CLIENT_WORKER_THREAD_NUM("tajo.internal.rpc.client.worker-thread-num", diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java index c480d04d44..5d53de073a 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java @@ -20,7 +20,6 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -512,8 +511,8 @@ public void alterTable(TajoMaster.MasterContext context, final QueryContext quer long numBytes = 0L; if (fs.exists(partitionPath)) { - ContentSummary summary = fs.getContentSummary(partitionPath); - numBytes = summary.getLength(); + FileTablespace fileTablespace = TablespaceManager.get(partitionPath.toUri()); + numBytes = fileTablespace.calculateSize(partitionPath); } catalog.alterTable(CatalogUtil.addOrDropPartition(qualifiedName, alterTable.getPartitionColumns(), @@ -624,6 +623,7 @@ public void repairPartition(TajoMaster.MasterContext context, final QueryContext } // Find missing partitions from CatalogStore + Tablespace tablespace = TablespaceManager.get(tableDesc.getUri()); List targetPartitions = new ArrayList<>(); for(Path filteredPath : filteredPaths) { @@ -632,7 +632,7 @@ public void repairPartition(TajoMaster.MasterContext context, final QueryContext // if there is partition column in the path if (startIdx > -1) { - PartitionDescProto targetPartition = getPartitionDesc(tablePath, filteredPath, fs); + PartitionDescProto targetPartition = getPartitionDesc(tablespace, tablePath, filteredPath); if (!existingPartitionNames.contains(targetPartition.getPartitionName())) { if (LOG.isDebugEnabled()) { LOG.debug("Partitions not in CatalogStore:" + targetPartition.getPartitionName()); @@ -658,7 +658,8 @@ public void repairPartition(TajoMaster.MasterContext context, final QueryContext LOG.info("Total added partitions to CatalogStore: " + targetPartitions.size()); } - private PartitionDescProto getPartitionDesc(Path tablePath, Path partitionPath, FileSystem fs) throws IOException { + private PartitionDescProto getPartitionDesc(Tablespace tablespace, Path tablePath, Path partitionPath) + throws IOException { String partitionName = StringUtils.unescapePathName(partitionPath.toString()); int startIndex = partitionName.indexOf(tablePath.toString()) + tablePath.toString().length(); @@ -680,9 +681,8 @@ private PartitionDescProto getPartitionDesc(Path tablePath, Path partitionPath, } builder.setPath(partitionPath.toString()); - - ContentSummary contentSummary = fs.getContentSummary(partitionPath); - builder.setNumBytes(contentSummary.getLength()); + FileTablespace fileTablespace = TablespaceManager.get(partitionPath.toUri()); + builder.setNumBytes(fileTablespace.calculateSize(partitionPath)); return builder.build(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java index 289d933bd1..ba0af3a2c9 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java @@ -23,8 +23,6 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.fs.ContentSummary; -import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.*; @@ -50,6 +48,7 @@ import org.apache.tajo.master.event.*; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; +import org.apache.tajo.storage.FileTablespace; import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.storage.Tablespace; @@ -540,8 +539,8 @@ private QueryState finalizeQuery(Query query, QueryCompletedEvent event) { if (queryContext.hasOutputTableUri() && queryContext.hasPartition()) { List partitions = query.getPartitions(); if (partitions != null) { - // Set contents length and file count to PartitionDescProto by listing final output directories. - List finalPartitions = getPartitionsWithContentsSummary(query.systemConf, + // Find each partition volume by listing all partitions. + List finalPartitions = getPartitionsWithContentsSummary(queryContext, finalOutputDir, partitions); String databaseName, simpleTableName; @@ -572,16 +571,14 @@ private QueryState finalizeQuery(Query query, QueryCompletedEvent event) { return QueryState.QUERY_SUCCEEDED; } - private List getPartitionsWithContentsSummary(TajoConf conf, Path outputDir, - List partitions) throws IOException { + private List getPartitionsWithContentsSummary(QueryContext queryContext, + Path outputDir, List partitions) throws IOException { List finalPartitions = new ArrayList<>(); - FileSystem fileSystem = outputDir.getFileSystem(conf); for (PartitionDescProto partition : partitions) { PartitionDescProto.Builder builder = partition.toBuilder(); Path partitionPath = new Path(outputDir, partition.getPath()); - ContentSummary contentSummary = fileSystem.getContentSummary(partitionPath); - builder.setNumBytes(contentSummary.getLength()); + builder.setNumBytes(calculateSize(partitionPath)); finalPartitions.add(builder.build()); } return finalPartitions; @@ -688,7 +685,7 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo TableStats aggregated = query.aggregateTableStatsOfTerminalBlock(); resultTableDesc.setStats(aggregated); } else { - stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir)); + stats.setNumBytes(calculateSize(finalOutputDir)); resultTableDesc.setStats(stats); } @@ -731,7 +728,7 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo TableStats aggregated = query.aggregateTableStatsOfTerminalBlock(); tableDescTobeCreated.setStats(aggregated); } else { - stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir)); + stats.setNumBytes(calculateSize(finalOutputDir)); tableDescTobeCreated.setStats(stats); } @@ -774,7 +771,7 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo TableStats aggregated = query.aggregateTableStatsOfTerminalBlock(); finalTable.setStats(aggregated); } else { - long volume = getTableVolume(query.systemConf, finalOutputDir); + long volume = calculateSize(finalOutputDir); stats.setNumBytes(volume); finalTable.setStats(stats); } @@ -792,10 +789,9 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo } } - public static long getTableVolume(TajoConf systemConf, Path tablePath) throws IOException { - FileSystem fs = tablePath.getFileSystem(systemConf); - ContentSummary directorySummary = fs.getContentSummary(tablePath); - return directorySummary.getLength(); + public static long calculateSize(Path path) throws IOException { + FileTablespace fileTablespace = TablespaceManager.get(path.toUri()); + return fileTablespace.calculateSize(path); } public static class StageCompletedTransition implements SingleArcTransition { diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml index 7280e1f2fc..5afe67a924 100644 --- a/tajo-dist/pom.xml +++ b/tajo-dist/pom.xml @@ -146,6 +146,7 @@ run cp -r $ROOT/tajo-sql-parser/target/tajo-sql-parser-${project.version}/* . run cp -r $ROOT/tajo-storage/tajo-storage-jdbc/target/tajo-storage-jdbc-${project.version}.jar . run cp -r $ROOT/tajo-storage/tajo-storage-pgsql/target/tajo-storage-pgsql-${project.version}.jar . + run cp -r $ROOT/tajo-storage/tajo-storage-s3/target/tajo-storage-s3-${project.version}.jar . run cp -r $ROOT/tajo-pullserver/target/tajo-pullserver-${project.version}.jar . run cp -r $ROOT/tajo-metrics/target/tajo-metrics-${project.version}.jar . run cp -r $ROOT/tajo-core/target/tajo-core-${project.version}.jar . @@ -156,7 +157,11 @@ run mkdir -p lib run cp -r $ROOT/tajo-storage/tajo-storage-hdfs/target/lib/hive-*.jar lib/ - + + run mkdir aws_s3 + run cp -r $ROOT/tajo-storage/tajo-storage-s3/target/lib/*.jar aws_s3/ + + run mkdir -p share/jdbc-dist run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo index 8dffe9ac29..30b2f91b75 100755 --- a/tajo-dist/src/main/bin/tajo +++ b/tajo-dist/src/main/bin/tajo @@ -336,6 +336,17 @@ fi # Hive Home Configuration End ############################################################################## +############################################################################## +# Find and Set AWS S3 CLASSPATH +############################################################################## + +AWS_S3_LIB=$TAJO_HOME/aws_s3 + +if [ -d ${AWS_S3_LIB} ]; then + for f in ${AWS_S3_LIB}/*.jar; do + CLASSPATH=${CLASSPATH}:$f; + done +fi ############################################################################## # Find and Set Tajo CLASSPATH diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java index 2aa2b91025..a1efa6b626 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java @@ -128,13 +128,13 @@ protected void storageInit() throws IOException { @Override public long getTableVolume(TableDesc table, Optional filter) throws UnsupportedException { Path path = new Path(table.getUri()); - ContentSummary summary; + long totalVolume = 0L; try { - summary = fs.getContentSummary(path); + totalVolume = calculateSize(path); } catch (IOException e) { throw new TajoInternalError(e); } - return summary.getLength(); + return totalVolume; } @Override diff --git a/tajo-storage/tajo-storage-s3/pom.xml b/tajo-storage/tajo-storage-s3/pom.xml index a9a541aed1..4af62e0062 100644 --- a/tajo-storage/tajo-storage-s3/pom.xml +++ b/tajo-storage/tajo-storage-s3/pom.xml @@ -34,6 +34,11 @@ UTF-8 UTF-8 + 0.122 + 1.7.4 + 2.8.2 + 11.0.2 + 2.2.3 @@ -97,6 +102,27 @@ true + + + org.apache.maven.plugins + maven-dependency-plugin + + + copy-dependencies + package + + copy-dependencies + + + runtime + ${project.build.directory}/lib + false + false + true + + + + @@ -105,16 +131,182 @@ org.apache.tajo tajo-common provided + + + org.apache.hadoop + hadoop-common + + + com.google.protobuf + protobuf-java + + + commons-logging + commons-logging + + + commons-logging + commons-logging-api + + + commons-lang + commons-lang + + + com.google.guava + guava + + + com.google.code.gson + gson + + + io.netty + netty-buffer + + + org.iq80.snappy + snappy + + org.apache.tajo tajo-storage-common provided + + + org.apache.tajo + tajo-common + + + org.apache.tajo + tajo-catalog-common + + + org.apache.tajo + tajo-plan + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + com.google.protobuf + protobuf-java + + + io.netty + netty-buffer + + org.apache.tajo tajo-storage-hdfs provided + + + io.netty + netty-transport + + + io.netty + netty-codec + + + io.netty + netty-codec-http + + + org.apache.tajo + tajo-common + + + org.apache.tajo + tajo-catalog-common + + + org.apache.tajo + tajo-plan + + + org.apache.tajo + tajo-storage-common + + + org.apache.avro + trevni-core + + + org.apache.avro + trevni-avro + + + org.apache.hadoop + hadoop-common + + + org.apache.hadoop + hadoop-hdfs + + + org.apache.hadoop + hadoop-mapreduce-client-core + + + com.google.protobuf + protobuf-java + + + com.twitter + parquet-column + + + com.twitter + parquet-hadoop + + + com.twitter + parquet-format + + + io.netty + netty-buffer + + + com.facebook.presto + presto-orc + + + + + + org.apache.tajo + tajo-catalog-common + test + + + org.apache.tajo + tajo-common + + + org.apache.hadoop + hadoop-common + + + com.google.protobuf + protobuf-java + + @@ -122,56 +314,321 @@ hadoop-common provided + + com.google.code.gson + gson + + + com.jcraft + jsch + + + org.apache.curator + curator-client + + + org.apache.curator + curator-recipes + + + org.apache.htrace + htrace-core + zookeeper org.apache.zookeeper - slf4j-api - org.slf4j + org.apache.commons + commons-compress - jersey-json - com.sun.jersey + org.apache.ant + ant + + + org.apache.commons + commons-compress + + org.apache.hadoop hadoop-hdfs - provided + test - commons-el - commons-el + org.apache.hadoop + hadoop-annotations + + + org.apache.hadoop + hadoop-auth - tomcat - jasper-runtime + org.apache.hadoop + hadoop-common - tomcat - jasper-compiler + com.google.guava + guava org.mortbay.jetty - jsp-2.1-jetty + jetty - com.sun.jersey.jersey-test-framework - jersey-test-framework-grizzly2 + org.mortbay.jetty + jetty-util + + + com.sun.jersey + jersey-core + + + com.sun.jersey + jersey-server + + + commons-cli + commons-cli + + + commons-codec + commons-codec + + + commons-io + commons-io + + + commons-lang + commons-lang + + + commons-logging + commons-logging + + + commons-daemon + commons-daemon + + + log4j + log4j + + + com.google.protobuf + protobuf-java + + + javax.servlet + servlet-api + + + org.slf4j + slf4j-log4j12 + + + org.codehaus.jackson + jackson-core-asl + + + org.codehaus.jackson + jackson-mapper-asl + + + xmlenc + xmlenc - netty-all io.netty + netty-all + + + xerces + xercesImpl + + + org.apache.htrace + htrace-core + + + org.fusesource.leveldbjni + leveldbjni-all + + + org.apache.tajo + tajo-plan + test + + + + com.fasterxml.jackson.core + jackson-databind + test + ${jackson2.version} + + + com.fasterxml.jackson.core + jackson-annotations + test + ${jackson2.version} + + junit junit test + + + io.airlift + units + ${airlft.version} + + + javax.validation + validation-api + + + com.google.guava + guava + + + com.fasterxml.jackson.core + jackson-annotations + + + + + + org.weakref + jmxutils + 1.18 + + + + com.amazonaws + aws-java-sdk + ${aws-java-sdk.version} + + + commons-logging + commons-logging + + + org.apache.httpcomponents + httpclient + + + joda-time + joda-time + + + javax.mail + mail + + + org.freemarker + freemarker + + + org.springframework + spring-beans + + + org.springframework + spring-core + + + org.springframework + spring-context + + + org.springframework + spring-test + + + org.aspectj + aspectjrt + + + junit + junit + + + com.fasterxml.jackson.core + jackson-annotations + + + com.fasterxml.jackson.core + jackson-core + + + com.fasterxml.jackson.core + jackson-databind + + + + + + org.apache.httpcomponents + httpclient + 4.5 + + + org.apache.httpcomponents + httpcore + + + commons-logging + commons-logging + + + commons-codec + commons-codec + + + + + + org.apache.httpcomponents + httpcore + 4.4.1 + + + junit + junit + + + + + + org.testng + testng + 6.9.6 + test + + + + joda-time + joda-time + ${joda-time.version} + compile + + + + javax.validation + validation-api + 1.1.0.Final + + diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index 4bcdb60a68..9fa61cfbed 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -18,14 +18,147 @@ package org.apache.tajo.storage.s3; +import java.io.IOException; import java.net.URI; +import java.util.stream.Stream; +import java.util.stream.StreamSupport; +import com.amazonaws.ClientConfiguration; +import com.amazonaws.Protocol; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import com.amazonaws.auth.InstanceProfileCredentialsProvider; +import com.amazonaws.internal.StaticCredentialsProvider; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.amazonaws.services.s3.iterable.S3Objects; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import com.google.common.annotations.VisibleForTesting; +import com.google.common.primitives.Ints; +import io.airlift.units.Duration; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.tajo.conf.TajoConf; import org.apache.tajo.storage.FileTablespace; import net.minidev.json.JSONObject; +import static com.google.common.base.Preconditions.checkArgument; +import static com.google.common.base.Strings.nullToEmpty; + public class S3TableSpace extends FileTablespace { + private final static Log LOG = LogFactory.getLog(S3TableSpace.class); + + private AmazonS3 s3; + private boolean useInstanceCredentials; + //use a custom endpoint? + public static final String ENDPOINT = "fs.s3a.endpoint"; + public S3TableSpace(String spaceName, URI uri, JSONObject config) { super(spaceName, uri, config); } + + @Override + public void init(TajoConf tajoConf) throws IOException { + super.init(tajoConf); + + int maxErrorRetries = conf.getIntVar(TajoConf.ConfVars.S3_MAX_ERROR_RETRIES); + boolean sslEnabled = conf.getBoolVar(TajoConf.ConfVars.S3_SSL_ENABLED); + + Duration connectTimeout = Duration.valueOf(conf.getVar(TajoConf.ConfVars.S3_CONNECT_TIMEOUT)); + Duration socketTimeout = Duration.valueOf(conf.getVar(TajoConf.ConfVars.S3_SOCKET_TIMEOUT)); + int maxConnections = conf.getIntVar(TajoConf.ConfVars.S3_MAX_CONNECTIONS); + + this.useInstanceCredentials = conf.getBoolVar(TajoConf.ConfVars.S3_USE_INSTANCE_CREDENTIALS); + + ClientConfiguration configuration = new ClientConfiguration() + .withMaxErrorRetry(maxErrorRetries) + .withProtocol(sslEnabled ? Protocol.HTTPS : Protocol.HTTP) + .withConnectionTimeout(Ints.checkedCast(connectTimeout.toMillis())) + .withSocketTimeout(Ints.checkedCast(socketTimeout.toMillis())) + .withMaxConnections(maxConnections); + + this.s3 = createAmazonS3Client(uri, conf, configuration); + + if (s3 != null) { + String endPoint = conf.getTrimmed(ENDPOINT,""); + try { + if (!endPoint.isEmpty()) { + s3.setEndpoint(endPoint); + } + } catch (IllegalArgumentException e) { + String msg = "Incorrect endpoint: " + e.getMessage(); + LOG.error(msg); + throw new IllegalArgumentException(msg, e); + } + + LOG.info("Amazon3Client is initialized."); + } + } + + private AmazonS3Client createAmazonS3Client(URI uri, Configuration hadoopConfig, ClientConfiguration clientConfig) { + AWSCredentialsProvider credentials = getAwsCredentialsProvider(uri, hadoopConfig); + AmazonS3Client client = new AmazonS3Client(credentials, clientConfig); + return client; + } + + private AWSCredentialsProvider getAwsCredentialsProvider(URI uri, Configuration conf) { + // first try credentials from URI or static properties + try { + return new StaticCredentialsProvider(getAwsCredentials(uri, conf)); + } catch (IllegalArgumentException ignored) { + } + + if (useInstanceCredentials) { + return new InstanceProfileCredentialsProvider(); + } + + throw new RuntimeException("S3 credentials not configured"); + } + + private static AWSCredentials getAwsCredentials(URI uri, Configuration conf) { + TajoS3Credentials credentials = new TajoS3Credentials(); + credentials.initialize(uri, conf); + return new BasicAWSCredentials(credentials.getAccessKey(), credentials.getSecretAccessKey()); + } + + @Override + public long calculateSize(Path path) throws IOException { + String key = keyFromPath(path); + if (!key.isEmpty()) { + key += "/"; + } + + Iterable objectSummaries = S3Objects.withPrefix(s3, uri.getHost(), key); + Stream objectStream = StreamSupport.stream(objectSummaries.spliterator(), false); + long totalBucketSize = objectStream.mapToLong(object -> object.getSize()).sum(); + objectStream.close(); + return totalBucketSize; + } + + private String keyFromPath(Path path) + { + checkArgument(path.isAbsolute(), "Path is not absolute: %s", path); + String key = nullToEmpty(path.toUri().getPath()); + if (key.startsWith("/")) { + key = key.substring(1); + } + if (key.endsWith("/")) { + key = key.substring(0, key.length() - 1); + } + return key; + } + + @VisibleForTesting + public AmazonS3 getAmazonS3Client() { + return s3; + } + + @VisibleForTesting + public void setAmazonS3Client(AmazonS3 s3) { + this.s3 = s3; + } } diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java new file mode 100644 index 0000000000..716d92260e --- /dev/null +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java @@ -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.tajo.storage.s3; + +import java.net.URI; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; + +/** + *

+ * Extracts AWS credentials from the filesystem URI or configuration. (borrowed from hadoop-aws package) + *

+ */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +public class TajoS3Credentials { + + private String accessKey; + private String secretAccessKey; + + /** + * @throws IllegalArgumentException if credentials for S3 cannot be + * determined. + */ + public void initialize(URI uri, Configuration conf) { + if (uri.getHost() == null) { + throw new IllegalArgumentException("Invalid hostname in URI " + uri); + } + + String userInfo = uri.getUserInfo(); + if (userInfo != null) { + int index = userInfo.indexOf(':'); + if (index != -1) { + accessKey = userInfo.substring(0, index); + secretAccessKey = userInfo.substring(index + 1); + } else { + accessKey = userInfo; + } + } + + String scheme = uri.getScheme(); + String accessKeyProperty = String.format("fs.%s.awsAccessKeyId", scheme); + String secretAccessKeyProperty = + String.format("fs.%s.awsSecretAccessKey", scheme); + if (accessKey == null) { + accessKey = conf.getTrimmed(accessKeyProperty); + } + if (secretAccessKey == null) { + secretAccessKey = conf.getTrimmed(secretAccessKeyProperty); + } + if (accessKey == null && secretAccessKey == null) { + throw new IllegalArgumentException("AWS Access Key ID and Secret Access Key must be specified as the " + + "username or password (respectively) of a " + scheme + " URL, or by setting the " + + accessKeyProperty + " or " + secretAccessKeyProperty + " properties (respectively)."); + } else if (accessKey == null) { + throw new IllegalArgumentException("AWS Access Key ID must be specified as the username of a " + scheme + + " URL, or by setting the " + accessKeyProperty + " property."); + } else if (secretAccessKey == null) { + throw new IllegalArgumentException("AWS Secret Access Key must be specified as the password of a " + + scheme + " URL, or by setting the " + secretAccessKeyProperty + " property."); + } + + } + + public String getAccessKey() { + return accessKey; + } + + public String getSecretAccessKey() { + return secretAccessKey; + } +} \ No newline at end of file diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java new file mode 100644 index 0000000000..e85514f199 --- /dev/null +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java @@ -0,0 +1,598 @@ +/** + * 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.tajo.storage.s3; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.AmazonServiceException; +import com.amazonaws.AmazonWebServiceRequest; +import com.amazonaws.HttpMethod; +import com.amazonaws.regions.Region; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.S3ClientOptions; +import com.amazonaws.services.s3.S3ResponseMetadata; +import com.amazonaws.services.s3.model.*; + +import java.io.File; +import java.io.InputStream; +import java.net.URL; +import java.util.Date; +import java.util.List; + +import static org.apache.http.HttpStatus.SC_OK; + +public class MockAmazonS3 implements AmazonS3 { + private int getObjectHttpCode = SC_OK; + private int getObjectMetadataHttpCode = SC_OK; + + public void setGetObjectHttpErrorCode(int getObjectHttpErrorCode) { + this.getObjectHttpCode = getObjectHttpErrorCode; + } + + public void setGetObjectMetadataHttpCode(int getObjectMetadataHttpCode) { + this.getObjectMetadataHttpCode = getObjectMetadataHttpCode; + } + + @Override + public void setEndpoint(String endpoint) { + } + + @Override + public void setRegion(Region region) + throws IllegalArgumentException { + } + + @Override + public void setS3ClientOptions(S3ClientOptions clientOptions) { + } + + @Override + public void changeObjectStorageClass(String bucketName, String key, StorageClass newStorageClass) + throws AmazonClientException { + } + + @Override + public void setObjectRedirectLocation(String bucketName, String key, String newRedirectLocation) + throws AmazonClientException { + } + + @Override + public ObjectListing listObjects(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public ObjectListing listObjects(String bucketName, String prefix) + throws AmazonClientException { + return null; + } + + @Override + public ObjectListing listObjects(ListObjectsRequest listObjectsRequest) + throws AmazonClientException { + return null; + } + + @Override + public ObjectListing listNextBatchOfObjects(ObjectListing previousObjectListing) + throws AmazonClientException { + return null; + } + + @Override + public VersionListing listVersions(String bucketName, String prefix) + throws AmazonClientException { + return null; + } + + @Override + public VersionListing listNextBatchOfVersions(VersionListing previousVersionListing) + throws AmazonClientException { + return null; + } + + @Override + public VersionListing listVersions(String bucketName, String prefix, String keyMarker, String versionIdMarker, + String delimiter, Integer maxResults) throws AmazonClientException { + return null; + } + + @Override + public VersionListing listVersions(ListVersionsRequest listVersionsRequest) + throws AmazonClientException { + return null; + } + + @Override + public Owner getS3AccountOwner() + throws AmazonClientException { + return null; + } + + @Override + public boolean doesBucketExist(String bucketName) + throws AmazonClientException { + return false; + } + + @Override + public List listBuckets() + throws AmazonClientException { + return null; + } + + @Override + public List listBuckets(ListBucketsRequest listBucketsRequest) + throws AmazonClientException { + return null; + } + + @Override + public String getBucketLocation(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public String getBucketLocation(GetBucketLocationRequest getBucketLocationRequest) + throws AmazonClientException { + return null; + } + + @Override + public Bucket createBucket(CreateBucketRequest createBucketRequest) + throws AmazonClientException { + return null; + } + + @Override + public Bucket createBucket(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public Bucket createBucket(String bucketName, com.amazonaws.services.s3.model.Region region) + throws AmazonClientException { + return null; + } + + @Override + public Bucket createBucket(String bucketName, String region) + throws AmazonClientException { + return null; + } + + @Override + public AccessControlList getObjectAcl(String bucketName, String key) + throws AmazonClientException { + return null; + } + + @Override + public AccessControlList getObjectAcl(String bucketName, String key, String versionId) + throws AmazonClientException { + return null; + } + + @Override + public void setObjectAcl(String bucketName, String key, AccessControlList acl) + throws AmazonClientException { + } + + @Override + public void setObjectAcl(String bucketName, String key, CannedAccessControlList acl) + throws AmazonClientException { + } + + @Override + public void setObjectAcl(String bucketName, String key, String versionId, AccessControlList acl) + throws AmazonClientException { + } + + @Override + public void setObjectAcl(String bucketName, String key, String versionId, CannedAccessControlList acl) + throws AmazonClientException { + } + + @Override + public AccessControlList getBucketAcl(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public void setBucketAcl(SetBucketAclRequest setBucketAclRequest) + throws AmazonClientException { + } + + @Override + public AccessControlList getBucketAcl(GetBucketAclRequest getBucketAclRequest) + throws AmazonClientException { + return null; + } + + @Override + public void setBucketAcl(String bucketName, AccessControlList acl) + throws AmazonClientException { + } + + @Override + public void setBucketAcl(String bucketName, CannedAccessControlList acl) + throws AmazonClientException { + } + + @Override + public ObjectMetadata getObjectMetadata(String bucketName, String key) + throws AmazonClientException { + if (getObjectMetadataHttpCode != SC_OK) { + AmazonS3Exception exception = new AmazonS3Exception("Failing getObjectMetadata call with " + + getObjectMetadataHttpCode); + exception.setStatusCode(getObjectMetadataHttpCode); + throw exception; + } + return null; + } + + @Override + public ObjectMetadata getObjectMetadata(GetObjectMetadataRequest getObjectMetadataRequest) + throws AmazonClientException { + return null; + } + + @Override + public S3Object getObject(String bucketName, String key) + throws AmazonClientException { + return null; + } + + @Override + public S3Object getObject(GetObjectRequest getObjectRequest) + throws AmazonClientException { + if (getObjectHttpCode != SC_OK) { + AmazonS3Exception exception = new AmazonS3Exception("Failing getObject call with " + getObjectHttpCode); + exception.setStatusCode(getObjectHttpCode); + throw exception; + } + return null; + } + + @Override + public ObjectMetadata getObject(GetObjectRequest getObjectRequest, File destinationFile) + throws AmazonClientException { + return null; + } + + @Override + public void deleteBucket(DeleteBucketRequest deleteBucketRequest) + throws AmazonClientException { + } + + @Override + public void deleteBucket(String bucketName) + throws AmazonClientException { + } + + @Override + public PutObjectResult putObject(PutObjectRequest putObjectRequest) + throws AmazonClientException { + return null; + } + + @Override + public PutObjectResult putObject(String bucketName, String key, File file) + throws AmazonClientException { + return null; + } + + @Override + public PutObjectResult putObject(String bucketName, String key, InputStream input, ObjectMetadata metadata) + throws AmazonClientException { + return null; + } + + @Override + public CopyObjectResult copyObject(String sourceBucketName, String sourceKey, String destinationBucketName, + String destinationKey) throws AmazonClientException { + return null; + } + + @Override + public CopyObjectResult copyObject(CopyObjectRequest copyObjectRequest) + throws AmazonClientException { + return null; + } + + @Override + public CopyPartResult copyPart(CopyPartRequest copyPartRequest) + throws AmazonClientException { + return null; + } + + @Override + public void deleteObject(String bucketName, String key) + throws AmazonClientException { + } + + @Override + public void deleteObject(DeleteObjectRequest deleteObjectRequest) + throws AmazonClientException { + } + + @Override + public DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteObjectsRequest) + throws AmazonClientException { + return null; + } + + @Override + public void deleteVersion(String bucketName, String key, String versionId) + throws AmazonClientException { + } + + @Override + public void deleteVersion(DeleteVersionRequest deleteVersionRequest) + throws AmazonClientException { + } + + @Override + public BucketLoggingConfiguration getBucketLoggingConfiguration(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public void setBucketLoggingConfiguration(SetBucketLoggingConfigurationRequest setBucketLoggingConfigurationRequest) + throws AmazonClientException { + } + + @Override + public BucketVersioningConfiguration getBucketVersioningConfiguration(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public void setBucketVersioningConfiguration(SetBucketVersioningConfigurationRequest + setBucketVersioningConfigurationRequest) throws AmazonClientException { + } + + @Override + public BucketLifecycleConfiguration getBucketLifecycleConfiguration(String bucketName) { + return null; + } + + @Override + public void setBucketLifecycleConfiguration(String bucketName, + BucketLifecycleConfiguration bucketLifecycleConfiguration) { + } + + @Override + public void setBucketLifecycleConfiguration(SetBucketLifecycleConfigurationRequest + setBucketLifecycleConfigurationRequest) { + } + + @Override + public void deleteBucketLifecycleConfiguration(String bucketName) { + } + + @Override + public void deleteBucketLifecycleConfiguration(DeleteBucketLifecycleConfigurationRequest + deleteBucketLifecycleConfigurationRequest) { + } + + @Override + public BucketCrossOriginConfiguration getBucketCrossOriginConfiguration(String bucketName) { + return null; + } + + @Override + public void setBucketCrossOriginConfiguration(String bucketName, BucketCrossOriginConfiguration + bucketCrossOriginConfiguration) { + } + + @Override + public void setBucketCrossOriginConfiguration(SetBucketCrossOriginConfigurationRequest + setBucketCrossOriginConfigurationRequest) { + } + + @Override + public void deleteBucketCrossOriginConfiguration(String bucketName) { + } + + @Override + public void deleteBucketCrossOriginConfiguration(DeleteBucketCrossOriginConfigurationRequest + deleteBucketCrossOriginConfigurationRequest) { + } + + @Override + public BucketTaggingConfiguration getBucketTaggingConfiguration(String bucketName) { + return null; + } + + @Override + public void setBucketTaggingConfiguration(String bucketName, BucketTaggingConfiguration bucketTaggingConfiguration) { + } + + @Override + public void setBucketTaggingConfiguration(SetBucketTaggingConfigurationRequest setBucketTaggingConfigurationRequest) { + } + + @Override + public void deleteBucketTaggingConfiguration(String bucketName) { + } + + @Override + public void deleteBucketTaggingConfiguration(DeleteBucketTaggingConfigurationRequest + deleteBucketTaggingConfigurationRequest) { + } + + @Override + public BucketNotificationConfiguration getBucketNotificationConfiguration(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public void setBucketNotificationConfiguration(SetBucketNotificationConfigurationRequest + setBucketNotificationConfigurationRequest) + throws AmazonClientException { + } + + @Override + public void setBucketNotificationConfiguration(String bucketName, BucketNotificationConfiguration + bucketNotificationConfiguration) + throws AmazonClientException { + } + + @Override + public BucketWebsiteConfiguration getBucketWebsiteConfiguration(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public BucketWebsiteConfiguration getBucketWebsiteConfiguration(GetBucketWebsiteConfigurationRequest + getBucketWebsiteConfigurationRequest) + throws AmazonClientException { + return null; + } + + @Override + public void setBucketWebsiteConfiguration(String bucketName, BucketWebsiteConfiguration configuration) + throws AmazonClientException { + } + + @Override + public void setBucketWebsiteConfiguration(SetBucketWebsiteConfigurationRequest setBucketWebsiteConfigurationRequest) + throws AmazonClientException { + } + + @Override + public void deleteBucketWebsiteConfiguration(String bucketName) + throws AmazonClientException { + } + + @Override + public void deleteBucketWebsiteConfiguration(DeleteBucketWebsiteConfigurationRequest + deleteBucketWebsiteConfigurationRequest) + throws AmazonClientException { + } + + @Override + public BucketPolicy getBucketPolicy(String bucketName) + throws AmazonClientException { + return null; + } + + @Override + public BucketPolicy getBucketPolicy(GetBucketPolicyRequest getBucketPolicyRequest) + throws AmazonClientException { + return null; + } + + @Override + public void setBucketPolicy(String bucketName, String policyText) + throws AmazonClientException { + } + + @Override + public void setBucketPolicy(SetBucketPolicyRequest setBucketPolicyRequest) + throws AmazonClientException { + } + + @Override + public void deleteBucketPolicy(String bucketName) + throws AmazonClientException { + } + + @Override + public void deleteBucketPolicy(DeleteBucketPolicyRequest deleteBucketPolicyRequest) + throws AmazonClientException { + } + + @Override + public URL generatePresignedUrl(String bucketName, String key, Date expiration) + throws AmazonClientException { + return null; + } + + @Override + public URL generatePresignedUrl(String bucketName, String key, Date expiration, HttpMethod method) + throws AmazonClientException { + return null; + } + + @Override + public URL generatePresignedUrl(GeneratePresignedUrlRequest generatePresignedUrlRequest) + throws AmazonClientException { + return null; + } + + @Override + public InitiateMultipartUploadResult initiateMultipartUpload(InitiateMultipartUploadRequest request) + throws AmazonClientException { + return null; + } + + @Override + public UploadPartResult uploadPart(UploadPartRequest request) + throws AmazonClientException { + return null; + } + + @Override + public PartListing listParts(ListPartsRequest request) + throws AmazonClientException { + return null; + } + + @Override + public void abortMultipartUpload(AbortMultipartUploadRequest request) + throws AmazonClientException { + } + + @Override + public CompleteMultipartUploadResult completeMultipartUpload(CompleteMultipartUploadRequest request) + throws AmazonClientException { + return null; + } + + @Override + public MultipartUploadListing listMultipartUploads(ListMultipartUploadsRequest request) + throws AmazonClientException { + return null; + } + + @Override + public S3ResponseMetadata getCachedResponseMetadata(AmazonWebServiceRequest request) { + return null; + } + + @Override + public void restoreObject(RestoreObjectRequest request) + throws AmazonServiceException { + } + + @Override + public void restoreObject(String bucketName, String key, int expirationInDays) + throws AmazonServiceException { + } + +} \ No newline at end of file diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java index 2d0677885c..d6f4ae7b91 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java @@ -18,7 +18,14 @@ package org.apache.tajo.storage.s3; +import com.amazonaws.AmazonClientException; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.InstanceProfileCredentialsProvider; +import com.amazonaws.services.s3.AmazonS3; +import com.amazonaws.services.s3.AmazonS3Client; +import com.google.common.base.Throwables; import net.minidev.json.JSONObject; +import org.apache.hadoop.fs.Path; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.storage.TablespaceManager; import org.junit.AfterClass; @@ -26,10 +33,11 @@ import org.junit.Test; import java.io.IOException; +import java.lang.reflect.Field; import java.net.URI; -import static org.junit.Assert.assertEquals; -import static org.junit.Assert.assertTrue; +import static com.google.common.base.Preconditions.checkArgument; +import static org.junit.Assert.*; public class TestS3TableSpace { public static final String SPACENAME = "s3_cluster"; @@ -41,7 +49,10 @@ public static void setUp() throws Exception { TajoConf tajoConf = new TajoConf(); tajoConf.set("fs.s3.impl", MockS3FileSystem.class.getName()); + tajoConf.set("fs.s3a.access.key", "test_access_key_id"); + tajoConf.set("fs.s3a.secret.key", "test_secret_access_key"); tablespace.init(tajoConf); + tablespace.setAmazonS3Client(new MockAmazonS3()); TablespaceManager.addTableSpaceForTest(tablespace); } @@ -59,4 +70,43 @@ public void testTablespaceHandler() throws Exception { assertTrue((TablespaceManager.get(S3_URI)) instanceof S3TableSpace); assertEquals(S3_URI, TablespaceManager.get(S3_URI).getUri().toASCIIString()); } + + @Test + public void testInstanceCredentialsEnabled() throws Exception { + assertTrue((TablespaceManager.getByName(SPACENAME)) instanceof S3TableSpace); + S3TableSpace tableSpace = TablespaceManager.getByName(SPACENAME); + + assertNotNull(tableSpace.getAmazonS3Client()); + assertTrue((tableSpace.getAmazonS3Client()) instanceof AmazonS3Client); + + assertTrue(getAwsCredentialsProvider(tableSpace.getAmazonS3Client()) + instanceof InstanceProfileCredentialsProvider); + } + + private AWSCredentialsProvider getAwsCredentialsProvider(AmazonS3 s3) { + return getFieldValue(s3, "awsCredentialsProvider", AWSCredentialsProvider.class); + } + + @SuppressWarnings("unchecked") + private T getFieldValue(Object instance, String name, Class type) { + try { + Field field = instance.getClass().getDeclaredField(name); + checkArgument(field.getType() == type, "expected %s but found %s", type, field.getType()); + field.setAccessible(true); + return (T) field.get(instance); + } + catch (ReflectiveOperationException e) { + throw Throwables.propagate(e); + } + } + + @Test(expected = AmazonClientException.class) + public void testCalculateSize() throws Exception { + assertTrue((TablespaceManager.getByName(SPACENAME)) instanceof S3TableSpace); + S3TableSpace tableSpace = TablespaceManager.getByName(SPACENAME); + + assertNotNull(tableSpace.getAmazonS3Client()); + assertTrue((tableSpace.getAmazonS3Client()) instanceof AmazonS3Client); + tableSpace.calculateSize(new Path("s3n://test-bucket/test")); + } } From 735c773bdc724aa8e12c8c9a5b0cc9c15a58eeb5 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Fri, 20 May 2016 17:43:40 +0900 Subject: [PATCH 02/27] Use configurations of S3AFileSystem --- .../java/org/apache/tajo/conf/TajoConf.java | 9 ---- .../apache/tajo/storage/s3/S3TableSpace.java | 26 ++++----- .../tajo/storage/s3/TajoS3Constants.java | 54 +++++++++++++++++++ .../tajo/storage/s3/TajoS3Credentials.java | 10 ++-- 4 files changed, 70 insertions(+), 29 deletions(-) create mode 100644 tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java diff --git a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java index 0618b8dc57..6e3eaeae8a 100644 --- a/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java +++ b/tajo-common/src/main/java/org/apache/tajo/conf/TajoConf.java @@ -232,15 +232,6 @@ public static enum ConfVars implements ConfigKey { // for RCFile HIVEUSEEXPLICITRCFILEHEADER("tajo.exec.rcfile.use.explicit.header", true, Validators.bool()), - // S3 Configuration -------------------------------------------------- - S3_MAX_ERROR_RETRIES("tajo.s3.max-error-retries", 10), - S3_SSL_ENABLED("tajo.s3.ssl.enabled", true), - S3_CONNECT_TIMEOUT("tajo.s3.connect-timeout", "5m"), - S3_SOCKET_TIMEOUT("tajo.s3.socket-timeout", "5m"), - S3_MAX_CONNECTIONS("tajo.s3.max-connections", 500), - S3_USE_INSTANCE_CREDENTIALS("tajo.s3.use-instance-credentials", true), - S3_PIN_CLIENT_TO_CURRENT_REGION("tajo.s3.pin-client-to-current-region", false), - // RPC -------------------------------------------------------------------- // Internal RPC Client INTERNAL_RPC_CLIENT_WORKER_THREAD_NUM("tajo.internal.rpc.client.worker-thread-num", diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index 9fa61cfbed..58dc2d5afe 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -48,14 +48,12 @@ import static com.google.common.base.Preconditions.checkArgument; import static com.google.common.base.Strings.nullToEmpty; +import static org.apache.tajo.storage.s3.TajoS3Constants.*; public class S3TableSpace extends FileTablespace { private final static Log LOG = LogFactory.getLog(S3TableSpace.class); private AmazonS3 s3; - private boolean useInstanceCredentials; - //use a custom endpoint? - public static final String ENDPOINT = "fs.s3a.endpoint"; public S3TableSpace(String spaceName, URI uri, JSONObject config) { super(spaceName, uri, config); @@ -65,20 +63,18 @@ public S3TableSpace(String spaceName, URI uri, JSONObject config) { public void init(TajoConf tajoConf) throws IOException { super.init(tajoConf); - int maxErrorRetries = conf.getIntVar(TajoConf.ConfVars.S3_MAX_ERROR_RETRIES); - boolean sslEnabled = conf.getBoolVar(TajoConf.ConfVars.S3_SSL_ENABLED); + int maxErrorRetries = conf.getInt(MAX_ERROR_RETRIES, DEFAULT_MAX_ERROR_RETRIES); + boolean sslEnabled = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); - Duration connectTimeout = Duration.valueOf(conf.getVar(TajoConf.ConfVars.S3_CONNECT_TIMEOUT)); - Duration socketTimeout = Duration.valueOf(conf.getVar(TajoConf.ConfVars.S3_SOCKET_TIMEOUT)); - int maxConnections = conf.getIntVar(TajoConf.ConfVars.S3_MAX_CONNECTIONS); - - this.useInstanceCredentials = conf.getBoolVar(TajoConf.ConfVars.S3_USE_INSTANCE_CREDENTIALS); + int connectTimeout = conf.getInt(ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT); + int socketTimeout = conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT); + int maxConnections = conf.getInt(MAXIMUM_CONNECTIONS, DEFAULT_MAXIMUM_CONNECTIONS); ClientConfiguration configuration = new ClientConfiguration() .withMaxErrorRetry(maxErrorRetries) .withProtocol(sslEnabled ? Protocol.HTTPS : Protocol.HTTP) - .withConnectionTimeout(Ints.checkedCast(connectTimeout.toMillis())) - .withSocketTimeout(Ints.checkedCast(socketTimeout.toMillis())) + .withConnectionTimeout(connectTimeout) + .withSocketTimeout(socketTimeout) .withMaxConnections(maxConnections); this.s3 = createAmazonS3Client(uri, conf, configuration); @@ -112,11 +108,7 @@ private AWSCredentialsProvider getAwsCredentialsProvider(URI uri, Configuration } catch (IllegalArgumentException ignored) { } - if (useInstanceCredentials) { - return new InstanceProfileCredentialsProvider(); - } - - throw new RuntimeException("S3 credentials not configured"); + return new InstanceProfileCredentialsProvider(); } private static AWSCredentials getAwsCredentials(URI uri, Configuration conf) { diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java new file mode 100644 index 0000000000..32fc575514 --- /dev/null +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java @@ -0,0 +1,54 @@ +/** + * 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.tajo.storage.s3; + +/** + * Borrow from org.apache.hadoop.fs.s3a.TajoS3Constants. + * + */ +public class TajoS3Constants { + // s3 access key + public static final String ACCESS_KEY = "fs.s3a.access.key"; + + // s3 secret key + public static final String SECRET_KEY = "fs.s3a.secret.key"; + + //use a custom endpoint? + public static final String ENDPOINT = "fs.s3a.endpoint"; + + // number of times we should retry errors + public static final String MAX_ERROR_RETRIES = "fs.s3a.attempts.maximum"; + public static final int DEFAULT_MAX_ERROR_RETRIES = 10; + + // connect to s3 over ssl? + public static final String SECURE_CONNECTIONS = "fs.s3a.connection.ssl.enabled"; + public static final boolean DEFAULT_SECURE_CONNECTIONS = true; + + // seconds until we give up trying to establish a connection to s3 + public static final String ESTABLISH_TIMEOUT = "fs.s3a.connection.establish.timeout"; + public static final int DEFAULT_ESTABLISH_TIMEOUT = 50000; + + // seconds until we give up on a connection to s3 + public static final String SOCKET_TIMEOUT = "fs.s3a.connection.timeout"; + public static final int DEFAULT_SOCKET_TIMEOUT = 50000; + + // number of simultaneous connections to s3 + public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; + public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15; +} diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java index 716d92260e..ed1d9dcc95 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java @@ -24,6 +24,9 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import static org.apache.tajo.storage.s3.TajoS3Constants.ACCESS_KEY; +import static org.apache.tajo.storage.s3.TajoS3Constants.SECRET_KEY; + /** *

* Extracts AWS credentials from the filesystem URI or configuration. (borrowed from hadoop-aws package) @@ -57,9 +60,10 @@ public void initialize(URI uri, Configuration conf) { } String scheme = uri.getScheme(); - String accessKeyProperty = String.format("fs.%s.awsAccessKeyId", scheme); - String secretAccessKeyProperty = - String.format("fs.%s.awsSecretAccessKey", scheme); + String accessKeyProperty = scheme.startsWith("s3a") ? ACCESS_KEY : String.format("fs.%s.awsAccessKeyId", scheme); + String secretAccessKeyProperty = scheme.startsWith("s3a") ? SECRET_KEY : String.format("fs.%s.awsSecretAccessKey", + scheme); + if (accessKey == null) { accessKey = conf.getTrimmed(accessKeyProperty); } From fe61e4053c912f5b83cf113a030d670ce66a15a6 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Fri, 20 May 2016 17:46:49 +0900 Subject: [PATCH 03/27] Remove unnecessary modifications --- .../apache/tajo/master/exec/DDLExecutor.java | 16 +++++------ .../org/apache/tajo/querymaster/Query.java | 28 +++++++++++-------- 2 files changed, 24 insertions(+), 20 deletions(-) diff --git a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java index 5d53de073a..c480d04d44 100644 --- a/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java +++ b/tajo-core/src/main/java/org/apache/tajo/master/exec/DDLExecutor.java @@ -20,6 +20,7 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -511,8 +512,8 @@ public void alterTable(TajoMaster.MasterContext context, final QueryContext quer long numBytes = 0L; if (fs.exists(partitionPath)) { - FileTablespace fileTablespace = TablespaceManager.get(partitionPath.toUri()); - numBytes = fileTablespace.calculateSize(partitionPath); + ContentSummary summary = fs.getContentSummary(partitionPath); + numBytes = summary.getLength(); } catalog.alterTable(CatalogUtil.addOrDropPartition(qualifiedName, alterTable.getPartitionColumns(), @@ -623,7 +624,6 @@ public void repairPartition(TajoMaster.MasterContext context, final QueryContext } // Find missing partitions from CatalogStore - Tablespace tablespace = TablespaceManager.get(tableDesc.getUri()); List targetPartitions = new ArrayList<>(); for(Path filteredPath : filteredPaths) { @@ -632,7 +632,7 @@ public void repairPartition(TajoMaster.MasterContext context, final QueryContext // if there is partition column in the path if (startIdx > -1) { - PartitionDescProto targetPartition = getPartitionDesc(tablespace, tablePath, filteredPath); + PartitionDescProto targetPartition = getPartitionDesc(tablePath, filteredPath, fs); if (!existingPartitionNames.contains(targetPartition.getPartitionName())) { if (LOG.isDebugEnabled()) { LOG.debug("Partitions not in CatalogStore:" + targetPartition.getPartitionName()); @@ -658,8 +658,7 @@ public void repairPartition(TajoMaster.MasterContext context, final QueryContext LOG.info("Total added partitions to CatalogStore: " + targetPartitions.size()); } - private PartitionDescProto getPartitionDesc(Tablespace tablespace, Path tablePath, Path partitionPath) - throws IOException { + private PartitionDescProto getPartitionDesc(Path tablePath, Path partitionPath, FileSystem fs) throws IOException { String partitionName = StringUtils.unescapePathName(partitionPath.toString()); int startIndex = partitionName.indexOf(tablePath.toString()) + tablePath.toString().length(); @@ -681,8 +680,9 @@ private PartitionDescProto getPartitionDesc(Tablespace tablespace, Path tablePat } builder.setPath(partitionPath.toString()); - FileTablespace fileTablespace = TablespaceManager.get(partitionPath.toUri()); - builder.setNumBytes(fileTablespace.calculateSize(partitionPath)); + + ContentSummary contentSummary = fs.getContentSummary(partitionPath); + builder.setNumBytes(contentSummary.getLength()); return builder.build(); } diff --git a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java index ba0af3a2c9..289d933bd1 100644 --- a/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java +++ b/tajo-core/src/main/java/org/apache/tajo/querymaster/Query.java @@ -23,6 +23,8 @@ import org.apache.commons.lang.exception.ExceptionUtils; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; import org.apache.hadoop.yarn.event.EventHandler; import org.apache.hadoop.yarn.state.*; @@ -48,7 +50,6 @@ import org.apache.tajo.master.event.*; import org.apache.tajo.plan.logical.*; import org.apache.tajo.plan.util.PlannerUtil; -import org.apache.tajo.storage.FileTablespace; import org.apache.tajo.schema.IdentifierUtil; import org.apache.tajo.storage.StorageConstants; import org.apache.tajo.storage.Tablespace; @@ -539,8 +540,8 @@ private QueryState finalizeQuery(Query query, QueryCompletedEvent event) { if (queryContext.hasOutputTableUri() && queryContext.hasPartition()) { List partitions = query.getPartitions(); if (partitions != null) { - // Find each partition volume by listing all partitions. - List finalPartitions = getPartitionsWithContentsSummary(queryContext, + // Set contents length and file count to PartitionDescProto by listing final output directories. + List finalPartitions = getPartitionsWithContentsSummary(query.systemConf, finalOutputDir, partitions); String databaseName, simpleTableName; @@ -571,14 +572,16 @@ private QueryState finalizeQuery(Query query, QueryCompletedEvent event) { return QueryState.QUERY_SUCCEEDED; } - private List getPartitionsWithContentsSummary(QueryContext queryContext, - Path outputDir, List partitions) throws IOException { + private List getPartitionsWithContentsSummary(TajoConf conf, Path outputDir, + List partitions) throws IOException { List finalPartitions = new ArrayList<>(); + FileSystem fileSystem = outputDir.getFileSystem(conf); for (PartitionDescProto partition : partitions) { PartitionDescProto.Builder builder = partition.toBuilder(); Path partitionPath = new Path(outputDir, partition.getPath()); - builder.setNumBytes(calculateSize(partitionPath)); + ContentSummary contentSummary = fileSystem.getContentSummary(partitionPath); + builder.setNumBytes(contentSummary.getLength()); finalPartitions.add(builder.build()); } return finalPartitions; @@ -685,7 +688,7 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo TableStats aggregated = query.aggregateTableStatsOfTerminalBlock(); resultTableDesc.setStats(aggregated); } else { - stats.setNumBytes(calculateSize(finalOutputDir)); + stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir)); resultTableDesc.setStats(stats); } @@ -728,7 +731,7 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo TableStats aggregated = query.aggregateTableStatsOfTerminalBlock(); tableDescTobeCreated.setStats(aggregated); } else { - stats.setNumBytes(calculateSize(finalOutputDir)); + stats.setNumBytes(getTableVolume(query.systemConf, finalOutputDir)); tableDescTobeCreated.setStats(stats); } @@ -771,7 +774,7 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo TableStats aggregated = query.aggregateTableStatsOfTerminalBlock(); finalTable.setStats(aggregated); } else { - long volume = calculateSize(finalOutputDir); + long volume = getTableVolume(query.systemConf, finalOutputDir); stats.setNumBytes(volume); finalTable.setStats(stats); } @@ -789,9 +792,10 @@ public void execute(QueryMaster.QueryMasterContext context, QueryContext queryCo } } - public static long calculateSize(Path path) throws IOException { - FileTablespace fileTablespace = TablespaceManager.get(path.toUri()); - return fileTablespace.calculateSize(path); + public static long getTableVolume(TajoConf systemConf, Path tablePath) throws IOException { + FileSystem fs = tablePath.getFileSystem(systemConf); + ContentSummary directorySummary = fs.getContentSummary(tablePath); + return directorySummary.getLength(); } public static class StageCompletedTransition implements SingleArcTransition { From 83bcfca2a2c5523201eef68df9474300e1b41d8c Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Fri, 20 May 2016 17:59:50 +0900 Subject: [PATCH 04/27] Remove unnecessary unit tests --- .../tajo/storage/s3/TestS3TableSpace.java | 33 ++----------------- 1 file changed, 2 insertions(+), 31 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java index d6f4ae7b91..1101557696 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java @@ -71,42 +71,13 @@ public void testTablespaceHandler() throws Exception { assertEquals(S3_URI, TablespaceManager.get(S3_URI).getUri().toASCIIString()); } - @Test - public void testInstanceCredentialsEnabled() throws Exception { - assertTrue((TablespaceManager.getByName(SPACENAME)) instanceof S3TableSpace); - S3TableSpace tableSpace = TablespaceManager.getByName(SPACENAME); - - assertNotNull(tableSpace.getAmazonS3Client()); - assertTrue((tableSpace.getAmazonS3Client()) instanceof AmazonS3Client); - - assertTrue(getAwsCredentialsProvider(tableSpace.getAmazonS3Client()) - instanceof InstanceProfileCredentialsProvider); - } - - private AWSCredentialsProvider getAwsCredentialsProvider(AmazonS3 s3) { - return getFieldValue(s3, "awsCredentialsProvider", AWSCredentialsProvider.class); - } - - @SuppressWarnings("unchecked") - private T getFieldValue(Object instance, String name, Class type) { - try { - Field field = instance.getClass().getDeclaredField(name); - checkArgument(field.getType() == type, "expected %s but found %s", type, field.getType()); - field.setAccessible(true); - return (T) field.get(instance); - } - catch (ReflectiveOperationException e) { - throw Throwables.propagate(e); - } - } - @Test(expected = AmazonClientException.class) public void testCalculateSize() throws Exception { + Path path = new Path(S3_URI, "/test"); assertTrue((TablespaceManager.getByName(SPACENAME)) instanceof S3TableSpace); - S3TableSpace tableSpace = TablespaceManager.getByName(SPACENAME); + S3TableSpace tableSpace = TablespaceManager.get(path.toUri()); assertNotNull(tableSpace.getAmazonS3Client()); - assertTrue((tableSpace.getAmazonS3Client()) instanceof AmazonS3Client); tableSpace.calculateSize(new Path("s3n://test-bucket/test")); } } From dedd49d0cdbf052d44424bb6e3c8d1522b77d158 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Sat, 21 May 2016 02:16:30 +0900 Subject: [PATCH 05/27] Clean up S3TableSpace --- .../apache/tajo/storage/s3/S3TableSpace.java | 69 +++++++++++++++++-- .../tajo/storage/s3/TajoS3Constants.java | 7 ++ 2 files changed, 70 insertions(+), 6 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index 58dc2d5afe..e66e271f77 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -18,6 +18,7 @@ package org.apache.tajo.storage.s3; +import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; import java.util.stream.Stream; @@ -33,6 +34,8 @@ import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3Client; import com.amazonaws.services.s3.iterable.S3Objects; +import com.amazonaws.services.s3.model.ListObjectsRequest; +import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.annotations.VisibleForTesting; import com.google.common.primitives.Ints; @@ -40,6 +43,8 @@ import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.tajo.conf.TajoConf; import org.apache.tajo.storage.FileTablespace; @@ -54,6 +59,7 @@ public class S3TableSpace extends FileTablespace { private final static Log LOG = LogFactory.getLog(S3TableSpace.class); private AmazonS3 s3; + private int maxKeys; public S3TableSpace(String spaceName, URI uri, JSONObject config) { super(spaceName, uri, config); @@ -63,6 +69,7 @@ public S3TableSpace(String spaceName, URI uri, JSONObject config) { public void init(TajoConf tajoConf) throws IOException { super.init(tajoConf); + maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS); int maxErrorRetries = conf.getInt(MAX_ERROR_RETRIES, DEFAULT_MAX_ERROR_RETRIES); boolean sslEnabled = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); @@ -120,17 +127,67 @@ private static AWSCredentials getAwsCredentials(URI uri, Configuration conf) { @Override public long calculateSize(Path path) throws IOException { String key = keyFromPath(path); - if (!key.isEmpty()) { - key += "/"; + + final FileStatus fileStatus = fs.getFileStatus(path); + long totalBucketSize = 0L; + + if (fileStatus.isDirectory()) { + if (!key.isEmpty()) { + key = key + "/"; + } + + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(uri.getHost()); + request.setPrefix(key); + request.setMaxKeys(maxKeys); + + if (LOG.isDebugEnabled()) { + LOG.debug("listStatus: doing listObjects for directory " + key); + } + + ObjectListing objects = s3.listObjects(request); + + while (true) { + for (S3ObjectSummary summary : objects.getObjectSummaries()) { + Path keyPath = keyToPath(summary.getKey()).makeQualified(uri, fs.getWorkingDirectory()); + + // Skip over keys that are ourselves and old S3N _$folder$ files + if (keyPath.equals(path) || summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring: " + keyPath); + } + continue; + } + + if (!objectRepresentsDirectory(summary.getKey(), summary.getSize())) { + totalBucketSize += summary.getSize(); + } + } + + if (objects.isTruncated()) { + if (LOG.isDebugEnabled()) { + LOG.debug("listStatus: list truncated - getting next batch"); + } + objects = s3.listNextBatchOfObjects(objects); + } else { + break; + } + } + } else { + return fileStatus.getLen(); } - Iterable objectSummaries = S3Objects.withPrefix(s3, uri.getHost(), key); - Stream objectStream = StreamSupport.stream(objectSummaries.spliterator(), false); - long totalBucketSize = objectStream.mapToLong(object -> object.getSize()).sum(); - objectStream.close(); return totalBucketSize; } + private boolean objectRepresentsDirectory(final String name, final long size) { + return !name.isEmpty() && name.charAt(name.length() - 1) == '/' && size == 0L; + } + + private Path keyToPath(String key) { + return new Path("/" + key); + } + private String keyFromPath(Path path) { checkArgument(path.isAbsolute(), "Path is not absolute: %s", path); diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java index 32fc575514..879059de18 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java @@ -51,4 +51,11 @@ public class TajoS3Constants { // number of simultaneous connections to s3 public static final String MAXIMUM_CONNECTIONS = "fs.s3a.connection.maximum"; public static final int DEFAULT_MAXIMUM_CONNECTIONS = 15; + + // number of records to get while paging through a directory listing + public static final String MAX_PAGING_KEYS = "fs.s3a.paging.maximum"; + public static final int DEFAULT_MAX_PAGING_KEYS = 5000; + + public static final String S3N_FOLDER_SUFFIX = "_$folder$"; + } From c15c0006d96c92b994981a2da21c14fa4b4ce9fc Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Sat, 21 May 2016 02:17:42 +0900 Subject: [PATCH 06/27] Remove unused packages --- .../main/java/org/apache/tajo/storage/s3/S3TableSpace.java | 7 ------- 1 file changed, 7 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index e66e271f77..9b563e8545 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -18,11 +18,8 @@ package org.apache.tajo.storage.s3; -import java.io.FileNotFoundException; import java.io.IOException; import java.net.URI; -import java.util.stream.Stream; -import java.util.stream.StreamSupport; import com.amazonaws.ClientConfiguration; import com.amazonaws.Protocol; @@ -33,17 +30,13 @@ import com.amazonaws.internal.StaticCredentialsProvider; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3Client; -import com.amazonaws.services.s3.iterable.S3Objects; import com.amazonaws.services.s3.model.ListObjectsRequest; import com.amazonaws.services.s3.model.ObjectListing; import com.amazonaws.services.s3.model.S3ObjectSummary; import com.google.common.annotations.VisibleForTesting; -import com.google.common.primitives.Ints; -import io.airlift.units.Duration; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.tajo.conf.TajoConf; From 30dd01bc2adc1d07530049991e9ed0175752e085 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 11:59:21 +0900 Subject: [PATCH 07/27] Implement normal unit test cases --- .../apache/tajo/storage/s3/MockAmazonS3.java | 7 ++- .../tajo/storage/s3/MockObjectListing.java | 48 +++++++++++++++++++ .../tajo/storage/s3/MockS3FileSystem.java | 6 ++- .../tajo/storage/s3/TestS3TableSpace.java | 19 ++------ 4 files changed, 64 insertions(+), 16 deletions(-) create mode 100644 tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockObjectListing.java diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java index e85514f199..422c4d4f9f 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java @@ -86,7 +86,12 @@ public ObjectListing listObjects(String bucketName, String prefix) @Override public ObjectListing listObjects(ListObjectsRequest listObjectsRequest) throws AmazonClientException { - return null; + if (listObjectsRequest.getBucketName().equals("tajo-test") && listObjectsRequest.getPrefix().equals("test/")) { + MockObjectListing objectListing = new MockObjectListing(); + return objectListing; + } else { + return null; + } } @Override diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockObjectListing.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockObjectListing.java new file mode 100644 index 0000000000..96a7f809df --- /dev/null +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockObjectListing.java @@ -0,0 +1,48 @@ +/** + * 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.tajo.storage.s3; + +import com.amazonaws.services.s3.model.ObjectListing; +import com.amazonaws.services.s3.model.S3ObjectSummary; +import com.google.common.collect.Lists; + +import java.util.List; + +public class MockObjectListing extends ObjectListing { + + @Override + public List getObjectSummaries() { + final String bucketName = "tajo-test"; + + List objectSummaries = Lists.newArrayList(); + objectSummaries.add(getS3ObjectSummary(bucketName, "test/data01", 10L)); + objectSummaries.add(getS3ObjectSummary(bucketName, "test/data02", 10L)); + objectSummaries.add(getS3ObjectSummary(bucketName, "test/data03", 10L)); + + return objectSummaries; + } + + private S3ObjectSummary getS3ObjectSummary(String bucketName, String key, long size) { + S3ObjectSummary objectSummary = new S3ObjectSummary(); + objectSummary.setBucketName(bucketName); + objectSummary.setKey(key); + objectSummary.setSize(size); + return objectSummary; + } +} diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java index 15c6a337f6..ea73e22b88 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java @@ -105,6 +105,10 @@ public boolean mkdirs(Path f, FsPermission permission) throws IOException { @Override public FileStatus getFileStatus(Path f) throws IOException { - return null; + if (f.equals(new Path(TestS3TableSpace.S3_URI, "test"))) { + return new FileStatus(0, true, 1, 0, 0, f); + } else { + return null; + } } } diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java index 1101557696..3ef7d5eb49 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java @@ -18,12 +18,6 @@ package org.apache.tajo.storage.s3; -import com.amazonaws.AmazonClientException; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.InstanceProfileCredentialsProvider; -import com.amazonaws.services.s3.AmazonS3; -import com.amazonaws.services.s3.AmazonS3Client; -import com.google.common.base.Throwables; import net.minidev.json.JSONObject; import org.apache.hadoop.fs.Path; import org.apache.tajo.conf.TajoConf; @@ -33,10 +27,8 @@ import org.junit.Test; import java.io.IOException; -import java.lang.reflect.Field; import java.net.URI; -import static com.google.common.base.Preconditions.checkArgument; import static org.junit.Assert.*; public class TestS3TableSpace { @@ -52,7 +44,6 @@ public static void setUp() throws Exception { tajoConf.set("fs.s3a.access.key", "test_access_key_id"); tajoConf.set("fs.s3a.secret.key", "test_secret_access_key"); tablespace.init(tajoConf); - tablespace.setAmazonS3Client(new MockAmazonS3()); TablespaceManager.addTableSpaceForTest(tablespace); } @@ -71,13 +62,13 @@ public void testTablespaceHandler() throws Exception { assertEquals(S3_URI, TablespaceManager.get(S3_URI).getUri().toASCIIString()); } - @Test(expected = AmazonClientException.class) + @Test public void testCalculateSize() throws Exception { - Path path = new Path(S3_URI, "/test"); + Path path = new Path(S3_URI, "test"); assertTrue((TablespaceManager.getByName(SPACENAME)) instanceof S3TableSpace); S3TableSpace tableSpace = TablespaceManager.get(path.toUri()); - - assertNotNull(tableSpace.getAmazonS3Client()); - tableSpace.calculateSize(new Path("s3n://test-bucket/test")); + tableSpace.setAmazonS3Client(new MockAmazonS3()); + long size = tableSpace.calculateSize(path); + assertEquals(30L, size); } } From 5c8d99aa48fe04373ed1a71be0aa304690d7bcf6 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 12:00:16 +0900 Subject: [PATCH 08/27] Rmeove unused method --- .../main/java/org/apache/tajo/storage/s3/S3TableSpace.java | 5 ----- 1 file changed, 5 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index 9b563e8545..d1baed9164 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -194,11 +194,6 @@ private String keyFromPath(Path path) return key; } - @VisibleForTesting - public AmazonS3 getAmazonS3Client() { - return s3; - } - @VisibleForTesting public void setAmazonS3Client(AmazonS3 s3) { this.s3 = s3; From 0e7da2c7a554e20f4cc94875bd42138d195b657b Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 15:40:57 +0900 Subject: [PATCH 09/27] Clean up pom files --- tajo-dist/pom.xml | 5 +- tajo-project/pom.xml | 5 + tajo-storage/tajo-storage-s3/pom.xml | 163 +-------------------------- 3 files changed, 7 insertions(+), 166 deletions(-) diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml index 5afe67a924..80999dbd92 100644 --- a/tajo-dist/pom.xml +++ b/tajo-dist/pom.xml @@ -146,7 +146,6 @@ run cp -r $ROOT/tajo-sql-parser/target/tajo-sql-parser-${project.version}/* . run cp -r $ROOT/tajo-storage/tajo-storage-jdbc/target/tajo-storage-jdbc-${project.version}.jar . run cp -r $ROOT/tajo-storage/tajo-storage-pgsql/target/tajo-storage-pgsql-${project.version}.jar . - run cp -r $ROOT/tajo-storage/tajo-storage-s3/target/tajo-storage-s3-${project.version}.jar . run cp -r $ROOT/tajo-pullserver/target/tajo-pullserver-${project.version}.jar . run cp -r $ROOT/tajo-metrics/target/tajo-metrics-${project.version}.jar . run cp -r $ROOT/tajo-core/target/tajo-core-${project.version}.jar . @@ -158,9 +157,7 @@ run mkdir -p lib run cp -r $ROOT/tajo-storage/tajo-storage-hdfs/target/lib/hive-*.jar lib/ - run mkdir aws_s3 - run cp -r $ROOT/tajo-storage/tajo-storage-s3/target/lib/*.jar aws_s3/ - + run cp -r $ROOT/tajo-storage/tajo-storage-s3/target/lib/*.jar lib/ run mkdir -p share/jdbc-dist run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml index 283bdc3dd5..416e26f13d 100644 --- a/tajo-project/pom.xml +++ b/tajo-project/pom.xml @@ -803,6 +803,11 @@ tajo-storage-hdfs ${project.version} + + org.apache.tajo + tajo-storage-s3 + ${project.version} + org.apache.tajo tajo-storage-hbase diff --git a/tajo-storage/tajo-storage-s3/pom.xml b/tajo-storage/tajo-storage-s3/pom.xml index 4af62e0062..ef802e6948 100644 --- a/tajo-storage/tajo-storage-s3/pom.xml +++ b/tajo-storage/tajo-storage-s3/pom.xml @@ -34,7 +34,6 @@ UTF-8 UTF-8 - 0.122 1.7.4 2.8.2 11.0.2 @@ -97,12 +96,8 @@ org.apache.maven.plugins - maven-surefire-plugin - - true - + maven-surefire-report-plugin - org.apache.maven.plugins maven-dependency-plugin @@ -293,20 +288,6 @@ org.apache.tajo tajo-catalog-common test - - - org.apache.tajo - tajo-common - - - org.apache.hadoop - hadoop-common - - - com.google.protobuf - protobuf-java - - @@ -353,113 +334,10 @@ - org.apache.hadoop hadoop-hdfs test - - - org.apache.hadoop - hadoop-annotations - - - org.apache.hadoop - hadoop-auth - - - org.apache.hadoop - hadoop-common - - - com.google.guava - guava - - - org.mortbay.jetty - jetty - - - org.mortbay.jetty - jetty-util - - - com.sun.jersey - jersey-core - - - com.sun.jersey - jersey-server - - - commons-cli - commons-cli - - - commons-codec - commons-codec - - - commons-io - commons-io - - - commons-lang - commons-lang - - - commons-logging - commons-logging - - - commons-daemon - commons-daemon - - - log4j - log4j - - - com.google.protobuf - protobuf-java - - - javax.servlet - servlet-api - - - org.slf4j - slf4j-log4j12 - - - org.codehaus.jackson - jackson-core-asl - - - org.codehaus.jackson - jackson-mapper-asl - - - xmlenc - xmlenc - - - io.netty - netty-all - - - xerces - xercesImpl - - - org.apache.htrace - htrace-core - - - org.fusesource.leveldbjni - leveldbjni-all - - @@ -487,26 +365,6 @@ test - - io.airlift - units - ${airlft.version} - - - javax.validation - validation-api - - - com.google.guava - guava - - - com.fasterxml.jackson.core - jackson-annotations - - - - org.weakref jmxutils @@ -628,28 +486,9 @@ validation-api 1.1.0.Final - - - - test-storage-s3 - - - - org.apache.maven.plugins - maven-surefire-plugin - - - TRUE - - -Xms128m -Xmx1024m -Dfile.encoding=UTF-8 - - - - - docs From 3cd12ac60d0461ff8087fb801a4cfa11b5bee856 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 15:41:58 +0900 Subject: [PATCH 10/27] Remove unnecessary codes --- tajo-dist/src/main/bin/tajo | 11 ----------- 1 file changed, 11 deletions(-) diff --git a/tajo-dist/src/main/bin/tajo b/tajo-dist/src/main/bin/tajo index 30b2f91b75..8dffe9ac29 100755 --- a/tajo-dist/src/main/bin/tajo +++ b/tajo-dist/src/main/bin/tajo @@ -336,17 +336,6 @@ fi # Hive Home Configuration End ############################################################################## -############################################################################## -# Find and Set AWS S3 CLASSPATH -############################################################################## - -AWS_S3_LIB=$TAJO_HOME/aws_s3 - -if [ -d ${AWS_S3_LIB} ]; then - for f in ${AWS_S3_LIB}/*.jar; do - CLASSPATH=${CLASSPATH}:$f; - done -fi ############################################################################## # Find and Set Tajo CLASSPATH From 8d47723e267e962a793b8bd7f43870f45047edf4 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 17:03:32 +0900 Subject: [PATCH 11/27] Clean up dependencies of s3 module --- tajo-dist/pom.xml | 2 - tajo-project/pom.xml | 1 + tajo-storage/tajo-storage-s3/pom.xml | 353 +-------------------------- 3 files changed, 4 insertions(+), 352 deletions(-) diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml index 80999dbd92..beafdf23ca 100644 --- a/tajo-dist/pom.xml +++ b/tajo-dist/pom.xml @@ -157,8 +157,6 @@ run mkdir -p lib run cp -r $ROOT/tajo-storage/tajo-storage-hdfs/target/lib/hive-*.jar lib/ - run cp -r $ROOT/tajo-storage/tajo-storage-s3/target/lib/*.jar lib/ - run mkdir -p share/jdbc-dist run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml index 416e26f13d..9def61986a 100644 --- a/tajo-project/pom.xml +++ b/tajo-project/pom.xml @@ -40,6 +40,7 @@ 2.6 6.1.26 1.8.1 + 1.7.4 ${project.parent.relativePath}/.. src/main/hadoop-${hadoop.version} diff --git a/tajo-storage/tajo-storage-s3/pom.xml b/tajo-storage/tajo-storage-s3/pom.xml index ef802e6948..5e4cde47ce 100644 --- a/tajo-storage/tajo-storage-s3/pom.xml +++ b/tajo-storage/tajo-storage-s3/pom.xml @@ -34,10 +34,6 @@ UTF-8 UTF-8 - 1.7.4 - 2.8.2 - 11.0.2 - 2.2.3 @@ -65,13 +61,6 @@ org.apache.rat apache-rat-plugin - - - src/test/resources/dataset/** - src/test/resources/queries/** - src/test/resources/results/** - - verify @@ -126,366 +115,30 @@ org.apache.tajo tajo-common provided - - - org.apache.hadoop - hadoop-common - - - com.google.protobuf - protobuf-java - - - commons-logging - commons-logging - - - commons-logging - commons-logging-api - - - commons-lang - commons-lang - - - com.google.guava - guava - - - com.google.code.gson - gson - - - io.netty - netty-buffer - - - org.iq80.snappy - snappy - - org.apache.tajo tajo-storage-common provided - - - org.apache.tajo - tajo-common - - - org.apache.tajo - tajo-catalog-common - - - org.apache.tajo - tajo-plan - - - org.apache.hadoop - hadoop-common - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - com.google.protobuf - protobuf-java - - - io.netty - netty-buffer - - org.apache.tajo tajo-storage-hdfs provided - - - io.netty - netty-transport - - - io.netty - netty-codec - - - io.netty - netty-codec-http - - - org.apache.tajo - tajo-common - - - org.apache.tajo - tajo-catalog-common - - - org.apache.tajo - tajo-plan - - - org.apache.tajo - tajo-storage-common - - - org.apache.avro - trevni-core - - - org.apache.avro - trevni-avro - - - org.apache.hadoop - hadoop-common - - - org.apache.hadoop - hadoop-hdfs - - - org.apache.hadoop - hadoop-mapreduce-client-core - - - com.google.protobuf - protobuf-java - - - com.twitter - parquet-column - - - com.twitter - parquet-hadoop - - - com.twitter - parquet-format - - - io.netty - netty-buffer - - - com.facebook.presto - presto-orc - - - - - - org.apache.tajo - tajo-catalog-common - test - - - - org.apache.hadoop - hadoop-common - provided - - - com.google.code.gson - gson - - - com.jcraft - jsch - - - org.apache.curator - curator-client - - - org.apache.curator - curator-recipes - - - org.apache.htrace - htrace-core - - - zookeeper - org.apache.zookeeper - - - org.apache.commons - commons-compress - - - org.apache.ant - ant - - - org.apache.commons - commons-compress - - - - - - org.apache.hadoop - hadoop-hdfs - test - - - - org.apache.tajo - tajo-plan - test - - - - com.fasterxml.jackson.core - jackson-databind - test - ${jackson2.version} - - - com.fasterxml.jackson.core - jackson-annotations - test - ${jackson2.version} - - - - junit - junit - test - - - - org.weakref - jmxutils - 1.18 com.amazonaws aws-java-sdk ${aws-java-sdk.version} - - - commons-logging - commons-logging - - - org.apache.httpcomponents - httpclient - - - joda-time - joda-time - - - javax.mail - mail - - - org.freemarker - freemarker - - - org.springframework - spring-beans - - - org.springframework - spring-core - - - org.springframework - spring-context - - - org.springframework - spring-test - - - org.aspectj - aspectjrt - - - junit - junit - - - com.fasterxml.jackson.core - jackson-annotations - - - com.fasterxml.jackson.core - jackson-core - - - com.fasterxml.jackson.core - jackson-databind - - - - - - org.apache.httpcomponents - httpclient - 4.5 - - - org.apache.httpcomponents - httpcore - - - commons-logging - commons-logging - - - commons-codec - commons-codec - - - - - - org.apache.httpcomponents - httpcore - 4.4.1 - - - junit - junit - - + provided - org.testng - testng - 6.9.6 + junit + junit test - - - joda-time - joda-time - ${joda-time.version} - compile - - - - javax.validation - validation-api - 1.1.0.Final - From 004d9b13432507517b3c05c86029d8b8b2fe921c Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 17:07:46 +0900 Subject: [PATCH 12/27] Remove unnecessary modification --- pom.xml | 607 +++++++++++++++++++------------------------------------- 1 file changed, 206 insertions(+), 401 deletions(-) diff --git a/pom.xml b/pom.xml index 71c062bbfa..7280e1f2fc 100644 --- a/pom.xml +++ b/pom.xml @@ -18,453 +18,258 @@ --> - 4.0.0 - org.apache - apache - 15 + tajo-project + org.apache.tajo + 0.12.0-SNAPSHOT + ../tajo-project - org.apache.tajo - tajo-main - 0.12.0-SNAPSHOT - Tajo Main - http://tajo.apache.org/ - Tajo Main - pom - - - - The Apache Software License, Version 2.0 - http://www.apache.org/licenses/LICENSE-2.0.txt - - - - - The Apache Software Foundation - http://www.apache.org/ - - - - Jira - https://issues.apache.org/jira/browse/TAJO - - - - Jenkins - https://builds.apache.org - - - - https://git-wip-us.apache.org/repos/asf/tajo.git - scm:git:https://git-wip-us.apache.org/repos/asf/tajo.git - scm:git:https://git-wip-us.apache.org/repos/asf/tajo.git - HEAD - - - - - eclipse-jetty - http://repo2.maven.org/maven2/org/eclipse/jetty/jetty-distribution/ - - false - - - - - - UTF-8 - UTF-8 - ${basedir} - - - - tajo-maven-plugins - tajo-project - tajo-common - tajo-algebra - tajo-plan - tajo-core - tajo-rpc - tajo-catalog - tajo-client - tajo-client-example - tajo-jdbc - tajo-sql-parser - tajo-storage - tajo-pullserver - tajo-yarn - tajo-thirdparty/asm - tajo-cli - tajo-metrics - tajo-core-tests - tajo-cluster-tests - tajo-dist - + 4.0.0 + tajo-dist + Tajo Distribution + Tajo Distribution + jar - - - - org.apache.maven.plugins - maven-dependency-plugin - 2.4 - - - org.apache.maven.plugins - maven-surefire-report-plugin - 2.19 - - true - - - - org.apache.maven.plugins - maven-compiler-plugin - 3.3 - - 1.8 - 1.8 - ${project.build.sourceEncoding} - - - - org.apache.maven.plugins - maven-enforcer-plugin - 1.4 - - - - [3.0.2,) - - - 1.8 - - - - - - maven-assembly-plugin - 2.3 - - - org.apache.maven.plugins - maven-deploy-plugin - 2.5 - - - org.apache.maven.plugins - maven-antrun-plugin - 1.6 - - - validate - - run - - - - - PROTOC_VERSION=`protoc --version` - if [ "${PROTOC_VERSION}" == "" ]; then - echo - echo "Protocol buffer is not installed or protocol buffer path did not add to your PATH variable." - echo - exit -1 - fi - if [ "${PROTOC_VERSION}" != "libprotoc 2.5.0" ]; then - echo - echo "Tajo requires protocol buffer version 2.5.0, another versions is not supported." - echo - exit -1 - fi - - - - - - - - - - - org.apache.rat - apache-rat-plugin - 0.8 - - - org.apache.maven.plugins - maven-site-plugin - 3.0 - - - org.apache.maven.wagon - wagon-ssh - 1.0 - - - - - - org.eclipse.m2e - lifecycle-mapping - 1.0.0 - - - - - - org.apache.maven.plugins - maven-antrun-plugin - [1.6,) - - run - - - - - - - - - org.apache.maven.plugins - maven-resources-plugin - [2.2,) - - testResources - resources - - - - - - - - - org.apache.avro - avro-maven-plugin - [1.5.3,) - - schema - protocol - - - - - - - - - org.codehaus.mojo.jspc - jspc-maven-plugin - [2.0-alpha-3,) - - compile - - - - - - - - - org.apache.maven.plugins - maven-dependency-plugin - [2.1,) - - copy-dependencies - build-classpath - - - - - - - - - org.codehaus.mojo - exec-maven-plugin - [1.2,) - - exec - - - - - - - - - org.apache.maven.plugins - maven-jar-plugin - [2.3.1,) - - test-jar - - - - - - - - - - - - - - org.apache.maven.plugins - maven-enforcer-plugin - false + org.apache.rat + apache-rat-plugin - clean - - enforce - - pre-clean - - - default - - enforce - - validate - - - site + verify - enforce + check - pre-site - - - org.apache.rat - apache-rat-plugin - false - .git/** - .gitignore - .idea/** - .reviewboardrc - CHANGES - tajo-docs/src/main/sphinx/** + src/main/conf/workers + src/main/conf/storage-site.json.template - - - - userdocs - - tajo-docs - - - false - - + + + org.apache.tajo + tajo-common + provided + + + org.apache.tajo + tajo-rpc-protobuf + provided + + + org.apache.tajo + tajo-ws-rs + provided + + + org.apache.tajo + tajo-catalog-server + provided + + + org.apache.tajo + tajo-core + provided + + + org.apache.tajo + tajo-storage-common + provided + + + org.apache.tajo + tajo-storage-hdfs + provided + + + - apache-release + dist false + + tar + - - maven-assembly-plugin - - - org.apache.apache.resources - apache-source-release-assembly-descriptor - 1.0.4 - - + org.apache.maven.plugins + maven-antrun-plugin - source-release-assembly - package + dist + prepare-package - single + run - false - true - tajo-${project.version}-src - - ${sourceReleaseAssemblyDescriptor} - - - tajo-dist/src/main/resources/assemblies/tajo-src.xml - - gnu + + + run() { + echo "\$ ${@}" + "${@}" + res=$? + if [ $res != 0 ]; then + echo + echo "Failed!" + echo + exit $res + fi + } + + ROOT=`cd ${basedir}/..;pwd` + echo + echo "Current directory `pwd`" + echo + run rm -rf tajo-${project.version} + run mkdir tajo-${project.version} + run cd tajo-${project.version} + run cp -r $ROOT/tajo-common/target/tajo-common-${project.version}/* . + run cp -r $ROOT/tajo-rpc/target/tajo-rpc-${project.version}/* . + run cp -r $ROOT/tajo-algebra/target/tajo-algebra-${project.version}/* . + run cp -r $ROOT/tajo-plan/target/tajo-plan-${project.version}/* . + run cp -r $ROOT/tajo-client/target/tajo-client-${project.version}/* . + run cp -r $ROOT/tajo-client-example/target/tajo-client-example-${project.version}/* . + run cp -r $ROOT/tajo-cli/target/tajo-cli-${project.version}/* . + run cp -r $ROOT/tajo-catalog/target/tajo-catalog-${project.version}/* . + run cp -r $ROOT/tajo-storage/target/tajo-storage-${project.version}/* . + run cp -r $ROOT/tajo-sql-parser/target/tajo-sql-parser-${project.version}/* . + run cp -r $ROOT/tajo-storage/tajo-storage-jdbc/target/tajo-storage-jdbc-${project.version}.jar . + run cp -r $ROOT/tajo-storage/tajo-storage-pgsql/target/tajo-storage-pgsql-${project.version}.jar . + run cp -r $ROOT/tajo-pullserver/target/tajo-pullserver-${project.version}.jar . + run cp -r $ROOT/tajo-metrics/target/tajo-metrics-${project.version}.jar . + run cp -r $ROOT/tajo-core/target/tajo-core-${project.version}.jar . + run cp -r $ROOT/tajo-core/target/lib . + run cp -r ${project.basedir}/src/main/bin . + run cp -r ${project.basedir}/src/main/conf . + run rm -rf lib/tajo-*-${project.version}.jar + + run mkdir -p lib + run cp -r $ROOT/tajo-storage/tajo-storage-hdfs/target/lib/hive-*.jar lib/ + + run mkdir -p share/jdbc-dist + run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar + + run mkdir -p share/yarn-dist + run cp -r $ROOT/tajo-yarn/target/tajo-yarn-${project.version}-jar-with-dependencies.jar ./share/yarn-dist/tajo-yarn-${project.version}.jar + + run mkdir -p extlib + + echo + echo "Tajo dist layout available at: ${project.build.directory}/tajo-${project.version}" + echo + + + + + + - - - - - true - org.apache.maven.plugins - maven-deploy-plugin - - true - - - - org.apache.maven.plugins - maven-source-plugin - - - attach-sources - - jar - - - - - - org.apache.maven.plugins - maven-javadoc-plugin - - attach-javadocs + tar + package - jar + run + + + + run() { + echo "\$ ${@}" + "${@}" + res=$? + if [ $res != 0 ]; then + echo + echo "Failed!" + echo + exit $res + fi + } + + run tar czf tajo-${project.version}.tar.gz tajo-${project.version} + echo + echo "Tajo dist tar available at: ${project.build.directory}/tajo-${project.version}.tar.gz" + echo + + + + + + - + + + + + visualize + + false + + + + org.apache.hadoop + hadoop-yarn-common + + + org.apache.hadoop + hadoop-common + + + commons-el + commons-el + + + tomcat + jasper-runtime + + + tomcat + jasper-compiler + + + org.mortbay.jetty + jsp-2.1-jetty + + + + + + - org.apache.maven.plugins - maven-gpg-plugin - - ${gpg.passphrase} - ${gpg.useagent} - + org.codehaus.mojo + exec-maven-plugin + compile - sign + java + + test + org.apache.hadoop.yarn.state.VisualizeStateMachine + + Tajo + org.apache.tajo.querymaster.Query, + org.apache.tajo.querymaster.Stage, + org.apache.tajo.querymaster.Task, + org.apache.tajo.querymaster.TaskAttempt + + Tajo.gv + + From 2819df875440f6fb802055d050edf4c9bd26dd17 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 17:13:40 +0900 Subject: [PATCH 13/27] Remove unnecessary modifications --- pom.xml | 607 ++++++++++++++++++++++++++++++---------------- tajo-dist/pom.xml | 2 +- 2 files changed, 402 insertions(+), 207 deletions(-) diff --git a/pom.xml b/pom.xml index 7280e1f2fc..71c062bbfa 100644 --- a/pom.xml +++ b/pom.xml @@ -18,258 +18,453 @@ --> + 4.0.0 - tajo-project - org.apache.tajo - 0.12.0-SNAPSHOT - ../tajo-project + org.apache + apache + 15 - 4.0.0 - tajo-dist - Tajo Distribution - Tajo Distribution - jar + org.apache.tajo + tajo-main + 0.12.0-SNAPSHOT + Tajo Main + http://tajo.apache.org/ + Tajo Main + pom + + + + The Apache Software License, Version 2.0 + http://www.apache.org/licenses/LICENSE-2.0.txt + + + + + The Apache Software Foundation + http://www.apache.org/ + + + + Jira + https://issues.apache.org/jira/browse/TAJO + + + + Jenkins + https://builds.apache.org + + + + https://git-wip-us.apache.org/repos/asf/tajo.git + scm:git:https://git-wip-us.apache.org/repos/asf/tajo.git + scm:git:https://git-wip-us.apache.org/repos/asf/tajo.git + HEAD + + + + + eclipse-jetty + http://repo2.maven.org/maven2/org/eclipse/jetty/jetty-distribution/ + + false + + + + + + UTF-8 + UTF-8 + ${basedir} + + + + tajo-maven-plugins + tajo-project + tajo-common + tajo-algebra + tajo-plan + tajo-core + tajo-rpc + tajo-catalog + tajo-client + tajo-client-example + tajo-jdbc + tajo-sql-parser + tajo-storage + tajo-pullserver + tajo-yarn + tajo-thirdparty/asm + tajo-cli + tajo-metrics + tajo-core-tests + tajo-cluster-tests + tajo-dist + + + + + org.apache.maven.plugins + maven-dependency-plugin + 2.4 + + + org.apache.maven.plugins + maven-surefire-report-plugin + 2.19 + + true + + + + org.apache.maven.plugins + maven-compiler-plugin + 3.3 + + 1.8 + 1.8 + ${project.build.sourceEncoding} + + + + org.apache.maven.plugins + maven-enforcer-plugin + 1.4 + + + + [3.0.2,) + + + 1.8 + + + + + + maven-assembly-plugin + 2.3 + + + org.apache.maven.plugins + maven-deploy-plugin + 2.5 + + + org.apache.maven.plugins + maven-antrun-plugin + 1.6 + + + validate + + run + + + + + PROTOC_VERSION=`protoc --version` + if [ "${PROTOC_VERSION}" == "" ]; then + echo + echo "Protocol buffer is not installed or protocol buffer path did not add to your PATH variable." + echo + exit -1 + fi + if [ "${PROTOC_VERSION}" != "libprotoc 2.5.0" ]; then + echo + echo "Tajo requires protocol buffer version 2.5.0, another versions is not supported." + echo + exit -1 + fi + + + + + + + + + + + org.apache.rat + apache-rat-plugin + 0.8 + + + org.apache.maven.plugins + maven-site-plugin + 3.0 + + + org.apache.maven.wagon + wagon-ssh + 1.0 + + + + + + org.eclipse.m2e + lifecycle-mapping + 1.0.0 + + + + + + org.apache.maven.plugins + maven-antrun-plugin + [1.6,) + + run + + + + + + + + + org.apache.maven.plugins + maven-resources-plugin + [2.2,) + + testResources + resources + + + + + + + + + org.apache.avro + avro-maven-plugin + [1.5.3,) + + schema + protocol + + + + + + + + + org.codehaus.mojo.jspc + jspc-maven-plugin + [2.0-alpha-3,) + + compile + + + + + + + + + org.apache.maven.plugins + maven-dependency-plugin + [2.1,) + + copy-dependencies + build-classpath + + + + + + + + + org.codehaus.mojo + exec-maven-plugin + [1.2,) + + exec + + + + + + + + + org.apache.maven.plugins + maven-jar-plugin + [2.3.1,) + + test-jar + + + + + + + + + + + + + - org.apache.rat - apache-rat-plugin + org.apache.maven.plugins + maven-enforcer-plugin + false - verify + clean + + enforce + + pre-clean + + + default + + enforce + + validate + + + site - check + enforce + pre-site + + + org.apache.rat + apache-rat-plugin + false - src/main/conf/workers - src/main/conf/storage-site.json.template + .git/** + .gitignore + .idea/** + .reviewboardrc + CHANGES + tajo-docs/src/main/sphinx/** - - - org.apache.tajo - tajo-common - provided - - - org.apache.tajo - tajo-rpc-protobuf - provided - - - org.apache.tajo - tajo-ws-rs - provided - - - org.apache.tajo - tajo-catalog-server - provided - - - org.apache.tajo - tajo-core - provided - - - org.apache.tajo - tajo-storage-common - provided - - - org.apache.tajo - tajo-storage-hdfs - provided - - - + + + userdocs + + tajo-docs + + + false + + + - dist + apache-release false - - tar - + - org.apache.maven.plugins - maven-antrun-plugin + maven-assembly-plugin + + + org.apache.apache.resources + apache-source-release-assembly-descriptor + 1.0.4 + + - dist - prepare-package + source-release-assembly + package - run + single - - - run() { - echo "\$ ${@}" - "${@}" - res=$? - if [ $res != 0 ]; then - echo - echo "Failed!" - echo - exit $res - fi - } - - ROOT=`cd ${basedir}/..;pwd` - echo - echo "Current directory `pwd`" - echo - run rm -rf tajo-${project.version} - run mkdir tajo-${project.version} - run cd tajo-${project.version} - run cp -r $ROOT/tajo-common/target/tajo-common-${project.version}/* . - run cp -r $ROOT/tajo-rpc/target/tajo-rpc-${project.version}/* . - run cp -r $ROOT/tajo-algebra/target/tajo-algebra-${project.version}/* . - run cp -r $ROOT/tajo-plan/target/tajo-plan-${project.version}/* . - run cp -r $ROOT/tajo-client/target/tajo-client-${project.version}/* . - run cp -r $ROOT/tajo-client-example/target/tajo-client-example-${project.version}/* . - run cp -r $ROOT/tajo-cli/target/tajo-cli-${project.version}/* . - run cp -r $ROOT/tajo-catalog/target/tajo-catalog-${project.version}/* . - run cp -r $ROOT/tajo-storage/target/tajo-storage-${project.version}/* . - run cp -r $ROOT/tajo-sql-parser/target/tajo-sql-parser-${project.version}/* . - run cp -r $ROOT/tajo-storage/tajo-storage-jdbc/target/tajo-storage-jdbc-${project.version}.jar . - run cp -r $ROOT/tajo-storage/tajo-storage-pgsql/target/tajo-storage-pgsql-${project.version}.jar . - run cp -r $ROOT/tajo-pullserver/target/tajo-pullserver-${project.version}.jar . - run cp -r $ROOT/tajo-metrics/target/tajo-metrics-${project.version}.jar . - run cp -r $ROOT/tajo-core/target/tajo-core-${project.version}.jar . - run cp -r $ROOT/tajo-core/target/lib . - run cp -r ${project.basedir}/src/main/bin . - run cp -r ${project.basedir}/src/main/conf . - run rm -rf lib/tajo-*-${project.version}.jar - - run mkdir -p lib - run cp -r $ROOT/tajo-storage/tajo-storage-hdfs/target/lib/hive-*.jar lib/ - - run mkdir -p share/jdbc-dist - run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar - - run mkdir -p share/yarn-dist - run cp -r $ROOT/tajo-yarn/target/tajo-yarn-${project.version}-jar-with-dependencies.jar ./share/yarn-dist/tajo-yarn-${project.version}.jar - - run mkdir -p extlib - - echo - echo "Tajo dist layout available at: ${project.build.directory}/tajo-${project.version}" - echo - - - - - - + false + true + tajo-${project.version}-src + + ${sourceReleaseAssemblyDescriptor} + + + tajo-dist/src/main/resources/assemblies/tajo-src.xml + + gnu + + + + + true + org.apache.maven.plugins + maven-deploy-plugin + + true + + + + org.apache.maven.plugins + maven-source-plugin + - tar - package + attach-sources - run + jar - - - - run() { - echo "\$ ${@}" - "${@}" - res=$? - if [ $res != 0 ]; then - echo - echo "Failed!" - echo - exit $res - fi - } - - run tar czf tajo-${project.version}.tar.gz tajo-${project.version} - echo - echo "Tajo dist tar available at: ${project.build.directory}/tajo-${project.version}.tar.gz" - echo - - - - - - - - - - - visualize - - false - - - - org.apache.hadoop - hadoop-yarn-common - - - org.apache.hadoop - hadoop-common - - - commons-el - commons-el - - - tomcat - jasper-runtime - - - tomcat - jasper-compiler - - - org.mortbay.jetty - jsp-2.1-jetty - - - - - - - org.codehaus.mojo - exec-maven-plugin + org.apache.maven.plugins + maven-javadoc-plugin - compile + attach-javadocs - java + jar + + + + + + + org.apache.maven.plugins + maven-gpg-plugin + + ${gpg.passphrase} + ${gpg.useagent} + + + + + sign - - test - org.apache.hadoop.yarn.state.VisualizeStateMachine - - Tajo - org.apache.tajo.querymaster.Query, - org.apache.tajo.querymaster.Stage, - org.apache.tajo.querymaster.Task, - org.apache.tajo.querymaster.TaskAttempt - - Tajo.gv - - diff --git a/tajo-dist/pom.xml b/tajo-dist/pom.xml index beafdf23ca..7280e1f2fc 100644 --- a/tajo-dist/pom.xml +++ b/tajo-dist/pom.xml @@ -156,7 +156,7 @@ run mkdir -p lib run cp -r $ROOT/tajo-storage/tajo-storage-hdfs/target/lib/hive-*.jar lib/ - + run mkdir -p share/jdbc-dist run cp -r $ROOT/tajo-jdbc/target/tajo-jdbc-${project.version}-jar-with-dependencies.jar ./share/jdbc-dist/tajo-jdbc-${project.version}.jar From 4de642a39a3c29b1188700cc123d8ce1d3e548c8 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Tue, 24 May 2016 17:22:29 +0900 Subject: [PATCH 14/27] Add more dependencies --- tajo-project/pom.xml | 1 + tajo-storage/tajo-storage-s3/pom.xml | 39 +++++++++++++--------------- 2 files changed, 19 insertions(+), 21 deletions(-) diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml index 9def61986a..0352878e18 100644 --- a/tajo-project/pom.xml +++ b/tajo-project/pom.xml @@ -41,6 +41,7 @@ 6.1.26 1.8.1 1.7.4 + 4.5 ${project.parent.relativePath}/.. src/main/hadoop-${hadoop.version} diff --git a/tajo-storage/tajo-storage-s3/pom.xml b/tajo-storage/tajo-storage-s3/pom.xml index 5e4cde47ce..1394470baf 100644 --- a/tajo-storage/tajo-storage-s3/pom.xml +++ b/tajo-storage/tajo-storage-s3/pom.xml @@ -82,31 +82,10 @@ - org.apache.maven.plugins maven-surefire-report-plugin - - org.apache.maven.plugins - maven-dependency-plugin - - - copy-dependencies - package - - copy-dependencies - - - runtime - ${project.build.directory}/lib - false - false - true - - - - @@ -126,6 +105,16 @@ tajo-storage-hdfs provided + + org.apache.hadoop + hadoop-common + provided + + + org.apache.hadoop + hadoop-hdfs + provided + com.amazonaws @@ -139,6 +128,14 @@ junit test + + + org.apache.httpcomponents + httpclient + ${httpclient.version} + test + + From d20b1c88fcfa1f428fecd98fe0cdf60a74b8b2e7 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Wed, 25 May 2016 11:42:46 +0900 Subject: [PATCH 15/27] Never use S3Tablespace less than hadoop 2.6.0 --- .../tajo/storage/TablespaceManager.java | 19 +++++++++++++++++-- 1 file changed, 17 insertions(+), 2 deletions(-) diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java index 88410bb2cc..5f8a277318 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java @@ -184,8 +184,23 @@ private Pair> extractStorage(Map.Entry( - storageType, (Class) Class.forName(handlerClass)); + Pair> pair = null; + try { + pair = new Pair<>( + storageType, (Class) Class.forName(handlerClass)); + } catch (NoClassDefFoundError e) { + // If the version of hadoop is less than 2.6.0, hadoop doesn't include aws dependencies because it doesn't provide + // S3AFileSystem. In this case, tajo never uses S3Tablespace. + if (e.getMessage().equals("com/amazonaws/services/s3/AmazonS3")) { + LOG.warn(e); + handlerClass = "org.apache.tajo.storage.FileTablespace"; + pair = new Pair<>(storageType, (Class) Class.forName(handlerClass)); + } else { + throw new RuntimeException(e); + } + } + + return pair; } private void loadTableSpaces(JSONObject json, boolean override) { From 6615635cd7b1dbb3060ce741579297404db46224 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Wed, 25 May 2016 12:21:13 +0900 Subject: [PATCH 16/27] Add exception handling --- .../main/java/org/apache/tajo/storage/TablespaceManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java index 5f8a277318..2f04a14be2 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java @@ -198,8 +198,10 @@ private Pair> extractStorage(Map.Entry Date: Wed, 25 May 2016 12:50:01 +0900 Subject: [PATCH 17/27] Add ClassNotFoundException handling --- .../main/java/org/apache/tajo/storage/TablespaceManager.java | 4 +++- 1 file changed, 3 insertions(+), 1 deletion(-) diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java index 2f04a14be2..b6b5cd8a35 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java @@ -198,10 +198,12 @@ private Pair> extractStorage(Map.Entry Date: Sun, 29 May 2016 16:24:22 +0900 Subject: [PATCH 18/27] Remove NoClassDefFoundError handling codes for AmazonS3 --- .../tajo/storage/TablespaceManager.java | 23 ++----------------- 1 file changed, 2 insertions(+), 21 deletions(-) diff --git a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java index b6b5cd8a35..88410bb2cc 100644 --- a/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java +++ b/tajo-storage/tajo-storage-common/src/main/java/org/apache/tajo/storage/TablespaceManager.java @@ -184,27 +184,8 @@ private Pair> extractStorage(Map.Entry> pair = null; - try { - pair = new Pair<>( - storageType, (Class) Class.forName(handlerClass)); - } catch (NoClassDefFoundError e) { - // If the version of hadoop is less than 2.6.0, hadoop doesn't include aws dependencies because it doesn't provide - // S3AFileSystem. In this case, tajo never uses S3Tablespace. - if (e.getMessage().equals("com/amazonaws/services/s3/AmazonS3")) { - LOG.warn(e); - handlerClass = "org.apache.tajo.storage.FileTablespace"; - pair = new Pair<>(storageType, (Class) Class.forName(handlerClass)); - } else { - throw new RuntimeException(e); - } - } catch (ClassNotFoundException e) { - throw e; - } catch (Exception e) { - throw new RuntimeException(e); - } - - return pair; + return new Pair<>( + storageType, (Class) Class.forName(handlerClass)); } private void loadTableSpaces(JSONObject json, boolean override) { From 0bbd297437a017640714877036ebb90fad933760 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 10:11:58 +0900 Subject: [PATCH 19/27] Add NoClassDefFoundError handling codes to S3TableSpace --- .../apache/tajo/storage/s3/S3TableSpace.java | 161 ++++++++++-------- 1 file changed, 89 insertions(+), 72 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index d1baed9164..6a8c76d811 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -40,18 +40,18 @@ import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.tajo.conf.TajoConf; +import org.apache.tajo.exception.TajoInternalError; import org.apache.tajo.storage.FileTablespace; import net.minidev.json.JSONObject; -import static com.google.common.base.Preconditions.checkArgument; -import static com.google.common.base.Strings.nullToEmpty; import static org.apache.tajo.storage.s3.TajoS3Constants.*; public class S3TableSpace extends FileTablespace { private final static Log LOG = LogFactory.getLog(S3TableSpace.class); private AmazonS3 s3; + private boolean s3Enabled; private int maxKeys; public S3TableSpace(String spaceName, URI uri, JSONObject config) { @@ -62,36 +62,47 @@ public S3TableSpace(String spaceName, URI uri, JSONObject config) { public void init(TajoConf tajoConf) throws IOException { super.init(tajoConf); - maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS); - int maxErrorRetries = conf.getInt(MAX_ERROR_RETRIES, DEFAULT_MAX_ERROR_RETRIES); - boolean sslEnabled = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); - - int connectTimeout = conf.getInt(ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT); - int socketTimeout = conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT); - int maxConnections = conf.getInt(MAXIMUM_CONNECTIONS, DEFAULT_MAXIMUM_CONNECTIONS); - - ClientConfiguration configuration = new ClientConfiguration() - .withMaxErrorRetry(maxErrorRetries) - .withProtocol(sslEnabled ? Protocol.HTTPS : Protocol.HTTP) - .withConnectionTimeout(connectTimeout) - .withSocketTimeout(socketTimeout) - .withMaxConnections(maxConnections); - - this.s3 = createAmazonS3Client(uri, conf, configuration); - - if (s3 != null) { - String endPoint = conf.getTrimmed(ENDPOINT,""); - try { - if (!endPoint.isEmpty()) { - s3.setEndpoint(endPoint); + try { + maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS); + int maxErrorRetries = conf.getInt(MAX_ERROR_RETRIES, DEFAULT_MAX_ERROR_RETRIES); + boolean sslEnabled = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); + + int connectTimeout = conf.getInt(ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT); + int socketTimeout = conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT); + int maxConnections = conf.getInt(MAXIMUM_CONNECTIONS, DEFAULT_MAXIMUM_CONNECTIONS); + + ClientConfiguration configuration = new ClientConfiguration() + .withMaxErrorRetry(maxErrorRetries) + .withProtocol(sslEnabled ? Protocol.HTTPS : Protocol.HTTP) + .withConnectionTimeout(connectTimeout) + .withSocketTimeout(socketTimeout) + .withMaxConnections(maxConnections); + + this.s3 = createAmazonS3Client(uri, conf, configuration); + + if (s3 != null) { + String endPoint = conf.getTrimmed(ENDPOINT,""); + try { + if (!endPoint.isEmpty()) { + s3.setEndpoint(endPoint); + } + } catch (IllegalArgumentException e) { + String msg = "Incorrect endpoint: " + e.getMessage(); + LOG.error(msg); + throw new IllegalArgumentException(msg, e); } - } catch (IllegalArgumentException e) { - String msg = "Incorrect endpoint: " + e.getMessage(); - LOG.error(msg); - throw new IllegalArgumentException(msg, e); + + LOG.info("Amazon3Client is initialized."); } - LOG.info("Amazon3Client is initialized."); + s3Enabled = true; + } catch (NoClassDefFoundError defFoundError) { + // If the version of hadoop is less than 2.6.0, hadoop doesn't include aws dependencies because it doesn't provide + // S3AFileSystem. In this case, tajo never uses aws s3 api directly. + LOG.warn(defFoundError); + s3Enabled = false; + } catch (Exception e) { + throw new TajoInternalError(e); } } @@ -119,55 +130,60 @@ private static AWSCredentials getAwsCredentials(URI uri, Configuration conf) { @Override public long calculateSize(Path path) throws IOException { - String key = keyFromPath(path); - - final FileStatus fileStatus = fs.getFileStatus(path); long totalBucketSize = 0L; - if (fileStatus.isDirectory()) { - if (!key.isEmpty()) { - key = key + "/"; - } + if (s3Enabled) { + String key = pathToKey(path); - ListObjectsRequest request = new ListObjectsRequest(); - request.setBucketName(uri.getHost()); - request.setPrefix(key); - request.setMaxKeys(maxKeys); + final FileStatus fileStatus = fs.getFileStatus(path); - if (LOG.isDebugEnabled()) { - LOG.debug("listStatus: doing listObjects for directory " + key); - } + if (fileStatus.isDirectory()) { + if (!key.isEmpty()) { + key = key + "/"; + } - ObjectListing objects = s3.listObjects(request); + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(uri.getHost()); + request.setPrefix(key); + request.setMaxKeys(maxKeys); - while (true) { - for (S3ObjectSummary summary : objects.getObjectSummaries()) { - Path keyPath = keyToPath(summary.getKey()).makeQualified(uri, fs.getWorkingDirectory()); + if (LOG.isDebugEnabled()) { + LOG.debug("listStatus: doing listObjects for directory " + key); + } - // Skip over keys that are ourselves and old S3N _$folder$ files - if (keyPath.equals(path) || summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) { - if (LOG.isDebugEnabled()) { - LOG.debug("Ignoring: " + keyPath); + ObjectListing objects = s3.listObjects(request); + + while (true) { + for (S3ObjectSummary summary : objects.getObjectSummaries()) { + Path keyPath = keyToPath(summary.getKey()).makeQualified(uri, fs.getWorkingDirectory()); + + // Skip over keys that are ourselves and old S3N _$folder$ files + if (keyPath.equals(path) || summary.getKey().endsWith(S3N_FOLDER_SUFFIX)) { + if (LOG.isDebugEnabled()) { + LOG.debug("Ignoring: " + keyPath); + } + continue; } - continue; - } - if (!objectRepresentsDirectory(summary.getKey(), summary.getSize())) { - totalBucketSize += summary.getSize(); + if (!objectRepresentsDirectory(summary.getKey(), summary.getSize())) { + totalBucketSize += summary.getSize(); + } } - } - if (objects.isTruncated()) { - if (LOG.isDebugEnabled()) { - LOG.debug("listStatus: list truncated - getting next batch"); + if (objects.isTruncated()) { + if (LOG.isDebugEnabled()) { + LOG.debug("listStatus: list truncated - getting next batch"); + } + objects = s3.listNextBatchOfObjects(objects); + } else { + break; } - objects = s3.listNextBatchOfObjects(objects); - } else { - break; } + } else { + return fileStatus.getLen(); } } else { - return fileStatus.getLen(); + totalBucketSize = fs.getContentSummary(path).getLength(); } return totalBucketSize; @@ -181,17 +197,18 @@ private Path keyToPath(String key) { return new Path("/" + key); } - private String keyFromPath(Path path) - { - checkArgument(path.isAbsolute(), "Path is not absolute: %s", path); - String key = nullToEmpty(path.toUri().getPath()); - if (key.startsWith("/")) { - key = key.substring(1); + /* Turns a path (relative or otherwise) into an S3 key + */ + private String pathToKey(Path path) { + if (!path.isAbsolute()) { + path = new Path(fs.getWorkingDirectory(), path); } - if (key.endsWith("/")) { - key = key.substring(0, key.length() - 1); + + if (path.toUri().getScheme() != null && path.toUri().getPath().isEmpty()) { + return ""; } - return key; + + return path.toUri().getPath().substring(1); } @VisibleForTesting From 499a3f5675bd55b0568c913ec4a74db5ceef6ada Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 10:17:45 +0900 Subject: [PATCH 20/27] Add descriptions for FileTablespace::calculateSize --- .../main/java/org/apache/tajo/storage/FileTablespace.java | 7 +++++++ .../main/java/org/apache/tajo/storage/s3/S3TableSpace.java | 7 +++++++ 2 files changed, 14 insertions(+) diff --git a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java index 2491dbfcd2..c8e75489f7 100644 --- a/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java +++ b/tajo-storage/tajo-storage-hdfs/src/main/java/org/apache/tajo/storage/FileTablespace.java @@ -246,6 +246,13 @@ public static FileFragment[] splitNG(Configuration conf, String tableName, Table return tablets; } + /** + * Calculate the total size of all files in the indicated Path + * + * @param path to use + * @return calculated size + * @throws IOException + */ public long calculateSize(Path tablePath) throws IOException { FileSystem fs = tablePath.getFileSystem(conf); long totalSize = 0; diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index 6a8c76d811..d1a47f6a3a 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -128,6 +128,13 @@ private static AWSCredentials getAwsCredentials(URI uri, Configuration conf) { return new BasicAWSCredentials(credentials.getAccessKey(), credentials.getSecretAccessKey()); } + /** + * Calculate the total size of all objects in the indicated bucket + * + * @param path to use + * @return calculated size + * @throws IOException + */ @Override public long calculateSize(Path path) throws IOException { long totalBucketSize = 0L; From b00c7d74ddf609f48ebc5d5967b74b4c6a14bdda Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 10:38:14 +0900 Subject: [PATCH 21/27] Borrow credential provides from hadoop --- .../s3/AnonymousAWSCredentialsProvider.java | 37 +++++ .../s3/BasicAWSCredentialsProvider.java | 51 +++++++ .../apache/tajo/storage/s3/S3TableSpace.java | 127 +++++++++++------- .../tajo/storage/s3/TajoS3Constants.java | 8 ++ .../tajo/storage/s3/TajoS3Credentials.java | 94 ------------- 5 files changed, 173 insertions(+), 144 deletions(-) create mode 100644 tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java create mode 100644 tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java delete mode 100644 tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java new file mode 100644 index 0000000000..1c6cbbadf6 --- /dev/null +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java @@ -0,0 +1,37 @@ +/** + * 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.tajo.storage.s3; + +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.AnonymousAWSCredentials; +import com.amazonaws.auth.AWSCredentials; + +public class AnonymousAWSCredentialsProvider implements AWSCredentialsProvider { + public AWSCredentials getCredentials() { + return new AnonymousAWSCredentials(); + } + + public void refresh() {} + + @Override + public String toString() { + return getClass().getSimpleName(); + } + +} diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java new file mode 100644 index 0000000000..4dcc80be0d --- /dev/null +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java @@ -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.tajo.storage.s3; + +import com.amazonaws.AmazonClientException; +import com.amazonaws.auth.AWSCredentials; +import com.amazonaws.auth.AWSCredentialsProvider; +import com.amazonaws.auth.BasicAWSCredentials; +import org.apache.commons.lang.StringUtils; + +public class BasicAWSCredentialsProvider implements AWSCredentialsProvider { + private final String accessKey; + private final String secretKey; + + public BasicAWSCredentialsProvider(String accessKey, String secretKey) { + this.accessKey = accessKey; + this.secretKey = secretKey; + } + + public AWSCredentials getCredentials() { + if (!StringUtils.isEmpty(accessKey) && !StringUtils.isEmpty(secretKey)) { + return new BasicAWSCredentials(accessKey, secretKey); + } + throw new AmazonClientException( + "Access key or secret key is null"); + } + + public void refresh() {} + + @Override + public String toString() { + return getClass().getSimpleName(); + } + +} diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index d1a47f6a3a..13fc3b2095 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -23,11 +23,7 @@ import com.amazonaws.ClientConfiguration; import com.amazonaws.Protocol; -import com.amazonaws.auth.AWSCredentials; -import com.amazonaws.auth.AWSCredentialsProvider; -import com.amazonaws.auth.BasicAWSCredentials; -import com.amazonaws.auth.InstanceProfileCredentialsProvider; -import com.amazonaws.internal.StaticCredentialsProvider; +import com.amazonaws.auth.*; import com.amazonaws.services.s3.AmazonS3; import com.amazonaws.services.s3.AmazonS3Client; import com.amazonaws.services.s3.model.ListObjectsRequest; @@ -36,7 +32,6 @@ import com.google.common.annotations.VisibleForTesting; import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; import org.apache.tajo.conf.TajoConf; @@ -63,38 +58,92 @@ public void init(TajoConf tajoConf) throws IOException { super.init(tajoConf); try { - maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS); - int maxErrorRetries = conf.getInt(MAX_ERROR_RETRIES, DEFAULT_MAX_ERROR_RETRIES); - boolean sslEnabled = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); - - int connectTimeout = conf.getInt(ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT); - int socketTimeout = conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT); - int maxConnections = conf.getInt(MAXIMUM_CONNECTIONS, DEFAULT_MAXIMUM_CONNECTIONS); - - ClientConfiguration configuration = new ClientConfiguration() - .withMaxErrorRetry(maxErrorRetries) - .withProtocol(sslEnabled ? Protocol.HTTPS : Protocol.HTTP) - .withConnectionTimeout(connectTimeout) - .withSocketTimeout(socketTimeout) - .withMaxConnections(maxConnections); + // Try to get our credentials or just connect anonymously + String accessKey = conf.get(ACCESS_KEY, null); + String secretKey = conf.get(SECRET_KEY, null); + + String userInfo = uri.getUserInfo(); + if (userInfo != null) { + int index = userInfo.indexOf(':'); + if (index != -1) { + accessKey = userInfo.substring(0, index); + secretKey = userInfo.substring(index + 1); + } else { + accessKey = userInfo; + } + } - this.s3 = createAmazonS3Client(uri, conf, configuration); + AWSCredentialsProviderChain credentials = new AWSCredentialsProviderChain( + new BasicAWSCredentialsProvider(accessKey, secretKey), + new InstanceProfileCredentialsProvider(), + new AnonymousAWSCredentialsProvider() + ); + + ClientConfiguration awsConf = new ClientConfiguration(); + awsConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS, + DEFAULT_MAXIMUM_CONNECTIONS)); + boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, + DEFAULT_SECURE_CONNECTIONS); + awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP); + awsConf.setMaxErrorRetry(conf.getInt(MAX_ERROR_RETRIES, + DEFAULT_MAX_ERROR_RETRIES)); + awsConf.setConnectionTimeout(conf.getInt(ESTABLISH_TIMEOUT, + DEFAULT_ESTABLISH_TIMEOUT)); + awsConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT, + DEFAULT_SOCKET_TIMEOUT)); + + String proxyHost = conf.getTrimmed(PROXY_HOST,""); + int proxyPort = conf.getInt(PROXY_PORT, -1); + if (!proxyHost.isEmpty()) { + awsConf.setProxyHost(proxyHost); + if (proxyPort >= 0) { + awsConf.setProxyPort(proxyPort); + } else { + if (secureConnections) { + LOG.warn("Proxy host set without port. Using HTTPS default 443"); + awsConf.setProxyPort(443); + } else { + LOG.warn("Proxy host set without port. Using HTTP default 80"); + awsConf.setProxyPort(80); + } + } + String proxyUsername = conf.getTrimmed(PROXY_USERNAME); + String proxyPassword = conf.getTrimmed(PROXY_PASSWORD); + if ((proxyUsername == null) != (proxyPassword == null)) { + String msg = "Proxy error: " + PROXY_USERNAME + " or " + + PROXY_PASSWORD + " set without the other."; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + awsConf.setProxyUsername(proxyUsername); + awsConf.setProxyPassword(proxyPassword); + awsConf.setProxyDomain(conf.getTrimmed(PROXY_DOMAIN)); + awsConf.setProxyWorkstation(conf.getTrimmed(PROXY_WORKSTATION)); + if (LOG.isDebugEnabled()) { + LOG.debug(String.format("Using proxy server %s:%d as user %s with password %s on domain %s as workstation " + + "%s", awsConf.getProxyHost(), awsConf.getProxyPort(), awsConf.getProxyUsername(), + awsConf.getProxyPassword(), awsConf.getProxyDomain(), awsConf.getProxyWorkstation())); + } + } else if (proxyPort >= 0) { + String msg = "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } - if (s3 != null) { - String endPoint = conf.getTrimmed(ENDPOINT,""); + s3 = new AmazonS3Client(credentials, awsConf); + String endPoint = conf.getTrimmed(ENDPOINT,""); + if (!endPoint.isEmpty()) { try { - if (!endPoint.isEmpty()) { - s3.setEndpoint(endPoint); - } + s3.setEndpoint(endPoint); } catch (IllegalArgumentException e) { String msg = "Incorrect endpoint: " + e.getMessage(); LOG.error(msg); throw new IllegalArgumentException(msg, e); } - - LOG.info("Amazon3Client is initialized."); } + maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS); + s3Enabled = true; } catch (NoClassDefFoundError defFoundError) { // If the version of hadoop is less than 2.6.0, hadoop doesn't include aws dependencies because it doesn't provide @@ -106,28 +155,6 @@ public void init(TajoConf tajoConf) throws IOException { } } - private AmazonS3Client createAmazonS3Client(URI uri, Configuration hadoopConfig, ClientConfiguration clientConfig) { - AWSCredentialsProvider credentials = getAwsCredentialsProvider(uri, hadoopConfig); - AmazonS3Client client = new AmazonS3Client(credentials, clientConfig); - return client; - } - - private AWSCredentialsProvider getAwsCredentialsProvider(URI uri, Configuration conf) { - // first try credentials from URI or static properties - try { - return new StaticCredentialsProvider(getAwsCredentials(uri, conf)); - } catch (IllegalArgumentException ignored) { - } - - return new InstanceProfileCredentialsProvider(); - } - - private static AWSCredentials getAwsCredentials(URI uri, Configuration conf) { - TajoS3Credentials credentials = new TajoS3Credentials(); - credentials.initialize(uri, conf); - return new BasicAWSCredentials(credentials.getAccessKey(), credentials.getSecretAccessKey()); - } - /** * Calculate the total size of all objects in the indicated bucket * diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java index 879059de18..48f76b882b 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Constants.java @@ -58,4 +58,12 @@ public class TajoS3Constants { public static final String S3N_FOLDER_SUFFIX = "_$folder$"; + //connect to s3 through a proxy server? + public static final String PROXY_HOST = "fs.s3a.proxy.host"; + public static final String PROXY_PORT = "fs.s3a.proxy.port"; + public static final String PROXY_USERNAME = "fs.s3a.proxy.username"; + public static final String PROXY_PASSWORD = "fs.s3a.proxy.password"; + public static final String PROXY_DOMAIN = "fs.s3a.proxy.domain"; + public static final String PROXY_WORKSTATION = "fs.s3a.proxy.workstation"; + } diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java deleted file mode 100644 index ed1d9dcc95..0000000000 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/TajoS3Credentials.java +++ /dev/null @@ -1,94 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - * - * http://www.apache.org/licenses/LICENSE-2.0 - * - * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.tajo.storage.s3; - -import java.net.URI; - -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; -import org.apache.hadoop.conf.Configuration; - -import static org.apache.tajo.storage.s3.TajoS3Constants.ACCESS_KEY; -import static org.apache.tajo.storage.s3.TajoS3Constants.SECRET_KEY; - -/** - *

- * Extracts AWS credentials from the filesystem URI or configuration. (borrowed from hadoop-aws package) - *

- */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -public class TajoS3Credentials { - - private String accessKey; - private String secretAccessKey; - - /** - * @throws IllegalArgumentException if credentials for S3 cannot be - * determined. - */ - public void initialize(URI uri, Configuration conf) { - if (uri.getHost() == null) { - throw new IllegalArgumentException("Invalid hostname in URI " + uri); - } - - String userInfo = uri.getUserInfo(); - if (userInfo != null) { - int index = userInfo.indexOf(':'); - if (index != -1) { - accessKey = userInfo.substring(0, index); - secretAccessKey = userInfo.substring(index + 1); - } else { - accessKey = userInfo; - } - } - - String scheme = uri.getScheme(); - String accessKeyProperty = scheme.startsWith("s3a") ? ACCESS_KEY : String.format("fs.%s.awsAccessKeyId", scheme); - String secretAccessKeyProperty = scheme.startsWith("s3a") ? SECRET_KEY : String.format("fs.%s.awsSecretAccessKey", - scheme); - - if (accessKey == null) { - accessKey = conf.getTrimmed(accessKeyProperty); - } - if (secretAccessKey == null) { - secretAccessKey = conf.getTrimmed(secretAccessKeyProperty); - } - if (accessKey == null && secretAccessKey == null) { - throw new IllegalArgumentException("AWS Access Key ID and Secret Access Key must be specified as the " + - "username or password (respectively) of a " + scheme + " URL, or by setting the " + - accessKeyProperty + " or " + secretAccessKeyProperty + " properties (respectively)."); - } else if (accessKey == null) { - throw new IllegalArgumentException("AWS Access Key ID must be specified as the username of a " + scheme + - " URL, or by setting the " + accessKeyProperty + " property."); - } else if (secretAccessKey == null) { - throw new IllegalArgumentException("AWS Secret Access Key must be specified as the password of a " + - scheme + " URL, or by setting the " + secretAccessKeyProperty + " property."); - } - - } - - public String getAccessKey() { - return accessKey; - } - - public String getSecretAccessKey() { - return secretAccessKey; - } -} \ No newline at end of file From 17abec266d52d555022252289608339fc6b21c4b Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 11:14:01 +0900 Subject: [PATCH 22/27] Add unit test cases for several prefixes --- .../tajo/storage/s3/MockS3FileSystem.java | 4 +- .../tajo/storage/s3/TestS3TableSpace.java | 81 ++++++++++++++++--- 2 files changed, 72 insertions(+), 13 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java index ea73e22b88..84650be7e7 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java @@ -105,7 +105,9 @@ public boolean mkdirs(Path f, FsPermission permission) throws IOException { @Override public FileStatus getFileStatus(Path f) throws IOException { - if (f.equals(new Path(TestS3TableSpace.S3_URI, "test"))) { + if (f.equals(new Path(TestS3TableSpace.S3_URI, "test")) + || f.equals(new Path(TestS3TableSpace.S3N_URI, "test")) + || f.equals(new Path(TestS3TableSpace.S3A_URI, "test"))) { return new FileStatus(0, true, 1, 0, 0, f); } else { return null; diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java index 3ef7d5eb49..2b630c034f 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/TestS3TableSpace.java @@ -32,43 +32,100 @@ import static org.junit.Assert.*; public class TestS3TableSpace { - public static final String SPACENAME = "s3_cluster"; + public static final String S3_SPACENAME = "s3_cluster"; + public static final String S3N_SPACENAME = "s3N_cluster"; + public static final String S3A_SPACENAME = "s3A_cluster"; + public static final String S3_URI = "s3://tajo-test/"; + public static final String S3N_URI = "s3n://tajo-test/"; + public static final String S3A_URI = "s3a://tajo-test/"; @BeforeClass public static void setUp() throws Exception { - S3TableSpace tablespace = new S3TableSpace(SPACENAME, URI.create(S3_URI), new JSONObject()); - + // Add tablespace for s3 prefix + S3TableSpace s3TableSpace = new S3TableSpace(S3_SPACENAME, URI.create(S3_URI), new JSONObject()); TajoConf tajoConf = new TajoConf(); tajoConf.set("fs.s3.impl", MockS3FileSystem.class.getName()); + tajoConf.set("fs.s3.awsAccessKeyId", "test_access_key_id"); + tajoConf.set("fs.s3.awsSecretAccessKey", "test_secret_access_key"); + s3TableSpace.init(tajoConf); + TablespaceManager.addTableSpaceForTest(s3TableSpace); + + // Add tablespace for s3n prefix + S3TableSpace s3nTableSpace = new S3TableSpace(S3N_SPACENAME, URI.create(S3N_URI), new JSONObject()); + tajoConf = new TajoConf(); + tajoConf.set("fs.s3n.impl", MockS3FileSystem.class.getName()); + tajoConf.set("fs.s3n.awsAccessKeyId", "test_access_key_id"); + tajoConf.set("fs.s3n.awsSecretAccessKey", "test_secret_access_key"); + s3nTableSpace.init(tajoConf); + TablespaceManager.addTableSpaceForTest(s3nTableSpace); + + // Add tablespace for s3a prefix + S3TableSpace s3aTableSpace = new S3TableSpace(S3A_SPACENAME, URI.create(S3A_URI), new JSONObject()); + tajoConf = new TajoConf(); + tajoConf.set("fs.s3a.impl", MockS3FileSystem.class.getName()); tajoConf.set("fs.s3a.access.key", "test_access_key_id"); tajoConf.set("fs.s3a.secret.key", "test_secret_access_key"); - tablespace.init(tajoConf); - - TablespaceManager.addTableSpaceForTest(tablespace); + s3aTableSpace.init(tajoConf); + TablespaceManager.addTableSpaceForTest(s3aTableSpace); } @AfterClass public static void tearDown() throws IOException { - TablespaceManager.removeTablespaceForTest(SPACENAME); + TablespaceManager.removeTablespaceForTest(S3_SPACENAME); + TablespaceManager.removeTablespaceForTest(S3N_SPACENAME); + TablespaceManager.removeTablespaceForTest(S3A_SPACENAME); } @Test public void testTablespaceHandler() throws Exception { - assertTrue((TablespaceManager.getByName(SPACENAME)) instanceof S3TableSpace); - assertEquals(SPACENAME, (TablespaceManager.getByName(SPACENAME).getName())); - + // Verify the tablespace for s3 prefix + assertTrue((TablespaceManager.getByName(S3_SPACENAME)) instanceof S3TableSpace); + assertEquals(S3_SPACENAME, (TablespaceManager.getByName(S3_SPACENAME).getName())); assertTrue((TablespaceManager.get(S3_URI)) instanceof S3TableSpace); assertEquals(S3_URI, TablespaceManager.get(S3_URI).getUri().toASCIIString()); + + // Verify the tablespace for s3n prefix + assertTrue((TablespaceManager.getByName(S3N_SPACENAME)) instanceof S3TableSpace); + assertEquals(S3N_SPACENAME, (TablespaceManager.getByName(S3N_SPACENAME).getName())); + assertTrue((TablespaceManager.get(S3N_URI)) instanceof S3TableSpace); + assertEquals(S3N_URI, TablespaceManager.get(S3N_URI).getUri().toASCIIString()); + + // Verify the tablespace for s3a prefix + assertTrue((TablespaceManager.getByName(S3A_SPACENAME)) instanceof S3TableSpace); + assertEquals(S3A_SPACENAME, (TablespaceManager.getByName(S3A_SPACENAME).getName())); + assertTrue((TablespaceManager.get(S3A_URI)) instanceof S3TableSpace); + assertEquals(S3A_URI, TablespaceManager.get(S3A_URI).getUri().toASCIIString()); } @Test - public void testCalculateSize() throws Exception { + public void testCalculateSizeWithS3Prefix() throws Exception { Path path = new Path(S3_URI, "test"); - assertTrue((TablespaceManager.getByName(SPACENAME)) instanceof S3TableSpace); + assertTrue((TablespaceManager.getByName(S3_SPACENAME)) instanceof S3TableSpace); S3TableSpace tableSpace = TablespaceManager.get(path.toUri()); tableSpace.setAmazonS3Client(new MockAmazonS3()); long size = tableSpace.calculateSize(path); assertEquals(30L, size); } + + @Test + public void testCalculateSizeWithS3NPrefix() throws Exception { + Path path = new Path(S3N_URI, "test"); + assertTrue((TablespaceManager.getByName(S3N_SPACENAME)) instanceof S3TableSpace); + S3TableSpace tableSpace = TablespaceManager.get(path.toUri()); + tableSpace.setAmazonS3Client(new MockAmazonS3()); + long size = tableSpace.calculateSize(path); + assertEquals(30L, size); + } + + @Test + public void testCalculateSizeWithS3APrefix() throws Exception { + Path path = new Path(S3A_URI, "test"); + assertTrue((TablespaceManager.getByName(S3A_SPACENAME)) instanceof S3TableSpace); + S3TableSpace tableSpace = TablespaceManager.get(path.toUri()); + tableSpace.setAmazonS3Client(new MockAmazonS3()); + long size = tableSpace.calculateSize(path); + assertEquals(30L, size); + } + } From 1258d4b0c6e1607629f33f97bd0bf021e22037f0 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 11:15:37 +0900 Subject: [PATCH 23/27] Add some descriptions for provides --- .../tajo/storage/s3/AnonymousAWSCredentialsProvider.java | 4 ++++ .../apache/tajo/storage/s3/BasicAWSCredentialsProvider.java | 4 ++++ 2 files changed, 8 insertions(+) diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java index 1c6cbbadf6..a884ca89b7 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/AnonymousAWSCredentialsProvider.java @@ -22,6 +22,10 @@ import com.amazonaws.auth.AnonymousAWSCredentials; import com.amazonaws.auth.AWSCredentials; +/** + * Borrow from org.apache.hadoop.fs.s3a.AnonymousAWSCredentialsProvider. + * + */ public class AnonymousAWSCredentialsProvider implements AWSCredentialsProvider { public AWSCredentials getCredentials() { return new AnonymousAWSCredentials(); diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java index 4dcc80be0d..0f4fbde763 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/BasicAWSCredentialsProvider.java @@ -24,6 +24,10 @@ import com.amazonaws.auth.BasicAWSCredentials; import org.apache.commons.lang.StringUtils; +/** + * Borrow from org.apache.hadoop.fs.s3a.BasicAWSCredentialsProvider. + * + */ public class BasicAWSCredentialsProvider implements AWSCredentialsProvider { private final String accessKey; private final String secretKey; From 660ca38cc0ed229961066b0cf06da8c0d4d59ac5 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 11:26:08 +0900 Subject: [PATCH 24/27] Add UnsupportedException to MockAmazonS3 and MockS3FileSystem --- .../apache/tajo/storage/s3/MockAmazonS3.java | 219 ++++++++++-------- .../tajo/storage/s3/MockS3FileSystem.java | 15 +- 2 files changed, 124 insertions(+), 110 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java index 422c4d4f9f..a0d8c2675d 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockAmazonS3.java @@ -27,6 +27,9 @@ import com.amazonaws.services.s3.S3ClientOptions; import com.amazonaws.services.s3.S3ResponseMetadata; import com.amazonaws.services.s3.model.*; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.exception.TajoRuntimeException; +import org.apache.tajo.exception.UnsupportedException; import java.io.File; import java.io.InputStream; @@ -40,47 +43,43 @@ public class MockAmazonS3 implements AmazonS3 { private int getObjectHttpCode = SC_OK; private int getObjectMetadataHttpCode = SC_OK; - public void setGetObjectHttpErrorCode(int getObjectHttpErrorCode) { - this.getObjectHttpCode = getObjectHttpErrorCode; - } - - public void setGetObjectMetadataHttpCode(int getObjectMetadataHttpCode) { - this.getObjectMetadataHttpCode = getObjectMetadataHttpCode; - } - @Override public void setEndpoint(String endpoint) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setRegion(Region region) throws IllegalArgumentException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setS3ClientOptions(S3ClientOptions clientOptions) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void changeObjectStorageClass(String bucketName, String key, StorageClass newStorageClass) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setObjectRedirectLocation(String bucketName, String key, String newRedirectLocation) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public ObjectListing listObjects(String bucketName) - throws AmazonClientException { - return null; + public ObjectListing listObjects(String bucketName) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public ObjectListing listObjects(String bucketName, String prefix) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override @@ -90,514 +89,528 @@ public ObjectListing listObjects(ListObjectsRequest listObjectsRequest) MockObjectListing objectListing = new MockObjectListing(); return objectListing; } else { - return null; + throw new TajoInternalError(new UnsupportedException()); } } @Override public ObjectListing listNextBatchOfObjects(ObjectListing previousObjectListing) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public VersionListing listVersions(String bucketName, String prefix) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public VersionListing listNextBatchOfVersions(VersionListing previousVersionListing) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public VersionListing listVersions(String bucketName, String prefix, String keyMarker, String versionIdMarker, String delimiter, Integer maxResults) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public VersionListing listVersions(ListVersionsRequest listVersionsRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public Owner getS3AccountOwner() throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public boolean doesBucketExist(String bucketName) throws AmazonClientException { - return false; + throw new TajoInternalError(new UnsupportedException()); } @Override public List listBuckets() throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public List listBuckets(ListBucketsRequest listBucketsRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public String getBucketLocation(String bucketName) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public String getBucketLocation(GetBucketLocationRequest getBucketLocationRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public Bucket createBucket(CreateBucketRequest createBucketRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public Bucket createBucket(String bucketName) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public Bucket createBucket(String bucketName, com.amazonaws.services.s3.model.Region region) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public Bucket createBucket(String bucketName, String region) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public AccessControlList getObjectAcl(String bucketName, String key) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public AccessControlList getObjectAcl(String bucketName, String key, String versionId) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setObjectAcl(String bucketName, String key, AccessControlList acl) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setObjectAcl(String bucketName, String key, CannedAccessControlList acl) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setObjectAcl(String bucketName, String key, String versionId, AccessControlList acl) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setObjectAcl(String bucketName, String key, String versionId, CannedAccessControlList acl) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public AccessControlList getBucketAcl(String bucketName) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketAcl(SetBucketAclRequest setBucketAclRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public AccessControlList getBucketAcl(GetBucketAclRequest getBucketAclRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketAcl(String bucketName, AccessControlList acl) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketAcl(String bucketName, CannedAccessControlList acl) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public ObjectMetadata getObjectMetadata(String bucketName, String key) - throws AmazonClientException { - if (getObjectMetadataHttpCode != SC_OK) { - AmazonS3Exception exception = new AmazonS3Exception("Failing getObjectMetadata call with " - + getObjectMetadataHttpCode); - exception.setStatusCode(getObjectMetadataHttpCode); - throw exception; - } - return null; + public ObjectMetadata getObjectMetadata(String bucketName, String key) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public ObjectMetadata getObjectMetadata(GetObjectMetadataRequest getObjectMetadataRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public S3Object getObject(String bucketName, String key) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override - public S3Object getObject(GetObjectRequest getObjectRequest) - throws AmazonClientException { - if (getObjectHttpCode != SC_OK) { - AmazonS3Exception exception = new AmazonS3Exception("Failing getObject call with " + getObjectHttpCode); - exception.setStatusCode(getObjectHttpCode); - throw exception; - } - return null; + public S3Object getObject(GetObjectRequest getObjectRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public ObjectMetadata getObject(GetObjectRequest getObjectRequest, File destinationFile) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override - public void deleteBucket(DeleteBucketRequest deleteBucketRequest) - throws AmazonClientException { + public void deleteBucket(DeleteBucketRequest deleteBucketRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public void deleteBucket(String bucketName) - throws AmazonClientException { + public void deleteBucket(String bucketName) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public PutObjectResult putObject(PutObjectRequest putObjectRequest) - throws AmazonClientException { - return null; + public PutObjectResult putObject(PutObjectRequest putObjectRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public PutObjectResult putObject(String bucketName, String key, File file) - throws AmazonClientException { - return null; + public PutObjectResult putObject(String bucketName, String key, File file) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public PutObjectResult putObject(String bucketName, String key, InputStream input, ObjectMetadata metadata) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public CopyObjectResult copyObject(String sourceBucketName, String sourceKey, String destinationBucketName, String destinationKey) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override - public CopyObjectResult copyObject(CopyObjectRequest copyObjectRequest) - throws AmazonClientException { - return null; + public CopyObjectResult copyObject(CopyObjectRequest copyObjectRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public CopyPartResult copyPart(CopyPartRequest copyPartRequest) - throws AmazonClientException { - return null; + public CopyPartResult copyPart(CopyPartRequest copyPartRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public void deleteObject(String bucketName, String key) - throws AmazonClientException { + public void deleteObject(String bucketName, String key) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public void deleteObject(DeleteObjectRequest deleteObjectRequest) - throws AmazonClientException { + public void deleteObject(DeleteObjectRequest deleteObjectRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteObjectsRequest) - throws AmazonClientException { - return null; + public DeleteObjectsResult deleteObjects(DeleteObjectsRequest deleteObjectsRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public void deleteVersion(String bucketName, String key, String versionId) - throws AmazonClientException { + public void deleteVersion(String bucketName, String key, String versionId) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public void deleteVersion(DeleteVersionRequest deleteVersionRequest) - throws AmazonClientException { + public void deleteVersion(DeleteVersionRequest deleteVersionRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override - public BucketLoggingConfiguration getBucketLoggingConfiguration(String bucketName) - throws AmazonClientException { - return null; + public BucketLoggingConfiguration getBucketLoggingConfiguration(String bucketName) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketLoggingConfiguration(SetBucketLoggingConfigurationRequest setBucketLoggingConfigurationRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketVersioningConfiguration getBucketVersioningConfiguration(String bucketName) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketVersioningConfiguration(SetBucketVersioningConfigurationRequest setBucketVersioningConfigurationRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketLifecycleConfiguration getBucketLifecycleConfiguration(String bucketName) { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketLifecycleConfiguration(String bucketName, BucketLifecycleConfiguration bucketLifecycleConfiguration) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketLifecycleConfiguration(SetBucketLifecycleConfigurationRequest setBucketLifecycleConfigurationRequest) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketLifecycleConfiguration(String bucketName) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketLifecycleConfiguration(DeleteBucketLifecycleConfigurationRequest deleteBucketLifecycleConfigurationRequest) { + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketCrossOriginConfiguration getBucketCrossOriginConfiguration(String bucketName) { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketCrossOriginConfiguration(String bucketName, BucketCrossOriginConfiguration bucketCrossOriginConfiguration) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketCrossOriginConfiguration(SetBucketCrossOriginConfigurationRequest setBucketCrossOriginConfigurationRequest) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketCrossOriginConfiguration(String bucketName) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketCrossOriginConfiguration(DeleteBucketCrossOriginConfigurationRequest deleteBucketCrossOriginConfigurationRequest) { + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketTaggingConfiguration getBucketTaggingConfiguration(String bucketName) { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketTaggingConfiguration(String bucketName, BucketTaggingConfiguration bucketTaggingConfiguration) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketTaggingConfiguration(SetBucketTaggingConfigurationRequest setBucketTaggingConfigurationRequest) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketTaggingConfiguration(String bucketName) { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketTaggingConfiguration(DeleteBucketTaggingConfigurationRequest deleteBucketTaggingConfigurationRequest) { + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketNotificationConfiguration getBucketNotificationConfiguration(String bucketName) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketNotificationConfiguration(SetBucketNotificationConfigurationRequest setBucketNotificationConfigurationRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketNotificationConfiguration(String bucketName, BucketNotificationConfiguration bucketNotificationConfiguration) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketWebsiteConfiguration getBucketWebsiteConfiguration(String bucketName) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketWebsiteConfiguration getBucketWebsiteConfiguration(GetBucketWebsiteConfigurationRequest getBucketWebsiteConfigurationRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketWebsiteConfiguration(String bucketName, BucketWebsiteConfiguration configuration) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketWebsiteConfiguration(SetBucketWebsiteConfigurationRequest setBucketWebsiteConfigurationRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketWebsiteConfiguration(String bucketName) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketWebsiteConfiguration(DeleteBucketWebsiteConfigurationRequest deleteBucketWebsiteConfigurationRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketPolicy getBucketPolicy(String bucketName) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public BucketPolicy getBucketPolicy(GetBucketPolicyRequest getBucketPolicyRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketPolicy(String bucketName, String policyText) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void setBucketPolicy(SetBucketPolicyRequest setBucketPolicyRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketPolicy(String bucketName) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void deleteBucketPolicy(DeleteBucketPolicyRequest deleteBucketPolicyRequest) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public URL generatePresignedUrl(String bucketName, String key, Date expiration) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public URL generatePresignedUrl(String bucketName, String key, Date expiration, HttpMethod method) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public URL generatePresignedUrl(GeneratePresignedUrlRequest generatePresignedUrlRequest) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public InitiateMultipartUploadResult initiateMultipartUpload(InitiateMultipartUploadRequest request) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public UploadPartResult uploadPart(UploadPartRequest request) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public PartListing listParts(ListPartsRequest request) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void abortMultipartUpload(AbortMultipartUploadRequest request) throws AmazonClientException { + throw new TajoInternalError(new UnsupportedException()); } @Override public CompleteMultipartUploadResult completeMultipartUpload(CompleteMultipartUploadRequest request) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public MultipartUploadListing listMultipartUploads(ListMultipartUploadsRequest request) throws AmazonClientException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public S3ResponseMetadata getCachedResponseMetadata(AmazonWebServiceRequest request) { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public void restoreObject(RestoreObjectRequest request) throws AmazonServiceException { + throw new TajoInternalError(new UnsupportedException()); } @Override public void restoreObject(String bucketName, String key, int expirationInDays) throws AmazonServiceException { + throw new TajoInternalError(new UnsupportedException()); } - } \ No newline at end of file diff --git a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java index 84650be7e7..6526269731 100644 --- a/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java +++ b/tajo-storage/tajo-storage-s3/src/test/java/org/apache/tajo/storage/s3/MockS3FileSystem.java @@ -21,6 +21,8 @@ import org.apache.hadoop.fs.*; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.util.Progressable; +import org.apache.tajo.exception.TajoInternalError; +import org.apache.tajo.exception.UnsupportedException; import java.io.IOException; import java.net.URI; @@ -36,7 +38,6 @@ public void initialize(URI uri, Configuration conf) throws IOException { this.uri = URI.create(uri.getScheme() + "://" + uri.getAuthority()); } - /** * Return the protocol scheme for the FileSystem. *

@@ -60,18 +61,18 @@ public Path makeQualified(Path path) { @Override public FSDataInputStream open(Path f, int bufferSize) throws IOException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize, short replication, long blockSize, Progressable progress) throws IOException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException { - return null; + throw new TajoInternalError(new UnsupportedException()); } @Override @@ -95,7 +96,7 @@ public void setWorkingDirectory(Path new_dir) { @Override public Path getWorkingDirectory() { - return null; + return new Path(uri); } @Override @@ -110,7 +111,7 @@ public FileStatus getFileStatus(Path f) throws IOException { || f.equals(new Path(TestS3TableSpace.S3A_URI, "test"))) { return new FileStatus(0, true, 1, 0, 0, f); } else { - return null; + throw new TajoInternalError(new UnsupportedException()); } } -} +} \ No newline at end of file From 5346472a171a2b8ce31762a1ad85571928107180 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 11:54:06 +0900 Subject: [PATCH 25/27] Clean up some codes of S3TableSpace --- .../apache/tajo/storage/s3/S3TableSpace.java | 19 ++++++------------- 1 file changed, 6 insertions(+), 13 deletions(-) diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index 13fc3b2095..c50a8bd703 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -80,17 +80,12 @@ public void init(TajoConf tajoConf) throws IOException { ); ClientConfiguration awsConf = new ClientConfiguration(); - awsConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS, - DEFAULT_MAXIMUM_CONNECTIONS)); - boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, - DEFAULT_SECURE_CONNECTIONS); + awsConf.setMaxConnections(conf.getInt(MAXIMUM_CONNECTIONS, DEFAULT_MAXIMUM_CONNECTIONS)); + boolean secureConnections = conf.getBoolean(SECURE_CONNECTIONS, DEFAULT_SECURE_CONNECTIONS); awsConf.setProtocol(secureConnections ? Protocol.HTTPS : Protocol.HTTP); - awsConf.setMaxErrorRetry(conf.getInt(MAX_ERROR_RETRIES, - DEFAULT_MAX_ERROR_RETRIES)); - awsConf.setConnectionTimeout(conf.getInt(ESTABLISH_TIMEOUT, - DEFAULT_ESTABLISH_TIMEOUT)); - awsConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT, - DEFAULT_SOCKET_TIMEOUT)); + awsConf.setMaxErrorRetry(conf.getInt(MAX_ERROR_RETRIES, DEFAULT_MAX_ERROR_RETRIES)); + awsConf.setConnectionTimeout(conf.getInt(ESTABLISH_TIMEOUT, DEFAULT_ESTABLISH_TIMEOUT)); + awsConf.setSocketTimeout(conf.getInt(SOCKET_TIMEOUT, DEFAULT_SOCKET_TIMEOUT)); String proxyHost = conf.getTrimmed(PROXY_HOST,""); int proxyPort = conf.getInt(PROXY_PORT, -1); @@ -110,8 +105,7 @@ public void init(TajoConf tajoConf) throws IOException { String proxyUsername = conf.getTrimmed(PROXY_USERNAME); String proxyPassword = conf.getTrimmed(PROXY_PASSWORD); if ((proxyUsername == null) != (proxyPassword == null)) { - String msg = "Proxy error: " + PROXY_USERNAME + " or " + - PROXY_PASSWORD + " set without the other."; + String msg = "Proxy error: " + PROXY_USERNAME + " or " + PROXY_PASSWORD + " set without the other."; LOG.error(msg); throw new IllegalArgumentException(msg); } @@ -143,7 +137,6 @@ public void init(TajoConf tajoConf) throws IOException { } maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS); - s3Enabled = true; } catch (NoClassDefFoundError defFoundError) { // If the version of hadoop is less than 2.6.0, hadoop doesn't include aws dependencies because it doesn't provide From 3728219e9aa96409ea99a465185cae0965535a69 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 12:00:24 +0900 Subject: [PATCH 26/27] Remove httpclient dependency --- tajo-project/pom.xml | 1 - tajo-storage/tajo-storage-s3/pom.xml | 14 +++++++------- 2 files changed, 7 insertions(+), 8 deletions(-) diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml index 0352878e18..9def61986a 100644 --- a/tajo-project/pom.xml +++ b/tajo-project/pom.xml @@ -41,7 +41,6 @@ 6.1.26 1.8.1 1.7.4 - 4.5 ${project.parent.relativePath}/.. src/main/hadoop-${hadoop.version} diff --git a/tajo-storage/tajo-storage-s3/pom.xml b/tajo-storage/tajo-storage-s3/pom.xml index 1394470baf..b0e248ed14 100644 --- a/tajo-storage/tajo-storage-s3/pom.xml +++ b/tajo-storage/tajo-storage-s3/pom.xml @@ -120,19 +120,19 @@ com.amazonaws aws-java-sdk ${aws-java-sdk.version} - provided + test - junit - junit - test + com.amazonaws + aws-java-sdk + ${aws-java-sdk.version} + provided - org.apache.httpcomponents - httpclient - ${httpclient.version} + junit + junit test From 15e4978742e03252d1807e519f3dd16f4a636341 Mon Sep 17 00:00:00 2001 From: JaeHwa Jung Date: Mon, 30 May 2016 14:44:10 +0900 Subject: [PATCH 27/27] Add httpclient dependency again and remove exception throwing codes --- tajo-project/pom.xml | 1 + tajo-storage/tajo-storage-s3/pom.xml | 10 +++++----- .../java/org/apache/tajo/storage/s3/S3TableSpace.java | 7 ++----- 3 files changed, 8 insertions(+), 10 deletions(-) diff --git a/tajo-project/pom.xml b/tajo-project/pom.xml index 9def61986a..0352878e18 100644 --- a/tajo-project/pom.xml +++ b/tajo-project/pom.xml @@ -41,6 +41,7 @@ 6.1.26 1.8.1 1.7.4 + 4.5 ${project.parent.relativePath}/.. src/main/hadoop-${hadoop.version} diff --git a/tajo-storage/tajo-storage-s3/pom.xml b/tajo-storage/tajo-storage-s3/pom.xml index b0e248ed14..69564111f9 100644 --- a/tajo-storage/tajo-storage-s3/pom.xml +++ b/tajo-storage/tajo-storage-s3/pom.xml @@ -120,14 +120,14 @@ com.amazonaws aws-java-sdk ${aws-java-sdk.version} - test + provided - com.amazonaws - aws-java-sdk - ${aws-java-sdk.version} - provided + org.apache.httpcomponents + httpclient + ${httpclient.version} + test diff --git a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java index c50a8bd703..ee6af31103 100644 --- a/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java +++ b/tajo-storage/tajo-storage-s3/src/main/java/org/apache/tajo/storage/s3/S3TableSpace.java @@ -107,7 +107,6 @@ public void init(TajoConf tajoConf) throws IOException { if ((proxyUsername == null) != (proxyPassword == null)) { String msg = "Proxy error: " + PROXY_USERNAME + " or " + PROXY_PASSWORD + " set without the other."; LOG.error(msg); - throw new IllegalArgumentException(msg); } awsConf.setProxyUsername(proxyUsername); awsConf.setProxyPassword(proxyPassword); @@ -121,7 +120,6 @@ public void init(TajoConf tajoConf) throws IOException { } else if (proxyPort >= 0) { String msg = "Proxy error: " + PROXY_PORT + " set without " + PROXY_HOST; LOG.error(msg); - throw new IllegalArgumentException(msg); } s3 = new AmazonS3Client(credentials, awsConf); @@ -132,16 +130,15 @@ public void init(TajoConf tajoConf) throws IOException { } catch (IllegalArgumentException e) { String msg = "Incorrect endpoint: " + e.getMessage(); LOG.error(msg); - throw new IllegalArgumentException(msg, e); } } maxKeys = conf.getInt(MAX_PAGING_KEYS, DEFAULT_MAX_PAGING_KEYS); s3Enabled = true; - } catch (NoClassDefFoundError defFoundError) { + } catch (NoClassDefFoundError e) { // If the version of hadoop is less than 2.6.0, hadoop doesn't include aws dependencies because it doesn't provide // S3AFileSystem. In this case, tajo never uses aws s3 api directly. - LOG.warn(defFoundError); + LOG.warn(e); s3Enabled = false; } catch (Exception e) { throw new TajoInternalError(e);