Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -107,11 +107,15 @@ private Constants() {
public static final String MAX_PAGING_KEYS = "fs.s3a.paging.maximum";
public static final int DEFAULT_MAX_PAGING_KEYS = 5000;

// the maximum number of threads to allow in the pool used by TransferManager
// the maximum number of threads to allow in the pool used by TransferManager for uploads
public static final String MAX_THREADS = "fs.s3a.threads.max";
public static final int DEFAULT_MAX_THREADS = 10;

// the time an idle thread waits before terminating
// the maximum number of threads to allow in the pool used by TransferManager for copies
public static final String COPY_MAX_THREADS = "fs.s3a.copy.threads.max";
public static final int DEFAULT_COPY_MAX_THREADS = 25;

// the time an idle thread waits before terminating for uploads and copies
public static final String KEEPALIVE_TIME = "fs.s3a.threads.keepalivetime";
public static final int DEFAULT_KEEPALIVE_TIME = 60;

Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,65 @@
/*
* 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.hadoop.fs.s3a;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Will need apache license text at top.

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Whoops, always forget those. Fixed.


import com.amazonaws.event.ProgressListener;
import com.amazonaws.services.s3.transfer.Copy;


/**
* A wrapper around a {@link Copy} object, the source key for the copy, the destination for the copy, the length of the
* key copied, and the {@link ProgressListener.ExceptionReporter} for the copy.
*/
class CopyContext {

private final Copy copy;
private final String srcKey;
private final String destKey;
private final long length;
private final ProgressListener.ExceptionReporter progressListener;

CopyContext(Copy copy, String srcKey, String destKey, long length,
ProgressListener.ExceptionReporter progressListener) {
this.copy = copy;
this.srcKey = srcKey;
this.destKey = destKey;
this.length = length;
this.progressListener = progressListener;
}

Copy getCopy() {
return copy;
}

String getSrcKey() {
return srcKey;
}

String getDestKey() {
return destKey;
}

long getLength() {
return length;
}

ProgressListener.ExceptionReporter getProgressListener() {
return progressListener;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,117 @@
/*
* 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.hadoop.fs.s3a;
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Ditto here.. new files need the apache license text at top

Copy link
Contributor Author

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Fixed


import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;

import com.amazonaws.services.s3.AmazonS3;
import com.amazonaws.services.s3.transfer.TransferManager;
import com.amazonaws.services.s3.transfer.TransferManagerConfiguration;

import org.apache.hadoop.conf.Configuration;

import static org.apache.hadoop.fs.s3a.Constants.COPY_MAX_THREADS;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_COPY_MAX_THREADS;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_KEEPALIVE_TIME;
import static org.apache.hadoop.fs.s3a.Constants.DEFAULT_MAX_THREADS;
import static org.apache.hadoop.fs.s3a.Constants.KEEPALIVE_TIME;
import static org.apache.hadoop.fs.s3a.Constants.MAX_THREADS;
import static org.apache.hadoop.fs.s3a.S3AUtils.getMaxThreads;
import static org.apache.hadoop.fs.s3a.S3AUtils.longOption;


/**
* A wrapper around a {@link TransferManager} that lazily initializes the {@link TransferManager}.
*/
class LazyTransferManager {

private final AmazonS3 s3;
private final TransferManagerConfiguration transferConfiguration;
private final int maxThreads;
private final long keepAliveTime;
private final String threadPoolName;

private TransferManager transferManager;
private ExecutorService executorService;

private LazyTransferManager(AmazonS3 s3, TransferManagerConfiguration transferConfiguration, int maxThreads,
long keepAliveTime, String threadPoolName) {
this.s3 = s3;
this.transferConfiguration = transferConfiguration;
this.maxThreads = maxThreads;
this.keepAliveTime = keepAliveTime;
this.threadPoolName = threadPoolName;
}

synchronized TransferManager get() {
if (transferManager == null) {
transferManager = createTransferManager();
return transferManager;
}
return transferManager;
}

boolean isInitialized() {
return transferManager == null;
}

ExecutorService getExecutorService() {
return executorService;
}

private TransferManager createTransferManager() {
executorService = new ThreadPoolExecutor(
maxThreads, Integer.MAX_VALUE,
keepAliveTime, TimeUnit.SECONDS,
new LinkedBlockingQueue<>(),
BlockingThreadPoolExecutorService.newDaemonThreadFactory(
threadPoolName));

TransferManager transferManager = new TransferManager(s3, executorService);
transferManager.setConfiguration(transferConfiguration);
return transferManager;
}

static LazyTransferManager createLazyUploadTransferManager(AmazonS3 s3, Configuration conf, long partSize,
long multiPartThreshold) {
int maxThreads = getMaxThreads(conf, MAX_THREADS, DEFAULT_MAX_THREADS);
long keepAliveTime = longOption(conf, KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME, 0);

TransferManagerConfiguration transferConfiguration = new TransferManagerConfiguration();
transferConfiguration.setMinimumUploadPartSize(partSize);
transferConfiguration.setMultipartUploadThreshold(multiPartThreshold);

return new LazyTransferManager(s3, transferConfiguration, maxThreads, keepAliveTime, "s3a-upload-unbounded");
}

static LazyTransferManager createLazyCopyTransferManager(AmazonS3 s3, Configuration conf, long partSize,
long multiPartThreshold) {
int maxThreads = getMaxThreads(conf, COPY_MAX_THREADS, DEFAULT_COPY_MAX_THREADS);
long keepAliveTime = longOption(conf, KEEPALIVE_TIME, DEFAULT_KEEPALIVE_TIME, 0);

TransferManagerConfiguration transferConfiguration = new TransferManagerConfiguration();
transferConfiguration.setMultipartCopyPartSize(partSize);
transferConfiguration.setMultipartCopyThreshold(multiPartThreshold);

return new LazyTransferManager(s3, transferConfiguration, maxThreads, keepAliveTime, "s3a-copy-unbounded");
}
}
Loading