diff --git a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml index 11b092674cf..a8f45a7f3a2 100644 --- a/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml +++ b/hadoop-cloud-storage-project/hadoop-cloud-storage/pom.xml @@ -133,5 +133,10 @@ hadoop-cos compile + + org.apache.hadoop + hadoop-huaweicloud + compile + diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/dev-support/findbugs-exclude.xml b/hadoop-cloud-storage-project/hadoop-huaweicloud/dev-support/findbugs-exclude.xml new file mode 100644 index 00000000000..40d78d0cd6c --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/dev-support/findbugs-exclude.xml @@ -0,0 +1,18 @@ + + + diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml b/hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml new file mode 100755 index 00000000000..d5fe11e9e3f --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml @@ -0,0 +1,180 @@ + + + + 4.0.0 + + org.apache.hadoop + hadoop-project + 3.4.0-SNAPSHOT + ../../hadoop-project + + hadoop-huaweicloud + 3.4.0-SNAPSHOT + Apache Hadoop OBS support + + This module contains code to support integration with OBS. + It also declares the dependencies needed to work with OBS services. + + jar + + UTF-8 + true + 3.20.4.2 + + + + + tests-off + + + src/test/resources/auth-keys.xml + + + + true + + + + tests-on + + + src/test/resources/auth-keys.xml + + + + false + + + + + + + + org.codehaus.mojo + findbugs-maven-plugin + + true + true + ${basedir}/dev-support/findbugs-exclude.xml + + Max + + + + org.apache.maven.plugins + maven-surefire-plugin + + 3600 + + + + org.apache.maven.plugins + maven-dependency-plugin + + + deplist + compile + + list + + + ${project.basedir}/target/hadoop-cloud-storage-deps/${project.artifactId}.cloud-storage-optional.txt + + + + + + + + + org.apache.hadoop + hadoop-common + provided + + + jdk.tools + jdk.tools + + + + + org.apache.hadoop + hadoop-common + test + test-jar + + + junit + junit + 4.12 + test + + + org.mockito + mockito-all + 1.10.19 + test + + + org.apache.hadoop + hadoop-mapreduce-client-jobclient + test + + + org.apache.hadoop + hadoop-yarn-server-tests + test + test-jar + + + org.apache.hadoop + hadoop-mapreduce-examples + test + jar + + + org.apache.hadoop + hadoop-distcp + test + + + org.apache.hadoop + hadoop-distcp + test + test-jar + + + com.huaweicloud + esdk-obs-java + ${esdk.version} + + + okio + com.squareup.okio + + + + + org.powermock + powermock-api-mockito + 1.7.4 + test + + + org.powermock + powermock-module-junit4 + 1.7.4 + test + + + \ No newline at end of file diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BasicSessionCredential.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BasicSessionCredential.java new file mode 100644 index 00000000000..7110af101ae --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BasicSessionCredential.java @@ -0,0 +1,45 @@ +/* + * 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.obs; + +/** + * Interface class for getting basic session credential. + */ +public interface BasicSessionCredential { + /** + * Get OBS access key. + * + * @return OBS access key + */ + String getOBSAccessKeyId(); + + /** + * Get OBS secret key. + * + * @return OBS secret key + */ + String getOBSSecretKey(); + + /** + * Get session token. + * + * @return session token + */ + String getSessionToken(); +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultOBSClientFactory.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultOBSClientFactory.java new file mode 100644 index 00000000000..e46a21bba7a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultOBSClientFactory.java @@ -0,0 +1,361 @@ +/* + * 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.obs; + +import com.obs.services.IObsCredentialsProvider; +import com.obs.services.ObsClient; +import com.obs.services.ObsConfiguration; +import com.obs.services.internal.ext.ExtObsConfiguration; +import com.obs.services.model.AuthTypeEnum; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.conf.Configured; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.net.URI; +import java.util.Optional; + +/** + * The default factory implementation, which calls the OBS SDK to configure and + * create an {@link ObsClient} that communicates with the OBS service. + */ +class DefaultOBSClientFactory extends Configured implements OBSClientFactory { + + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + DefaultOBSClientFactory.class); + + /** + * Initializes all OBS SDK settings related to connection management. + * + * @param conf Hadoop configuration + * @param obsConf OBS SDK configuration + */ + @SuppressWarnings("deprecation") + private static void initConnectionSettings(final Configuration conf, + final ExtObsConfiguration obsConf) { + + obsConf.setMaxConnections( + OBSCommonUtils.intOption(conf, OBSConstants.MAXIMUM_CONNECTIONS, + OBSConstants.DEFAULT_MAXIMUM_CONNECTIONS, + 1)); + + boolean secureConnections = conf.getBoolean( + OBSConstants.SECURE_CONNECTIONS, + OBSConstants.DEFAULT_SECURE_CONNECTIONS); + + obsConf.setHttpsOnly(secureConnections); + + obsConf.setMaxErrorRetry( + OBSCommonUtils.intOption(conf, OBSConstants.MAX_ERROR_RETRIES, + OBSConstants.DEFAULT_MAX_ERROR_RETRIES, 0)); + + obsConf.setConnectionTimeout( + OBSCommonUtils.intOption(conf, OBSConstants.ESTABLISH_TIMEOUT, + OBSConstants.DEFAULT_ESTABLISH_TIMEOUT, 0)); + + obsConf.setSocketTimeout( + OBSCommonUtils.intOption(conf, OBSConstants.SOCKET_TIMEOUT, + OBSConstants.DEFAULT_SOCKET_TIMEOUT, 0)); + + obsConf.setIdleConnectionTime( + OBSCommonUtils.intOption(conf, OBSConstants.IDLE_CONNECTION_TIME, + OBSConstants.DEFAULT_IDLE_CONNECTION_TIME, + 1)); + + obsConf.setMaxIdleConnections( + OBSCommonUtils.intOption(conf, OBSConstants.MAX_IDLE_CONNECTIONS, + OBSConstants.DEFAULT_MAX_IDLE_CONNECTIONS, + 1)); + + obsConf.setReadBufferSize( + OBSCommonUtils.intOption(conf, OBSConstants.READ_BUFFER_SIZE, + OBSConstants.DEFAULT_READ_BUFFER_SIZE, + -1)); // to be + // modified + obsConf.setWriteBufferSize( + OBSCommonUtils.intOption(conf, OBSConstants.WRITE_BUFFER_SIZE, + OBSConstants.DEFAULT_WRITE_BUFFER_SIZE, + -1)); // to be + // modified + obsConf.setUploadStreamRetryBufferSize( + OBSCommonUtils.intOption(conf, + OBSConstants.UPLOAD_STREAM_RETRY_SIZE, + OBSConstants.DEFAULT_UPLOAD_STREAM_RETRY_SIZE, 1)); + + obsConf.setSocketReadBufferSize( + OBSCommonUtils.intOption(conf, OBSConstants.SOCKET_RECV_BUFFER, + OBSConstants.DEFAULT_SOCKET_RECV_BUFFER, -1)); + obsConf.setSocketWriteBufferSize( + OBSCommonUtils.intOption(conf, OBSConstants.SOCKET_SEND_BUFFER, + OBSConstants.DEFAULT_SOCKET_SEND_BUFFER, -1)); + + obsConf.setKeepAlive(conf.getBoolean(OBSConstants.KEEP_ALIVE, + OBSConstants.DEFAULT_KEEP_ALIVE)); + obsConf.setValidateCertificate( + conf.getBoolean(OBSConstants.VALIDATE_CERTIFICATE, + OBSConstants.DEFAULT_VALIDATE_CERTIFICATE)); + obsConf.setVerifyResponseContentType( + conf.getBoolean(OBSConstants.VERIFY_RESPONSE_CONTENT_TYPE, + OBSConstants.DEFAULT_VERIFY_RESPONSE_CONTENT_TYPE)); + obsConf.setCname( + conf.getBoolean(OBSConstants.CNAME, OBSConstants.DEFAULT_CNAME)); + obsConf.setIsStrictHostnameVerification( + conf.getBoolean(OBSConstants.STRICT_HOSTNAME_VERIFICATION, + OBSConstants.DEFAULT_STRICT_HOSTNAME_VERIFICATION)); + + // sdk auth type negotiation enable + obsConf.setAuthTypeNegotiation( + conf.getBoolean(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE, + OBSConstants.DEFAULT_SDK_AUTH_TYPE_NEGOTIATION_ENABLE)); + // set SDK AUTH TYPE to OBS when auth type negotiation unenabled + if (!obsConf.isAuthTypeNegotiation()) { + obsConf.setAuthType(AuthTypeEnum.OBS); + } + + // okhttp retryOnConnectionFailure switch, default set to true + obsConf.retryOnConnectionFailureInOkhttp( + conf.getBoolean(OBSConstants.SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE, + OBSConstants.DEFAULT_SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE)); + + // sdk max retry times on unexpected end of stream exception, + // default: -1 don't retry + int retryTime = conf.getInt( + OBSConstants.SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION, + OBSConstants.DEFAULT_SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION); + if (retryTime > 0 + && retryTime < OBSConstants.DEFAULT_MAX_SDK_CONNECTION_RETRY_TIMES + || !obsConf.isRetryOnConnectionFailureInOkhttp() && retryTime < 0) { + retryTime = OBSConstants.DEFAULT_MAX_SDK_CONNECTION_RETRY_TIMES; + } + obsConf.setMaxRetryOnUnexpectedEndException(retryTime); + } + + /** + * Initializes OBS SDK proxy support if configured. + * + * @param conf Hadoop configuration + * @param obsConf OBS SDK configuration + * @throws IllegalArgumentException if misconfigured + * @throws IOException on any failure to initialize proxy + */ + private static void initProxySupport(final Configuration conf, + final ExtObsConfiguration obsConf) + throws IllegalArgumentException, IOException { + String proxyHost = conf.getTrimmed(OBSConstants.PROXY_HOST, ""); + int proxyPort = conf.getInt(OBSConstants.PROXY_PORT, -1); + + if (!proxyHost.isEmpty() && proxyPort < 0) { + if (conf.getBoolean(OBSConstants.SECURE_CONNECTIONS, + OBSConstants.DEFAULT_SECURE_CONNECTIONS)) { + LOG.warn("Proxy host set without port. Using HTTPS default " + + OBSConstants.DEFAULT_HTTPS_PORT); + obsConf.getHttpProxy() + .setProxyPort(OBSConstants.DEFAULT_HTTPS_PORT); + } else { + LOG.warn("Proxy host set without port. Using HTTP default " + + OBSConstants.DEFAULT_HTTP_PORT); + obsConf.getHttpProxy() + .setProxyPort(OBSConstants.DEFAULT_HTTP_PORT); + } + } + String proxyUsername = conf.getTrimmed(OBSConstants.PROXY_USERNAME); + String proxyPassword = null; + char[] proxyPass = conf.getPassword(OBSConstants.PROXY_PASSWORD); + if (proxyPass != null) { + proxyPassword = new String(proxyPass).trim(); + } + if ((proxyUsername == null) != (proxyPassword == null)) { + String msg = + "Proxy error: " + OBSConstants.PROXY_USERNAME + " or " + + OBSConstants.PROXY_PASSWORD + + " set without the other."; + LOG.error(msg); + throw new IllegalArgumentException(msg); + } + obsConf.setHttpProxy(proxyHost, proxyPort, proxyUsername, + proxyPassword); + if (LOG.isDebugEnabled()) { + LOG.debug( + "Using proxy server {}:{} as user {} on " + + "domain {} as workstation {}", + obsConf.getHttpProxy().getProxyAddr(), + obsConf.getHttpProxy().getProxyPort(), + obsConf.getHttpProxy().getProxyUName(), + obsConf.getHttpProxy().getDomain(), + obsConf.getHttpProxy().getWorkstation()); + } + } + + /** + * Creates an {@link ObsClient} from the established configuration. + * + * @param conf Hadoop configuration + * @param obsConf ObsConfiguration + * @param name URL + * @return ObsClient client + * @throws IOException on any failure to create Huawei OBS client + */ + private static ObsClient createHuaweiObsClient(final Configuration conf, + final ObsConfiguration obsConf, final URI name) + throws IOException { + Class credentialsProviderClass; + BasicSessionCredential credentialsProvider; + ObsClient obsClient; + + try { + credentialsProviderClass = conf.getClass( + OBSConstants.OBS_CREDENTIALS_PROVIDER, null); + } catch (RuntimeException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException( + "From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + + c, c); + } + + if (credentialsProviderClass == null) { + return createObsClientWithoutCredentialsProvider(conf, obsConf, + name); + } + + try { + Constructor cons = + credentialsProviderClass.getDeclaredConstructor(URI.class, + Configuration.class); + credentialsProvider = (BasicSessionCredential) cons.newInstance( + name, conf); + } catch (NoSuchMethodException + | SecurityException + | IllegalAccessException + | InstantiationException + | InvocationTargetException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException( + "From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + + c, c); + } + + String sessionToken = credentialsProvider.getSessionToken(); + String ak = credentialsProvider.getOBSAccessKeyId(); + String sk = credentialsProvider.getOBSSecretKey(); + String endPoint = conf.getTrimmed(OBSConstants.ENDPOINT, ""); + obsConf.setEndPoint(endPoint); + if (sessionToken != null && sessionToken.length() != 0) { + obsClient = new ObsClient(ak, sk, sessionToken, obsConf); + } else { + obsClient = new ObsClient(ak, sk, obsConf); + } + return obsClient; + } + + private static ObsClient createObsClientWithoutCredentialsProvider( + final Configuration conf, final ObsConfiguration obsConf, + final URI name) throws IOException { + ObsClient obsClient; + OBSLoginHelper.Login creds = OBSCommonUtils.getOBSAccessKeys(name, + conf); + + String ak = creds.getUser(); + String sk = creds.getPassword(); + String token = creds.getToken(); + + String endPoint = conf.getTrimmed(OBSConstants.ENDPOINT, ""); + obsConf.setEndPoint(endPoint); + + if (!StringUtils.isEmpty(ak) || !StringUtils.isEmpty(sk)) { + obsClient = new ObsClient(ak, sk, token, obsConf); + return obsClient; + } + + Class securityProviderClass; + try { + securityProviderClass = conf.getClass( + OBSConstants.OBS_SECURITY_PROVIDER, null); + LOG.info("From option {} get {}", + OBSConstants.OBS_SECURITY_PROVIDER, securityProviderClass); + } catch (RuntimeException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException( + "From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, + c); + } + + if (securityProviderClass == null) { + obsClient = new ObsClient(ak, sk, token, obsConf); + return obsClient; + } + + IObsCredentialsProvider securityProvider; + try { + Optional cons = tryGetConstructor( + securityProviderClass, + new Class[] {URI.class, Configuration.class}); + + if (cons.isPresent()) { + securityProvider = (IObsCredentialsProvider) cons.get() + .newInstance(name, conf); + } else { + securityProvider + = (IObsCredentialsProvider) securityProviderClass + .getDeclaredConstructor().newInstance(); + } + + } catch (NoSuchMethodException + | IllegalAccessException + | InstantiationException + | InvocationTargetException + | RuntimeException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException( + "From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, + c); + } + obsClient = new ObsClient(securityProvider, obsConf); + + return obsClient; + } + + public static Optional tryGetConstructor(final Class mainClss, + final Class[] args) { + try { + Constructor constructor = mainClss.getDeclaredConstructor(args); + return Optional.ofNullable(constructor); + } catch (NoSuchMethodException e) { + // ignore + return Optional.empty(); + } + } + + @Override + public ObsClient createObsClient(final URI name) throws IOException { + Configuration conf = getConf(); + ExtObsConfiguration obsConf = new ExtObsConfiguration(); + initConnectionSettings(conf, obsConf); + initProxySupport(conf, obsConf); + + return createHuaweiObsClient(conf, obsConf, name); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java new file mode 100644 index 00000000000..7384251b708 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java @@ -0,0 +1,40 @@ +/* + * 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.obs; + +import java.io.IOException; + +/** + * OBS file conflict exception. + */ +class FileConflictException extends IOException { + private static final long serialVersionUID = -897856973823710492L; + + /** + * Constructs a FileConflictException with the specified detail + * message. The string s can be retrieved later by the + * {@link Throwable#getMessage} + * method of class java.lang.Throwable. + * + * @param s the detail message. + */ + FileConflictException(final String s) { + super(s); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBS.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBS.java new file mode 100644 index 00000000000..3f05f007ee5 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBS.java @@ -0,0 +1,53 @@ +/* + * 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.obs; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +/** + * OBS implementation of AbstractFileSystem, which delegates to the {@link + * OBSFileSystem}. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +public final class OBS extends DelegateToFileSystem { + + /** + * @param theUri URI of the file system + * @param conf Configuration for the file system + * @throws IOException on any failure to initialize this instance + * @throws URISyntaxException theUri has syntax error + */ + public OBS(final URI theUri, final Configuration conf) + throws IOException, URISyntaxException { + super(theUri, new OBSFileSystem(), conf, "obs", false); + } + + @Override + public int getUriDefaultPort() { + return OBSConstants.OBS_DEFAULT_PORT; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java new file mode 100644 index 00000000000..d1967402776 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java @@ -0,0 +1,814 @@ +/* + * 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.obs; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.Futures; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListenableFuture; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.MoreExecutors; +import com.obs.services.exception.ObsException; +import com.obs.services.model.CompleteMultipartUploadResult; +import com.obs.services.model.PartEtag; +import com.obs.services.model.PutObjectRequest; +import com.obs.services.model.UploadPartRequest; +import com.obs.services.model.UploadPartResult; +import com.obs.services.model.fs.WriteFileRequest; +import com.sun.istack.NotNull; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.Syncable; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * OBS output stream based on block buffering. + *

+ * Upload files/parts directly via different buffering mechanisms: including + * memory and disk. + * + *

If the stream is closed and no update has started, then the upload is + * instead done as a single PUT operation. + * + *

Unstable: statistics and error handling might evolve. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +class OBSBlockOutputStream extends OutputStream implements Syncable { + + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + OBSBlockOutputStream.class); + + /** + * Owner FileSystem. + */ + private final OBSFileSystem fs; + + /** + * Key of the object being uploaded. + */ + private final String key; + + /** + * Length of object. + */ + private long objectLen; + + /** + * Size of all blocks. + */ + private final int blockSize; + + /** + * Callback for progress. + */ + private final ListeningExecutorService executorService; + + /** + * Factory for creating blocks. + */ + private final OBSDataBlocks.BlockFactory blockFactory; + + /** + * Preallocated byte buffer for writing single characters. + */ + private final byte[] singleCharWrite = new byte[1]; + + /** + * Closed flag. + */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** + * Has exception flag. + */ + private final AtomicBoolean hasException = new AtomicBoolean(false); + + /** + * Has flushed flag. + */ + private final AtomicBoolean appendAble; + + /** + * Multipart upload details; null means none started. + */ + private MultiPartUpload multiPartUpload; + + /** + * Current data block. Null means none currently active. + */ + private OBSDataBlocks.DataBlock activeBlock; + + /** + * Count of blocks uploaded. + */ + private long blockCount = 0; + + /** + * Write operation helper; encapsulation of the filesystem operations. + */ + private OBSWriteOperationHelper writeOperationHelper; + + /** + * Flag for mocking upload part error. + */ + private boolean mockUploadPartError = false; + + /** + * An OBS output stream which uploads partitions in a separate pool of + * threads; different {@link OBSDataBlocks.BlockFactory} instances can control + * where data is buffered. + * + * @param owner OBSFilesystem + * @param obsObjectKey OBS object to work on + * @param objLen object length + * @param execService the executor service to use to schedule work + * @param isAppendable if append is supported + * @throws IOException on any problem + */ + OBSBlockOutputStream( + final OBSFileSystem owner, + final String obsObjectKey, + final long objLen, + final ExecutorService execService, + final boolean isAppendable) + throws IOException { + this.appendAble = new AtomicBoolean(isAppendable); + this.fs = owner; + this.key = obsObjectKey; + this.objectLen = objLen; + this.blockFactory = owner.getBlockFactory(); + this.blockSize = (int) owner.getPartSize(); + this.writeOperationHelper = owner.getWriteHelper(); + Preconditions.checkArgument( + owner.getPartSize() >= OBSConstants.MULTIPART_MIN_SIZE, + "Block size is too small: %d", owner.getPartSize()); + this.executorService = MoreExecutors.listeningDecorator( + execService); + this.multiPartUpload = null; + // create that first block. This guarantees that an open + close + // sequence writes a 0-byte entry. + createBlockIfNeeded(); + LOG.debug( + "Initialized OBSBlockOutputStream for {}" + " output to {}", + owner.getWriteHelper(), + activeBlock); + } + + /** + * Demand create a destination block. + * + * @return the active block; null if there isn't one. + * @throws IOException on any failure to create + */ + private synchronized OBSDataBlocks.DataBlock createBlockIfNeeded() + throws IOException { + if (activeBlock == null) { + blockCount++; + if (blockCount >= OBSConstants.MAX_MULTIPART_COUNT) { + LOG.warn( + "Number of partitions in stream exceeds limit for OBS: " + + OBSConstants.MAX_MULTIPART_COUNT + + " write may fail."); + } + activeBlock = blockFactory.create(blockCount, this.blockSize); + } + return activeBlock; + } + + /** + * Synchronized accessor to the active block. + * + * @return the active block; null if there isn't one. + */ + synchronized OBSDataBlocks.DataBlock getActiveBlock() { + return activeBlock; + } + + /** + * Set mock error. + * + * @param isException mock error + */ + @VisibleForTesting + public void mockPutPartError(final boolean isException) { + this.mockUploadPartError = isException; + } + + /** + * Predicate to query whether or not there is an active block. + * + * @return true if there is an active block. + */ + private synchronized boolean hasActiveBlock() { + return activeBlock != null; + } + + /** + * Clear the active block. + */ + private synchronized void clearActiveBlock() { + if (activeBlock != null) { + LOG.debug("Clearing active block"); + } + activeBlock = null; + } + + /** + * Check for the filesystem being open. + * + * @throws IOException if the filesystem is closed. + */ + private void checkOpen() throws IOException { + if (closed.get()) { + throw new IOException( + "Filesystem " + writeOperationHelper.toString(key) + " closed"); + } + } + + /** + * The flush operation does not trigger an upload; that awaits the next block + * being full. What it does do is call {@code flush() } on the current block, + * leaving it to choose how to react. + * + * @throws IOException Any IO problem. + */ + @Override + public synchronized void flush() throws IOException { + checkOpen(); + OBSDataBlocks.DataBlock dataBlock = getActiveBlock(); + if (dataBlock != null) { + dataBlock.flush(); + } + } + + /** + * Writes a byte to the destination. If this causes the buffer to reach its + * limit, the actual upload is submitted to the threadpool. + * + * @param b the int of which the lowest byte is written + * @throws IOException on any problem + */ + @Override + public synchronized void write(final int b) throws IOException { + singleCharWrite[0] = (byte) b; + write(singleCharWrite, 0, 1); + } + + /** + * Writes a range of bytes from to the memory buffer. If this causes the + * buffer to reach its limit, the actual upload is submitted to the threadpool + * and the remainder of the array is written to memory (recursively). + * + * @param source byte array containing + * @param offset offset in array where to start + * @param len number of bytes to be written + * @throws IOException on any problem + */ + @Override + public synchronized void write(@NotNull final byte[] source, + final int offset, final int len) + throws IOException { + if (hasException.get()) { + String closeWarning = String.format( + "write has error. bs : pre upload obs[%s] has error.", key); + LOG.warn(closeWarning); + throw new IOException(closeWarning); + } + OBSDataBlocks.validateWriteArgs(source, offset, len); + checkOpen(); + if (len == 0) { + return; + } + + OBSDataBlocks.DataBlock block = createBlockIfNeeded(); + int written = block.write(source, offset, len); + int remainingCapacity = block.remainingCapacity(); + try { + innerWrite(source, offset, len, written, remainingCapacity); + } catch (IOException e) { + LOG.error( + "Write data for key {} of bucket {} error, error message {}", + key, fs.getBucket(), + e.getMessage()); + throw e; + } + } + + private synchronized void innerWrite(final byte[] source, final int offset, + final int len, + final int written, final int remainingCapacity) + throws IOException { + + if (written < len) { + // not everything was written the block has run out + // of capacity + // Trigger an upload then process the remainder. + LOG.debug( + "writing more data than block has capacity -triggering upload"); + if (appendAble.get()) { + // to write a buffer then append to obs + LOG.debug("[Append] open stream and single write size {} " + + "greater than buffer size {}, append buffer to obs.", + len, blockSize); + flushCurrentBlock(); + } else { + // block output stream logic, multi-part upload + uploadCurrentBlock(); + } + // tail recursion is mildly expensive, but given buffer sizes + // must be MB. it's unlikely to recurse very deeply. + this.write(source, offset + written, len - written); + } else { + if (remainingCapacity == 0) { + // the whole buffer is done, trigger an upload + if (appendAble.get()) { + // to write a buffer then append to obs + LOG.debug("[Append] open stream and already write size " + + "equal to buffer size {}, append buffer to obs.", + blockSize); + flushCurrentBlock(); + } else { + // block output stream logic, multi-part upload + uploadCurrentBlock(); + } + } + } + } + + /** + * Start an asynchronous upload of the current block. + * + * @throws IOException Problems opening the destination for upload or + * initializing the upload. + */ + private synchronized void uploadCurrentBlock() throws IOException { + Preconditions.checkState(hasActiveBlock(), "No active block"); + LOG.debug("Writing block # {}", blockCount); + + try { + if (multiPartUpload == null) { + LOG.debug("Initiating Multipart upload"); + multiPartUpload = new MultiPartUpload(); + } + multiPartUpload.uploadBlockAsync(getActiveBlock()); + } catch (IOException e) { + hasException.set(true); + LOG.error("Upload current block on ({}/{}) failed.", fs.getBucket(), + key, e); + throw e; + } finally { + // set the block to null, so the next write will create a new block. + clearActiveBlock(); + } + } + + /** + * Close the stream. + * + *

This will not return until the upload is complete or the attempt to + * perform the upload has failed. Exceptions raised in this method are + * indicative that the write has failed and data is at risk of being lost. + * + * @throws IOException on any failure. + */ + @Override + public synchronized void close() throws IOException { + if (closed.getAndSet(true)) { + // already closed + LOG.debug("Ignoring close() as stream is already closed"); + return; + } + if (hasException.get()) { + String closeWarning = String.format( + "closed has error. bs : pre write obs[%s] has error.", key); + LOG.warn(closeWarning); + throw new IOException(closeWarning); + } + // do upload + completeCurrentBlock(); + + // clear + clearHFlushOrSync(); + + // All end of write operations, including deleting fake parent + // directories + writeOperationHelper.writeSuccessful(key); + } + + /** + * If flush has take place, need to append file, else to put object. + * + * @throws IOException any problem in append or put object + */ + private synchronized void putObjectIfNeedAppend() throws IOException { + if (appendAble.get() && fs.exists( + OBSCommonUtils.keyToQualifiedPath(fs, key))) { + appendFsFile(); + } else { + putObject(); + } + } + + /** + * Append posix file. + * + * @throws IOException any problem + */ + private synchronized void appendFsFile() throws IOException { + LOG.debug("bucket is posix, to append file. key is {}", key); + final OBSDataBlocks.DataBlock block = getActiveBlock(); + WriteFileRequest writeFileReq; + if (block instanceof OBSDataBlocks.DiskBlock) { + writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key, + objectLen, (File) block.startUpload()); + } else { + writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key, + objectLen, (InputStream) block.startUpload()); + } + OBSCommonUtils.appendFile(fs, writeFileReq); + objectLen += block.dataSize(); + } + + /** + * Upload the current block as a single PUT request; if the buffer is empty a + * 0-byte PUT will be invoked, as it is needed to create an entry at the far + * end. + * + * @throws IOException any problem. + */ + private synchronized void putObject() throws IOException { + LOG.debug("Executing regular upload for {}", + writeOperationHelper.toString(key)); + + final OBSDataBlocks.DataBlock block = getActiveBlock(); + clearActiveBlock(); + final int size = block.dataSize(); + final PutObjectRequest putObjectRequest; + if (block instanceof OBSDataBlocks.DiskBlock) { + putObjectRequest = writeOperationHelper.newPutRequest(key, + (File) block.startUpload()); + + } else { + putObjectRequest = + writeOperationHelper.newPutRequest(key, + (InputStream) block.startUpload(), size); + + } + putObjectRequest.setAcl(fs.getCannedACL()); + fs.getSchemeStatistics().incrementWriteOps(1); + try { + // the putObject call automatically closes the input + // stream afterwards. + writeOperationHelper.putObject(putObjectRequest); + } finally { + OBSCommonUtils.closeAll(block); + } + } + + @Override + public synchronized String toString() { + final StringBuilder sb = new StringBuilder("OBSBlockOutputStream{"); + sb.append(writeOperationHelper.toString()); + sb.append(", blockSize=").append(blockSize); + OBSDataBlocks.DataBlock block = activeBlock; + if (block != null) { + sb.append(", activeBlock=").append(block); + } + sb.append('}'); + return sb.toString(); + } + + public synchronized void sync() { + // need to do + } + + @Override + public synchronized void hflush() throws IOException { + // hflush hsyn same + flushOrSync(); + } + + /** + * Flush local file or multipart to obs. focus: not posix bucket is not + * support + * + * @throws IOException io exception + */ + private synchronized void flushOrSync() throws IOException { + + checkOpen(); + if (hasException.get()) { + String flushWarning = String.format( + "flushOrSync has error. bs : pre write obs[%s] has error.", + key); + LOG.warn(flushWarning); + throw new IOException(flushWarning); + } + if (fs.isFsBucket()) { + // upload + flushCurrentBlock(); + + // clear + clearHFlushOrSync(); + } else { + LOG.warn("not posix bucket, not support hflush or hsync."); + flush(); + } + } + + /** + * Clear for hflush or hsync. + */ + private synchronized void clearHFlushOrSync() { + appendAble.set(true); + multiPartUpload = null; + } + + /** + * Upload block to obs. + * + * @param block block + * @param hasBlock jungle if has block + * @throws IOException io exception + */ + private synchronized void uploadWriteBlocks( + final OBSDataBlocks.DataBlock block, + final boolean hasBlock) + throws IOException { + if (multiPartUpload == null) { + if (hasBlock) { + // no uploads of data have taken place, put the single block + // up. This must happen even if there is no data, so that 0 byte + // files are created. + putObjectIfNeedAppend(); + } + } else { + // there has already been at least one block scheduled for upload; + // put up the current then wait + if (hasBlock && block.hasData()) { + // send last part + uploadCurrentBlock(); + } + // wait for the partial uploads to finish + final List partETags + = multiPartUpload.waitForAllPartUploads(); + // then complete the operation + multiPartUpload.complete(partETags); + } + LOG.debug("Upload complete for {}", writeOperationHelper.toString(key)); + } + + private synchronized void completeCurrentBlock() throws IOException { + OBSDataBlocks.DataBlock block = getActiveBlock(); + boolean hasBlock = hasActiveBlock(); + LOG.debug("{}: complete block #{}: current block= {}", this, blockCount, + hasBlock ? block : "(none)"); + try { + uploadWriteBlocks(block, hasBlock); + } catch (IOException ioe) { + LOG.error("Upload data to obs error. io exception : {}", + ioe.getMessage()); + throw ioe; + } catch (Exception e) { + LOG.error("Upload data to obs error. other exception : {}", + e.getMessage()); + throw e; + } finally { + OBSCommonUtils.closeAll(block); + clearActiveBlock(); + } + } + + private synchronized void flushCurrentBlock() throws IOException { + OBSDataBlocks.DataBlock block = getActiveBlock(); + boolean hasBlock = hasActiveBlock(); + LOG.debug( + "{}: complete block #{}: current block= {}", this, blockCount, + hasBlock ? block : "(none)"); + try { + uploadWriteBlocks(block, hasBlock); + } catch (IOException ioe) { + LOG.error("hflush data to obs error. io exception : {}", + ioe.getMessage()); + hasException.set(true); + throw ioe; + } catch (Exception e) { + LOG.error("hflush data to obs error. other exception : {}", + e.getMessage()); + hasException.set(true); + throw e; + } finally { + OBSCommonUtils.closeAll(block); + clearActiveBlock(); + } + } + + @Override + public synchronized void hsync() throws IOException { + flushOrSync(); + } + + /** + * Multiple partition upload. + */ + private class MultiPartUpload { + /** + * Upload id for multipart upload. + */ + private final String uploadId; + + /** + * List for async part upload future. + */ + private final List> partETagsFutures; + + MultiPartUpload() throws IOException { + this.uploadId = writeOperationHelper.initiateMultiPartUpload(key); + this.partETagsFutures = new ArrayList<>(2); + LOG.debug( + "Initiated multi-part upload for {} with , the key is {}" + + "id '{}'", + writeOperationHelper, + uploadId, + key); + } + + /** + * Upload a block of data asynchronously. + * + * @param block block to upload + * @throws IOException upload failure + */ + private void uploadBlockAsync(final OBSDataBlocks.DataBlock block) + throws IOException { + LOG.debug("Queueing upload of {}", block); + + final int size = block.dataSize(); + final int currentPartNumber = partETagsFutures.size() + 1; + final UploadPartRequest request; + if (block instanceof OBSDataBlocks.DiskBlock) { + request = writeOperationHelper.newUploadPartRequest( + key, + uploadId, + currentPartNumber, + size, + (File) block.startUpload()); + } else { + request = writeOperationHelper.newUploadPartRequest( + key, + uploadId, + currentPartNumber, + size, + (InputStream) block.startUpload()); + + } + ListenableFuture partETagFuture = executorService.submit( + () -> { + // this is the queued upload operation + LOG.debug("Uploading part {} for id '{}'", + currentPartNumber, uploadId); + // do the upload + PartEtag partETag = null; + try { + if (mockUploadPartError) { + throw new ObsException("mock upload part error"); + } + UploadPartResult uploadPartResult + = OBSCommonUtils.uploadPart(fs, request); + partETag = + new PartEtag(uploadPartResult.getEtag(), + uploadPartResult.getPartNumber()); + if (LOG.isDebugEnabled()) { + LOG.debug("Completed upload of {} to part {}", + block, partETag); + } + } catch (ObsException e) { + // catch all exception + hasException.set(true); + LOG.error("UploadPart failed (ObsException). {}", + OBSCommonUtils.translateException("UploadPart", key, + e).getMessage()); + } finally { + // close the stream and block + OBSCommonUtils.closeAll(block); + } + return partETag; + }); + partETagsFutures.add(partETagFuture); + } + + /** + * Block awaiting all outstanding uploads to complete. + * + * @return list of results + * @throws IOException IO Problems + */ + private List waitForAllPartUploads() throws IOException { + LOG.debug("Waiting for {} uploads to complete", + partETagsFutures.size()); + try { + return Futures.allAsList(partETagsFutures).get(); + } catch (InterruptedException ie) { + LOG.warn("Interrupted partUpload", ie); + LOG.debug("Cancelling futures"); + for (ListenableFuture future : partETagsFutures) { + future.cancel(true); + } + // abort multipartupload + this.abort(); + throw new IOException( + "Interrupted multi-part upload with id '" + uploadId + + "' to " + key); + } catch (ExecutionException ee) { + // there is no way of recovering so abort + // cancel all partUploads + LOG.debug("While waiting for upload completion", ee); + LOG.debug("Cancelling futures"); + for (ListenableFuture future : partETagsFutures) { + future.cancel(true); + } + // abort multipartupload + this.abort(); + throw OBSCommonUtils.extractException( + "Multi-part upload with id '" + uploadId + "' to " + key, + key, ee); + } + } + + /** + * This completes a multipart upload. Sometimes it fails; here retries are + * handled to avoid losing all data on a transient failure. + * + * @param partETags list of partial uploads + * @return result for completing multipart upload + * @throws IOException on any problem + */ + private CompleteMultipartUploadResult complete( + final List partETags) throws IOException { + String operation = String.format( + "Completing multi-part upload for key '%s'," + + " id '%s' with %s partitions ", + key, uploadId, partETags.size()); + try { + LOG.debug(operation); + return writeOperationHelper.completeMultipartUpload(key, + uploadId, partETags); + } catch (ObsException e) { + throw OBSCommonUtils.translateException(operation, key, e); + } + } + + /** + * Abort a multi-part upload. Retries are attempted on failures. + * IOExceptions are caught; this is expected to be run as a cleanup + * process. + */ + void abort() { + String operation = + String.format( + "Aborting multi-part upload for '%s', id '%s", + writeOperationHelper, uploadId); + try { + LOG.debug(operation); + writeOperationHelper.abortMultipartUpload(key, uploadId); + } catch (ObsException e) { + LOG.warn( + "Unable to abort multipart upload, you may need to purge " + + "uploaded parts", + e); + } + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSClientFactory.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSClientFactory.java new file mode 100644 index 00000000000..fbd54feae80 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSClientFactory.java @@ -0,0 +1,46 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import com.obs.services.ObsClient; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +import java.io.IOException; +import java.net.URI; + +/** + * Factory for creating OBS client instance to be used by {@link + * OBSFileSystem}. + */ +@InterfaceAudience.Private +@InterfaceStability.Unstable +interface OBSClientFactory { + /** + * Creates a new {@link ObsClient} client. This method accepts the OBS file + * system URI both in raw input form and validated form as separate arguments, + * because both values may be useful in logging. + * + * @param name raw input OBS file system URI + * @return OBS client + * @throws IOException IO problem + */ + ObsClient createObsClient(URI name) throws IOException; +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java new file mode 100644 index 00000000000..ba7550bc0d6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java @@ -0,0 +1,1546 @@ +/* + * 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.obs; + +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import org.apache.hadoop.thirdparty.com.google.common.collect.Lists; +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import com.obs.services.model.AbortMultipartUploadRequest; +import com.obs.services.model.DeleteObjectsRequest; +import com.obs.services.model.DeleteObjectsResult; +import com.obs.services.model.KeyAndVersion; +import com.obs.services.model.ListMultipartUploadsRequest; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.MultipartUpload; +import com.obs.services.model.MultipartUploadListing; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObjectMetadata; +import com.obs.services.model.ObsObject; +import com.obs.services.model.PutObjectRequest; +import com.obs.services.model.PutObjectResult; +import com.obs.services.model.UploadPartRequest; +import com.obs.services.model.UploadPartResult; +import com.obs.services.model.fs.FSStatusEnum; +import com.obs.services.model.fs.GetAttributeRequest; +import com.obs.services.model.fs.GetBucketFSStatusRequest; +import com.obs.services.model.fs.GetBucketFSStatusResult; +import com.obs.services.model.fs.ObsFSAttribute; +import com.obs.services.model.fs.WriteFileRequest; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.InvalidRequestException; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.security.ProviderUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.net.URI; +import java.nio.file.AccessDeniedException; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorService; + +/** + * Common utils for {@link OBSFileSystem}. + */ +final class OBSCommonUtils { + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + OBSCommonUtils.class); + + /** + * Moved permanently response code. + */ + static final int MOVED_PERMANENTLY_CODE = 301; + + /** + * Unauthorized response code. + */ + static final int UNAUTHORIZED_CODE = 401; + + /** + * Forbidden response code. + */ + static final int FORBIDDEN_CODE = 403; + + /** + * Not found response code. + */ + static final int NOT_FOUND_CODE = 404; + + /** + * File conflict. + */ + static final int CONFLICT_CODE = 409; + + /** + * Gone response code. + */ + static final int GONE_CODE = 410; + + /** + * EOF response code. + */ + static final int EOF_CODE = 416; + + /** + * Core property for provider path. Duplicated here for consistent code across + * Hadoop version: {@value}. + */ + static final String CREDENTIAL_PROVIDER_PATH + = "hadoop.security.credential.provider.path"; + + /** + * Max number of retry times. + */ + static final int MAX_RETRY_TIME = 3; + + /** + * Delay time between two retries. + */ + static final int DELAY_TIME = 10; + + /** + * Max number of listing keys for checking folder empty. + */ + static final int MAX_KEYS_FOR_CHECK_FOLDER_EMPTY = 3; + + /** + * Max number of listing keys for checking folder empty. + */ + static final int BYTE_TO_INT_MASK = 0xFF; + + private OBSCommonUtils() { + } + + /** + * Get the fs status of the bucket. + * + * @param obs OBS client instance + * @param bucketName bucket name + * @return boolean value indicating if this bucket is a posix bucket + * @throws FileNotFoundException the bucket is absent + * @throws IOException any other problem talking to OBS + */ + static boolean getBucketFsStatus(final ObsClient obs, + final String bucketName) + throws FileNotFoundException, IOException { + try { + GetBucketFSStatusRequest getBucketFsStatusRequest + = new GetBucketFSStatusRequest(); + getBucketFsStatusRequest.setBucketName(bucketName); + GetBucketFSStatusResult getBucketFsStatusResult = + obs.getBucketFSStatus(getBucketFsStatusRequest); + FSStatusEnum fsStatus = getBucketFsStatusResult.getStatus(); + return fsStatus == FSStatusEnum.ENABLED; + } catch (ObsException e) { + LOG.error(e.toString()); + throw translateException("getBucketFsStatus", bucketName, e); + } + } + + /** + * Turns a path (relative or otherwise) into an OBS key. + * + * @param owner the owner OBSFileSystem instance + * @param path input path, may be relative to the working dir + * @return a key excluding the leading "/", or, if it is the root path, "" + */ + static String pathToKey(final OBSFileSystem owner, final Path path) { + Path absolutePath = path; + if (!path.isAbsolute()) { + absolutePath = new Path(owner.getWorkingDirectory(), path); + } + + if (absolutePath.toUri().getScheme() != null && absolutePath.toUri() + .getPath() + .isEmpty()) { + return ""; + } + + return absolutePath.toUri().getPath().substring(1); + } + + /** + * Turns a path (relative or otherwise) into an OBS key, adding a trailing "/" + * if the path is not the root and does not already have a "/" at the + * end. + * + * @param key obs key or "" + * @return the with a trailing "/", or, if it is the root key, "", + */ + static String maybeAddTrailingSlash(final String key) { + if (!StringUtils.isEmpty(key) && !key.endsWith("/")) { + return key + '/'; + } else { + return key; + } + } + + /** + * Convert a path back to a key. + * + * @param key input key + * @return the path from this key + */ + static Path keyToPath(final String key) { + return new Path("/" + key); + } + + /** + * Convert a key to a fully qualified path. + * + * @param owner the owner OBSFileSystem instance + * @param key input key + * @return the fully qualified path including URI scheme and bucket name. + */ + static Path keyToQualifiedPath(final OBSFileSystem owner, + final String key) { + return qualify(owner, keyToPath(key)); + } + + /** + * Qualify a path. + * + * @param owner the owner OBSFileSystem instance + * @param path path to qualify + * @return a qualified path. + */ + static Path qualify(final OBSFileSystem owner, final Path path) { + return path.makeQualified(owner.getUri(), owner.getWorkingDirectory()); + } + + /** + * Delete obs key started '/'. + * + * @param key object key + * @return new key + */ + static String maybeDeleteBeginningSlash(final String key) { + return !StringUtils.isEmpty(key) && key.startsWith("/") ? key.substring( + 1) : key; + } + + /** + * Add obs key started '/'. + * + * @param key object key + * @return new key + */ + static String maybeAddBeginningSlash(final String key) { + return !StringUtils.isEmpty(key) && !key.startsWith("/") + ? "/" + key + : key; + } + + /** + * Translate an exception raised in an operation into an IOException. HTTP + * error codes are examined and can be used to build a more specific + * response. + * + * @param operation operation + * @param path path operated on (may be null) + * @param exception obs exception raised + * @return an IOE which wraps the caught exception. + */ + static IOException translateException( + final String operation, final String path, + final ObsException exception) { + String message = String.format("%s%s: status [%d] - request id [%s] " + + "- error code [%s] - error message [%s] - trace :%s ", + operation, path != null ? " on " + path : "", + exception.getResponseCode(), exception.getErrorRequestId(), + exception.getErrorCode(), + exception.getErrorMessage(), exception); + + IOException ioe; + + int status = exception.getResponseCode(); + switch (status) { + case MOVED_PERMANENTLY_CODE: + message = + String.format("Received permanent redirect response, " + + "status [%d] - request id [%s] - " + + "error code [%s] - message [%s]", + exception.getResponseCode(), + exception.getErrorRequestId(), exception.getErrorCode(), + exception.getErrorMessage()); + ioe = new OBSIOException(message, exception); + break; + // permissions + case UNAUTHORIZED_CODE: + case FORBIDDEN_CODE: + ioe = new AccessDeniedException(path, null, message); + ioe.initCause(exception); + break; + + // the object isn't there + case NOT_FOUND_CODE: + case GONE_CODE: + ioe = new FileNotFoundException(message); + ioe.initCause(exception); + break; + + // out of range. This may happen if an object is overwritten with + // a shorter one while it is being read. + case EOF_CODE: + ioe = new EOFException(message); + break; + + default: + // no specific exit code. Choose an IOE subclass based on the + // class + // of the caught exception + ioe = new OBSIOException(message, exception); + break; + } + return ioe; + } + + /** + * Reject any request to delete an object where the key is root. + * + * @param bucket bucket name + * @param key key to validate + * @throws InvalidRequestException if the request was rejected due to a + * mistaken attempt to delete the root + * directory. + */ + static void blockRootDelete(final String bucket, final String key) + throws InvalidRequestException { + if (key.isEmpty() || "/".equals(key)) { + throw new InvalidRequestException( + "Bucket " + bucket + " cannot be deleted"); + } + } + + /** + * Delete an object. Increments the {@code OBJECT_DELETE_REQUESTS} and write + * operation statistics. + * + * @param owner the owner OBSFileSystem instance + * @param key key to blob to delete. + * @throws IOException on any failure to delete object + */ + static void deleteObject(final OBSFileSystem owner, final String key) + throws IOException { + blockRootDelete(owner.getBucket(), key); + ObsException lastException = null; + for (int retryTime = 1; retryTime <= MAX_RETRY_TIME; retryTime++) { + try { + owner.getObsClient().deleteObject(owner.getBucket(), key); + owner.getSchemeStatistics().incrementWriteOps(1); + return; + } catch (ObsException e) { + lastException = e; + LOG.warn("Delete path failed with [{}], " + + "retry time [{}] - request id [{}] - " + + "error code [{}] - error message [{}]", + e.getResponseCode(), retryTime, e.getErrorRequestId(), + e.getErrorCode(), e.getErrorMessage()); + if (retryTime < MAX_RETRY_TIME) { + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + throw translateException("delete", key, e); + } + } + } + } + throw translateException( + String.format("retry max times [%s] delete failed", MAX_RETRY_TIME), + key, lastException); + } + + /** + * Perform a bulk object delete operation. Increments the {@code + * OBJECT_DELETE_REQUESTS} and write operation statistics. + * + * @param owner the owner OBSFileSystem instance + * @param deleteRequest keys to delete on the obs-backend + * @throws IOException on any failure to delete objects + */ + static void deleteObjects(final OBSFileSystem owner, + final DeleteObjectsRequest deleteRequest) throws IOException { + DeleteObjectsResult result; + deleteRequest.setQuiet(true); + try { + result = owner.getObsClient().deleteObjects(deleteRequest); + owner.getSchemeStatistics().incrementWriteOps(1); + } catch (ObsException e) { + LOG.warn("delete objects failed, request [{}], request id [{}] - " + + "error code [{}] - error message [{}]", + deleteRequest, e.getErrorRequestId(), e.getErrorCode(), + e.getErrorMessage()); + for (KeyAndVersion keyAndVersion + : deleteRequest.getKeyAndVersionsList()) { + deleteObject(owner, keyAndVersion.getKey()); + } + return; + } + + // delete one by one if there is errors + if (result != null) { + List errorResults + = result.getErrorResults(); + if (!errorResults.isEmpty()) { + LOG.warn("bulk delete {} objects, {} failed, begin to delete " + + "one by one.", + deleteRequest.getKeyAndVersionsList().size(), + errorResults.size()); + for (DeleteObjectsResult.ErrorResult errorResult + : errorResults) { + deleteObject(owner, errorResult.getObjectKey()); + } + } + } + } + + /** + * Create a putObject request. Adds the ACL and metadata + * + * @param owner the owner OBSFileSystem instance + * @param key key of object + * @param metadata metadata header + * @param srcfile source file + * @return the request + */ + static PutObjectRequest newPutObjectRequest(final OBSFileSystem owner, + final String key, final ObjectMetadata metadata, final File srcfile) { + Preconditions.checkNotNull(srcfile); + PutObjectRequest putObjectRequest = new PutObjectRequest( + owner.getBucket(), key, srcfile); + putObjectRequest.setAcl(owner.getCannedACL()); + putObjectRequest.setMetadata(metadata); + if (owner.getSse().isSseCEnable()) { + putObjectRequest.setSseCHeader(owner.getSse().getSseCHeader()); + } else if (owner.getSse().isSseKmsEnable()) { + putObjectRequest.setSseKmsHeader(owner.getSse().getSseKmsHeader()); + } + return putObjectRequest; + } + + /** + * Create a {@link PutObjectRequest} request. The metadata is assumed to have + * been configured with the size of the operation. + * + * @param owner the owner OBSFileSystem instance + * @param key key of object + * @param metadata metadata header + * @param inputStream source data. + * @return the request + */ + static PutObjectRequest newPutObjectRequest(final OBSFileSystem owner, + final String key, final ObjectMetadata metadata, + final InputStream inputStream) { + Preconditions.checkNotNull(inputStream); + PutObjectRequest putObjectRequest = new PutObjectRequest( + owner.getBucket(), key, inputStream); + putObjectRequest.setAcl(owner.getCannedACL()); + putObjectRequest.setMetadata(metadata); + if (owner.getSse().isSseCEnable()) { + putObjectRequest.setSseCHeader(owner.getSse().getSseCHeader()); + } else if (owner.getSse().isSseKmsEnable()) { + putObjectRequest.setSseKmsHeader(owner.getSse().getSseKmsHeader()); + } + return putObjectRequest; + } + + /** + * PUT an object directly (i.e. not via the transfer manager). Byte length is + * calculated from the file length, or, if there is no file, from the content + * length of the header. Important: this call will close any input stream + * in the request. + * + * @param owner the owner OBSFileSystem instance + * @param putObjectRequest the request + * @return the upload initiated + * @throws ObsException on problems + */ + static PutObjectResult putObjectDirect(final OBSFileSystem owner, + final PutObjectRequest putObjectRequest) throws ObsException { + long len; + if (putObjectRequest.getFile() != null) { + len = putObjectRequest.getFile().length(); + } else { + len = putObjectRequest.getMetadata().getContentLength(); + } + + PutObjectResult result = owner.getObsClient() + .putObject(putObjectRequest); + owner.getSchemeStatistics().incrementWriteOps(1); + owner.getSchemeStatistics().incrementBytesWritten(len); + return result; + } + + /** + * Upload part of a multi-partition file. Increments the write and put + * counters. Important: this call does not close any input stream in the + * request. + * + * @param owner the owner OBSFileSystem instance + * @param request request + * @return the result of the operation. + * @throws ObsException on problems + */ + static UploadPartResult uploadPart(final OBSFileSystem owner, + final UploadPartRequest request) throws ObsException { + long len = request.getPartSize(); + UploadPartResult uploadPartResult = owner.getObsClient() + .uploadPart(request); + owner.getSchemeStatistics().incrementWriteOps(1); + owner.getSchemeStatistics().incrementBytesWritten(len); + return uploadPartResult; + } + + static void removeKeys(final OBSFileSystem owner, + final List keysToDelete, final boolean clearKeys, + final boolean checkRootDelete) throws IOException { + if (keysToDelete.isEmpty()) { + // exit fast if there are no keys to delete + return; + } + + if (checkRootDelete) { + for (KeyAndVersion keyVersion : keysToDelete) { + blockRootDelete(owner.getBucket(), keyVersion.getKey()); + } + } + + if (!owner.isEnableMultiObjectDelete() + || keysToDelete.size() < owner.getMultiDeleteThreshold()) { + // delete one by one. + for (KeyAndVersion keyVersion : keysToDelete) { + deleteObject(owner, keyVersion.getKey()); + } + } else if (keysToDelete.size() <= owner.getMaxEntriesToDelete()) { + // Only one batch. + DeleteObjectsRequest deleteObjectsRequest + = new DeleteObjectsRequest(owner.getBucket()); + deleteObjectsRequest.setKeyAndVersions( + keysToDelete.toArray(new KeyAndVersion[0])); + deleteObjects(owner, deleteObjectsRequest); + } else { + // Multi batches. + List keys = new ArrayList<>( + owner.getMaxEntriesToDelete()); + for (KeyAndVersion key : keysToDelete) { + keys.add(key); + if (keys.size() == owner.getMaxEntriesToDelete()) { + // Delete one batch. + removeKeys(owner, keys, true, false); + } + } + // Delete the last batch + removeKeys(owner, keys, true, false); + } + + if (clearKeys) { + keysToDelete.clear(); + } + } + + /** + * Translate an exception raised in an operation into an IOException. The + * specific type of IOException depends on the class of {@link ObsException} + * passed in, and any status codes included in the operation. That is: HTTP + * error codes are examined and can be used to build a more specific + * response. + * + * @param operation operation + * @param path path operated on (must not be null) + * @param exception obs exception raised + * @return an IOE which wraps the caught exception. + */ + static IOException translateException(final String operation, + final Path path, final ObsException exception) { + return translateException(operation, path.toString(), exception); + } + + /** + * List the statuses of the files/directories in the given path if the path is + * a directory. + * + * @param owner the owner OBSFileSystem instance + * @param f given path + * @param recursive flag indicating if list is recursive + * @return the statuses of the files/directories in the given patch + * @throws FileNotFoundException when the path does not exist; + * @throws IOException due to an IO problem. + * @throws ObsException on failures inside the OBS SDK + */ + static FileStatus[] innerListStatus(final OBSFileSystem owner, final Path f, + final boolean recursive) + throws FileNotFoundException, IOException, ObsException { + Path path = qualify(owner, f); + String key = pathToKey(owner, path); + + List result; + final FileStatus fileStatus = owner.getFileStatus(path); + + if (fileStatus.isDirectory()) { + key = maybeAddTrailingSlash(key); + String delimiter = recursive ? null : "/"; + ListObjectsRequest request = createListObjectsRequest(owner, key, + delimiter); + LOG.debug( + "listStatus: doing listObjects for directory {} - recursive {}", + f, recursive); + + OBSListing.FileStatusListingIterator files = owner.getObsListing() + .createFileStatusListingIterator( + path, request, OBSListing.ACCEPT_ALL, + new OBSListing.AcceptAllButSelfAndS3nDirs(path)); + result = new ArrayList<>(files.getBatchSize()); + while (files.hasNext()) { + result.add(files.next()); + } + + return result.toArray(new FileStatus[0]); + } else { + LOG.debug("Adding: rd (not a dir): {}", path); + FileStatus[] stats = new FileStatus[1]; + stats[0] = fileStatus; + return stats; + } + } + + /** + * Create a {@code ListObjectsRequest} request against this bucket. + * + * @param owner the owner OBSFileSystem instance + * @param key key for request + * @param delimiter any delimiter + * @return the request + */ + static ListObjectsRequest createListObjectsRequest( + final OBSFileSystem owner, final String key, final String delimiter) { + return createListObjectsRequest(owner, key, delimiter, -1); + } + + static ListObjectsRequest createListObjectsRequest( + final OBSFileSystem owner, final String key, final String delimiter, + final int maxKeyNum) { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(owner.getBucket()); + if (maxKeyNum > 0 && maxKeyNum < owner.getMaxKeys()) { + request.setMaxKeys(maxKeyNum); + } else { + request.setMaxKeys(owner.getMaxKeys()); + } + request.setPrefix(key); + if (delimiter != null) { + request.setDelimiter(delimiter); + } + return request; + } + + /** + * Implements the specific logic to reject root directory deletion. The caller + * must return the result of this call, rather than attempt to continue with + * the delete operation: deleting root directories is never allowed. This + * method simply implements the policy of when to return an exit code versus + * raise an exception. + * + * @param bucket bucket name + * @param isEmptyDir flag indicating if the directory is empty + * @param recursive recursive flag from command + * @return a return code for the operation + * @throws PathIOException if the operation was explicitly rejected. + */ + static boolean rejectRootDirectoryDelete(final String bucket, + final boolean isEmptyDir, + final boolean recursive) + throws IOException { + LOG.info("obs delete the {} root directory of {}", bucket, recursive); + if (isEmptyDir) { + return true; + } + if (recursive) { + return false; + } else { + // reject + throw new PathIOException(bucket, "Cannot delete root path"); + } + } + + /** + * Make the given path and all non-existent parents into directories. + * + * @param owner the owner OBSFileSystem instance + * @param path path to create + * @return true if a directory was created + * @throws FileAlreadyExistsException there is a file at the path specified + * @throws IOException other IO problems + * @throws ObsException on failures inside the OBS SDK + */ + static boolean innerMkdirs(final OBSFileSystem owner, final Path path) + throws IOException, FileAlreadyExistsException, ObsException { + LOG.debug("Making directory: {}", path); + FileStatus fileStatus; + try { + fileStatus = owner.getFileStatus(path); + + if (fileStatus.isDirectory()) { + return true; + } else { + throw new FileAlreadyExistsException("Path is a file: " + path); + } + } catch (FileNotFoundException e) { + Path fPart = path.getParent(); + do { + try { + fileStatus = owner.getFileStatus(fPart); + if (fileStatus.isDirectory()) { + break; + } + if (fileStatus.isFile()) { + throw new FileAlreadyExistsException( + String.format("Can't make directory for path '%s'" + + " since it is a file.", fPart)); + } + } catch (FileNotFoundException fnfe) { + LOG.debug("file {} not fount, but ignore.", path); + } + fPart = fPart.getParent(); + } while (fPart != null); + + String key = pathToKey(owner, path); + if (owner.isFsBucket()) { + OBSPosixBucketUtils.fsCreateFolder(owner, key); + } else { + OBSObjectBucketUtils.createFakeDirectory(owner, key); + } + return true; + } + } + + /** + * Initiate a {@code listObjects} operation, incrementing metrics in the + * process. + * + * @param owner the owner OBSFileSystem instance + * @param request request to initiate + * @return the results + * @throws IOException on any failure to list objects + */ + static ObjectListing listObjects(final OBSFileSystem owner, + final ListObjectsRequest request) throws IOException { + if (request.getDelimiter() == null && request.getMarker() == null + && owner.isFsBucket() && owner.isObsClientDFSListEnable()) { + return OBSFsDFSListing.fsDFSListObjects(owner, request); + } + + return commonListObjects(owner, request); + } + + static ObjectListing commonListObjects(final OBSFileSystem owner, + final ListObjectsRequest request) { + for (int retryTime = 1; retryTime < MAX_RETRY_TIME; retryTime++) { + try { + owner.getSchemeStatistics().incrementReadOps(1); + return owner.getObsClient().listObjects(request); + } catch (ObsException e) { + LOG.warn("Failed to commonListObjects for request[{}], retry " + + "time [{}], due to exception[{}]", + request, retryTime, e); + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + LOG.error("Failed to commonListObjects for request[{}], " + + "retry time [{}], due to exception[{}]", + request, retryTime, e); + throw e; + } + } + } + + owner.getSchemeStatistics().incrementReadOps(1); + return owner.getObsClient().listObjects(request); + } + + /** + * List the next set of objects. + * + * @param owner the owner OBSFileSystem instance + * @param objects paged result + * @return the next result object + * @throws IOException on any failure to list the next set of objects + */ + static ObjectListing continueListObjects(final OBSFileSystem owner, + final ObjectListing objects) throws IOException { + if (objects.getDelimiter() == null && owner.isFsBucket() + && owner.isObsClientDFSListEnable()) { + return OBSFsDFSListing.fsDFSContinueListObjects(owner, + (OBSFsDFSListing) objects); + } + + return commonContinueListObjects(owner, objects); + } + + private static ObjectListing commonContinueListObjects( + final OBSFileSystem owner, final ObjectListing objects) { + String delimiter = objects.getDelimiter(); + int maxKeyNum = objects.getMaxKeys(); + // LOG.debug("delimiters: "+objects.getDelimiter()); + ListObjectsRequest request = new ListObjectsRequest(); + request.setMarker(objects.getNextMarker()); + request.setBucketName(owner.getBucket()); + request.setPrefix(objects.getPrefix()); + if (maxKeyNum > 0 && maxKeyNum < owner.getMaxKeys()) { + request.setMaxKeys(maxKeyNum); + } else { + request.setMaxKeys(owner.getMaxKeys()); + } + if (delimiter != null) { + request.setDelimiter(delimiter); + } + return commonContinueListObjects(owner, request); + } + + static ObjectListing commonContinueListObjects(final OBSFileSystem owner, + final ListObjectsRequest request) { + for (int retryTime = 1; retryTime < MAX_RETRY_TIME; retryTime++) { + try { + owner.getSchemeStatistics().incrementReadOps(1); + return owner.getObsClient().listObjects(request); + } catch (ObsException e) { + LOG.warn("Continue list objects failed for request[{}], retry" + + " time[{}], due to exception[{}]", + request, retryTime, e); + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + LOG.error("Continue list objects failed for request[{}], " + + "retry time[{}], due to exception[{}]", + request, retryTime, e); + throw e; + } + } + } + + owner.getSchemeStatistics().incrementReadOps(1); + return owner.getObsClient().listObjects(request); + } + + /** + * Predicate: does the object represent a directory?. + * + * @param name object name + * @param size object size + * @return true if it meets the criteria for being an object + */ + public static boolean objectRepresentsDirectory(final String name, + final long size) { + return !name.isEmpty() && name.charAt(name.length() - 1) == '/' + && size == 0L; + } + + /** + * Date to long conversion. Handles null Dates that can be returned by OBS by + * returning 0 + * + * @param date date from OBS query + * @return timestamp of the object + */ + public static long dateToLong(final Date date) { + if (date == null) { + return 0L; + } + + return date.getTime() / OBSConstants.SEC2MILLISEC_FACTOR + * OBSConstants.SEC2MILLISEC_FACTOR; + } + + // Used to check if a folder is empty or not. + static boolean isFolderEmpty(final OBSFileSystem owner, final String key) + throws FileNotFoundException, ObsException { + for (int retryTime = 1; retryTime < MAX_RETRY_TIME; retryTime++) { + try { + return innerIsFolderEmpty(owner, key); + } catch (ObsException e) { + LOG.warn( + "Failed to check empty folder for [{}], retry time [{}], " + + "exception [{}]", key, retryTime, e); + + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + } + + return innerIsFolderEmpty(owner, key); + } + + // Used to check if a folder is empty or not by counting the number of + // sub objects in list. + private static boolean isFolderEmpty(final String key, + final ObjectListing objects) { + int count = objects.getObjects().size(); + if (count >= 2) { + // There is a sub file at least. + return false; + } else if (count == 1 && !objects.getObjects() + .get(0) + .getObjectKey() + .equals(key)) { + // There is a sub file at least. + return false; + } + + count = objects.getCommonPrefixes().size(); + // There is a sub file at least. + // There is no sub object. + if (count >= 2) { + // There is a sub file at least. + return false; + } else { + return count != 1 || objects.getCommonPrefixes().get(0).equals(key); + } + } + + // Used to check if a folder is empty or not. + static boolean innerIsFolderEmpty(final OBSFileSystem owner, + final String key) + throws FileNotFoundException, ObsException { + String obsKey = maybeAddTrailingSlash(key); + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(owner.getBucket()); + request.setPrefix(obsKey); + request.setDelimiter("/"); + request.setMaxKeys(MAX_KEYS_FOR_CHECK_FOLDER_EMPTY); + owner.getSchemeStatistics().incrementReadOps(1); + ObjectListing objects = owner.getObsClient().listObjects(request); + + if (!objects.getCommonPrefixes().isEmpty() || !objects.getObjects() + .isEmpty()) { + if (isFolderEmpty(obsKey, objects)) { + LOG.debug("Found empty directory {}", obsKey); + return true; + } + if (LOG.isDebugEnabled()) { + LOG.debug("Found path as directory (with /): {}/{}", + objects.getCommonPrefixes().size(), + objects.getObjects().size()); + + for (ObsObject summary : objects.getObjects()) { + LOG.debug("Summary: {} {}", summary.getObjectKey(), + summary.getMetadata().getContentLength()); + } + for (String prefix : objects.getCommonPrefixes()) { + LOG.debug("Prefix: {}", prefix); + } + } + LOG.debug("Found non-empty directory {}", obsKey); + return false; + } else if (obsKey.isEmpty()) { + LOG.debug("Found root directory"); + return true; + } else if (owner.isFsBucket()) { + LOG.debug("Found empty directory {}", obsKey); + return true; + } + + LOG.debug("Not Found: {}", obsKey); + throw new FileNotFoundException("No such file or directory: " + obsKey); + } + + /** + * Build a {@link LocatedFileStatus} from a {@link FileStatus} instance. + * + * @param owner the owner OBSFileSystem instance + * @param status file status + * @return a located status with block locations set up from this FS. + * @throws IOException IO Problems. + */ + static LocatedFileStatus toLocatedFileStatus(final OBSFileSystem owner, + final FileStatus status) throws IOException { + return new LocatedFileStatus( + status, status.isFile() ? owner.getFileBlockLocations(status, 0, + status.getLen()) : null); + } + + /** + * Create a appendFile request. Adds the ACL and metadata + * + * @param owner the owner OBSFileSystem instance + * @param key key of object + * @param tmpFile temp file or input stream + * @param recordPosition client record next append position + * @return the request + * @throws IOException any problem + */ + static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner, + final String key, final long recordPosition, final File tmpFile) + throws IOException { + Preconditions.checkNotNull(key); + Preconditions.checkNotNull(tmpFile); + ObsFSAttribute obsFsAttribute; + try { + GetAttributeRequest getAttributeReq = new GetAttributeRequest( + owner.getBucket(), key); + obsFsAttribute = owner.getObsClient().getAttribute(getAttributeReq); + } catch (ObsException e) { + throw translateException("GetAttributeRequest", key, e); + } + + long appendPosition = Math.max(recordPosition, + obsFsAttribute.getContentLength()); + if (recordPosition != obsFsAttribute.getContentLength()) { + LOG.warn("append url[{}] position[{}], file contentLength[{}] not" + + " equal to recordPosition[{}].", key, appendPosition, + obsFsAttribute.getContentLength(), recordPosition); + } + WriteFileRequest writeFileReq = new WriteFileRequest(owner.getBucket(), + key, tmpFile, appendPosition); + writeFileReq.setAcl(owner.getCannedACL()); + return writeFileReq; + } + + /** + * Create a appendFile request. Adds the ACL and metadata + * + * @param owner the owner OBSFileSystem instance + * @param key key of object + * @param inputStream temp file or input stream + * @param recordPosition client record next append position + * @return the request + * @throws IOException any problem + */ + static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner, + final String key, final long recordPosition, + final InputStream inputStream) throws IOException { + Preconditions.checkNotNull(key); + Preconditions.checkNotNull(inputStream); + ObsFSAttribute obsFsAttribute; + try { + GetAttributeRequest getAttributeReq = new GetAttributeRequest( + owner.getBucket(), key); + obsFsAttribute = owner.getObsClient().getAttribute(getAttributeReq); + } catch (ObsException e) { + throw translateException("GetAttributeRequest", key, e); + } + + long appendPosition = Math.max(recordPosition, + obsFsAttribute.getContentLength()); + if (recordPosition != obsFsAttribute.getContentLength()) { + LOG.warn("append url[{}] position[{}], file contentLength[{}] not" + + " equal to recordPosition[{}].", key, appendPosition, + obsFsAttribute.getContentLength(), recordPosition); + } + WriteFileRequest writeFileReq = new WriteFileRequest(owner.getBucket(), + key, inputStream, appendPosition); + writeFileReq.setAcl(owner.getCannedACL()); + return writeFileReq; + } + + /** + * Append File. + * + * @param owner the owner OBSFileSystem instance + * @param appendFileRequest append object request + * @throws IOException on any failure to append file + */ + static void appendFile(final OBSFileSystem owner, + final WriteFileRequest appendFileRequest) throws IOException { + long len = 0; + if (appendFileRequest.getFile() != null) { + len = appendFileRequest.getFile().length(); + } + + try { + LOG.debug("Append file, key {} position {} size {}", + appendFileRequest.getObjectKey(), + appendFileRequest.getPosition(), + len); + owner.getObsClient().writeFile(appendFileRequest); + owner.getSchemeStatistics().incrementWriteOps(1); + owner.getSchemeStatistics().incrementBytesWritten(len); + } catch (ObsException e) { + throw translateException("AppendFile", + appendFileRequest.getObjectKey(), e); + } + } + + /** + * Close the Closeable objects and ignore any Exception or null + * pointers. (This is the SLF4J equivalent of that in {@code IOUtils}). + * + * @param closeables the objects to close + */ + static void closeAll(final java.io.Closeable... closeables) { + for (java.io.Closeable c : closeables) { + if (c != null) { + try { + if (LOG != null) { + LOG.debug("Closing {}", c); + } + c.close(); + } catch (Exception e) { + if (LOG != null && LOG.isDebugEnabled()) { + LOG.debug("Exception in closing {}", c, e); + } + } + } + } + } + + /** + * Extract an exception from a failed future, and convert to an IOE. + * + * @param operation operation which failed + * @param path path operated on (may be null) + * @param ee execution exception + * @return an IOE which can be thrown + */ + static IOException extractException(final String operation, + final String path, final ExecutionException ee) { + IOException ioe; + Throwable cause = ee.getCause(); + if (cause instanceof ObsException) { + ioe = translateException(operation, path, (ObsException) cause); + } else if (cause instanceof IOException) { + ioe = (IOException) cause; + } else { + ioe = new IOException(operation + " failed: " + cause, cause); + } + return ioe; + } + + /** + * Create a files status instance from a listing. + * + * @param keyPath path to entry + * @param summary summary from OBS + * @param blockSize block size to declare. + * @param owner owner of the file + * @return a status entry + */ + static OBSFileStatus createFileStatus( + final Path keyPath, final ObsObject summary, final long blockSize, + final String owner) { + if (objectRepresentsDirectory( + summary.getObjectKey(), summary.getMetadata().getContentLength())) { + return new OBSFileStatus(keyPath, owner); + } else { + return new OBSFileStatus( + summary.getMetadata().getContentLength(), + dateToLong(summary.getMetadata().getLastModified()), + keyPath, + blockSize, + owner); + } + } + + /** + * Return the access key and secret for OBS API use. Credentials may exist in + * configuration, within credential providers or indicated in the UserInfo of + * the name URI param. + * + * @param name the URI for which we need the access keys. + * @param conf the Configuration object to interrogate for keys. + * @return OBSAccessKeys + * @throws IOException problems retrieving passwords from KMS. + */ + static OBSLoginHelper.Login getOBSAccessKeys(final URI name, + final Configuration conf) + throws IOException { + OBSLoginHelper.Login login + = OBSLoginHelper.extractLoginDetailsWithWarnings(name); + Configuration c = + ProviderUtils.excludeIncompatibleCredentialProviders(conf, + OBSFileSystem.class); + String accessKey = getPassword(c, OBSConstants.ACCESS_KEY, + login.getUser()); + String secretKey = getPassword(c, OBSConstants.SECRET_KEY, + login.getPassword()); + String sessionToken = getPassword(c, OBSConstants.SESSION_TOKEN, + login.getToken()); + return new OBSLoginHelper.Login(accessKey, secretKey, sessionToken); + } + + /** + * Get a password from a configuration, or, if a value is passed in, pick that + * up instead. + * + * @param conf configuration + * @param key key to look up + * @param val current value: if non empty this is used instead of querying + * the configuration. + * @return a password or "". + * @throws IOException on any problem + */ + private static String getPassword(final Configuration conf, + final String key, final String val) throws IOException { + return StringUtils.isEmpty(val) ? lookupPassword(conf, key) : val; + } + + /** + * Get a password from a configuration/configured credential providers. + * + * @param conf configuration + * @param key key to look up + * @return a password or the value in {@code defVal} + * @throws IOException on any problem + */ + private static String lookupPassword(final Configuration conf, + final String key) throws IOException { + try { + final char[] pass = conf.getPassword(key); + return pass != null ? new String(pass).trim() : ""; + } catch (IOException ioe) { + throw new IOException("Cannot find password option " + key, ioe); + } + } + + /** + * String information about a summary entry for debug messages. + * + * @param summary summary object + * @return string value + */ + static String stringify(final ObsObject summary) { + return summary.getObjectKey() + " size=" + summary.getMetadata() + .getContentLength(); + } + + /** + * Get a integer option not smaller than the minimum allowed value. + * + * @param conf configuration + * @param key key to look up + * @param defVal default value + * @param min minimum value + * @return the value + * @throws IllegalArgumentException if the value is below the minimum + */ + static int intOption(final Configuration conf, final String key, + final int defVal, + final int min) { + int v = conf.getInt(key, defVal); + Preconditions.checkArgument( + v >= min, + String.format("Value of %s: %d is below the minimum value %d", key, + v, min)); + LOG.debug("Value of {} is {}", key, v); + return v; + } + + /** + * Get a long option not smaller than the minimum allowed value. + * + * @param conf configuration + * @param key key to look up + * @param defVal default value + * @param min minimum value + * @return the value + * @throws IllegalArgumentException if the value is below the minimum + */ + static long longOption(final Configuration conf, final String key, + final long defVal, + final long min) { + long v = conf.getLong(key, defVal); + Preconditions.checkArgument( + v >= min, + String.format("Value of %s: %d is below the minimum value %d", key, + v, min)); + LOG.debug("Value of {} is {}", key, v); + return v; + } + + /** + * Get a long option not smaller than the minimum allowed value, supporting + * memory prefixes K,M,G,T,P. + * + * @param conf configuration + * @param key key to look up + * @param defVal default value + * @param min minimum value + * @return the value + * @throws IllegalArgumentException if the value is below the minimum + */ + static long longBytesOption(final Configuration conf, final String key, + final long defVal, + final long min) { + long v = conf.getLongBytes(key, defVal); + Preconditions.checkArgument( + v >= min, + String.format("Value of %s: %d is below the minimum value %d", key, + v, min)); + LOG.debug("Value of {} is {}", key, v); + return v; + } + + /** + * Get a size property from the configuration: this property must be at least + * equal to {@link OBSConstants#MULTIPART_MIN_SIZE}. If it is too small, it is + * rounded up to that minimum, and a warning printed. + * + * @param conf configuration + * @param property property name + * @param defVal default value + * @return the value, guaranteed to be above the minimum size + */ + public static long getMultipartSizeProperty(final Configuration conf, + final String property, final long defVal) { + long partSize = conf.getLongBytes(property, defVal); + if (partSize < OBSConstants.MULTIPART_MIN_SIZE) { + LOG.warn("{} must be at least 5 MB; configured value is {}", + property, partSize); + partSize = OBSConstants.MULTIPART_MIN_SIZE; + } + return partSize; + } + + /** + * Ensure that the long value is in the range of an integer. + * + * @param name property name for error messages + * @param size original size + * @return the size, guaranteed to be less than or equal to the max value of + * an integer. + */ + static int ensureOutputParameterInRange(final String name, + final long size) { + if (size > Integer.MAX_VALUE) { + LOG.warn( + "obs: {} capped to ~2.14GB" + + " (maximum allowed size with current output mechanism)", + name); + return Integer.MAX_VALUE; + } else { + return (int) size; + } + } + + /** + * Propagates bucket-specific settings into generic OBS configuration keys. + * This is done by propagating the values of the form {@code + * fs.obs.bucket.${bucket}.key} to {@code fs.obs.key}, for all values of "key" + * other than a small set of unmodifiable values. + * + *

The source of the updated property is set to the key name of the + * bucket property, to aid in diagnostics of where things came from. + * + *

Returns a new configuration. Why the clone? You can use the same conf + * for different filesystems, and the original values are not updated. + * + *

The {@code fs.obs.impl} property cannot be set, nor can any with the + * prefix {@code fs.obs.bucket}. + * + *

This method does not propagate security provider path information + * from the OBS property into the Hadoop common provider: callers must call + * {@link #patchSecurityCredentialProviders(Configuration)} explicitly. + * + * @param source Source Configuration object. + * @param bucket bucket name. Must not be empty. + * @return a (potentially) patched clone of the original. + */ + static Configuration propagateBucketOptions(final Configuration source, + final String bucket) { + + Preconditions.checkArgument(StringUtils.isNotEmpty(bucket), "bucket"); + final String bucketPrefix = OBSConstants.FS_OBS_BUCKET_PREFIX + bucket + + '.'; + LOG.debug("Propagating entries under {}", bucketPrefix); + final Configuration dest = new Configuration(source); + for (Map.Entry entry : source) { + final String key = entry.getKey(); + // get the (unexpanded) value. + final String value = entry.getValue(); + if (!key.startsWith(bucketPrefix) || bucketPrefix.equals(key)) { + continue; + } + // there's a bucket prefix, so strip it + final String stripped = key.substring(bucketPrefix.length()); + if (stripped.startsWith("bucket.") || "impl".equals(stripped)) { + // tell user off + LOG.debug("Ignoring bucket option {}", key); + } else { + // propagate the value, building a new origin field. + // to track overwrites, the generic key is overwritten even if + // already matches the new one. + final String generic = OBSConstants.FS_OBS_PREFIX + stripped; + LOG.debug("Updating {}", generic); + dest.set(generic, value, key); + } + } + return dest; + } + + /** + * Patch the security credential provider information in {@link + * #CREDENTIAL_PROVIDER_PATH} with the providers listed in {@link + * OBSConstants#OBS_SECURITY_CREDENTIAL_PROVIDER_PATH}. + * + *

This allows different buckets to use different credential files. + * + * @param conf configuration to patch + */ + static void patchSecurityCredentialProviders(final Configuration conf) { + Collection customCredentials = + conf.getStringCollection( + OBSConstants.OBS_SECURITY_CREDENTIAL_PROVIDER_PATH); + Collection hadoopCredentials = conf.getStringCollection( + CREDENTIAL_PROVIDER_PATH); + if (!customCredentials.isEmpty()) { + List all = Lists.newArrayList(customCredentials); + all.addAll(hadoopCredentials); + String joined = StringUtils.join(all, ','); + LOG.debug("Setting {} to {}", CREDENTIAL_PROVIDER_PATH, joined); + conf.set(CREDENTIAL_PROVIDER_PATH, joined, "patch of " + + OBSConstants.OBS_SECURITY_CREDENTIAL_PROVIDER_PATH); + } + } + + /** + * Verify that the bucket exists. This does not check permissions, not even + * read access. + * + * @param owner the owner OBSFileSystem instance + * @throws FileNotFoundException the bucket is absent + * @throws IOException any other problem talking to OBS + */ + static void verifyBucketExists(final OBSFileSystem owner) + throws FileNotFoundException, IOException { + int retryTime = 1; + while (true) { + try { + if (!owner.getObsClient().headBucket(owner.getBucket())) { + throw new FileNotFoundException( + "Bucket " + owner.getBucket() + " does not exist"); + } + return; + } catch (ObsException e) { + LOG.warn("Failed to head bucket for [{}], retry time [{}], " + + "exception [{}]", owner.getBucket(), retryTime, + translateException("doesBucketExist", owner.getBucket(), + e)); + + if (MAX_RETRY_TIME == retryTime) { + throw translateException("doesBucketExist", + owner.getBucket(), e); + } + + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + retryTime++; + } + } + + /** + * initialize multi-part upload, purge larger than the value of + * PURGE_EXISTING_MULTIPART_AGE. + * + * @param owner the owner OBSFileSystem instance + * @param conf the configuration to use for the FS + * @throws IOException on any failure to initialize multipart upload + */ + static void initMultipartUploads(final OBSFileSystem owner, + final Configuration conf) + throws IOException { + boolean purgeExistingMultipart = + conf.getBoolean(OBSConstants.PURGE_EXISTING_MULTIPART, + OBSConstants.DEFAULT_PURGE_EXISTING_MULTIPART); + long purgeExistingMultipartAge = + longOption(conf, OBSConstants.PURGE_EXISTING_MULTIPART_AGE, + OBSConstants.DEFAULT_PURGE_EXISTING_MULTIPART_AGE, 0); + + if (!purgeExistingMultipart) { + return; + } + + final Date purgeBefore = new Date( + new Date().getTime() - purgeExistingMultipartAge * 1000); + + try { + ListMultipartUploadsRequest request + = new ListMultipartUploadsRequest(owner.getBucket()); + while (true) { + // List + purge + MultipartUploadListing uploadListing = owner.getObsClient() + .listMultipartUploads(request); + for (MultipartUpload upload + : uploadListing.getMultipartTaskList()) { + if (upload.getInitiatedDate().compareTo(purgeBefore) < 0) { + owner.getObsClient().abortMultipartUpload( + new AbortMultipartUploadRequest( + owner.getBucket(), upload.getObjectKey(), + upload.getUploadId())); + } + } + if (!uploadListing.isTruncated()) { + break; + } + request.setUploadIdMarker( + uploadListing.getNextUploadIdMarker()); + request.setKeyMarker(uploadListing.getNextKeyMarker()); + } + } catch (ObsException e) { + if (e.getResponseCode() == FORBIDDEN_CODE) { + LOG.debug("Failed to purging multipart uploads against {}," + + " FS may be read only", owner.getBucket(), + e); + } else { + throw translateException("purging multipart uploads", + owner.getBucket(), e); + } + } + } + + static void shutdownAll(final ExecutorService... executors) { + for (ExecutorService exe : executors) { + if (exe != null) { + try { + if (LOG != null) { + LOG.debug("Shutdown {}", exe); + } + exe.shutdown(); + } catch (Exception e) { + if (LOG != null && LOG.isDebugEnabled()) { + LOG.debug("Exception in shutdown {}", exe, e); + } + } + } + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java new file mode 100644 index 00000000000..ac72e0404c4 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java @@ -0,0 +1,726 @@ +/* + * 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.obs; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; + +/** + * All constants used by {@link OBSFileSystem}. + * + *

Some of the strings are marked as {@code Unstable}. This means that they + * may be unsupported in future; at which point they will be marked as + * deprecated and simply ignored. + */ +@InterfaceAudience.Public +@InterfaceStability.Evolving +final class OBSConstants { + /** + * Minimum multipart size which OBS supports. + */ + static final int MULTIPART_MIN_SIZE = 5 * 1024 * 1024; + + /** + * OBS access key. + */ + static final String ACCESS_KEY = "fs.obs.access.key"; + + /** + * OBS secret key. + */ + static final String SECRET_KEY = "fs.obs.secret.key"; + + /** + * OBS credentials provider. + */ + static final String OBS_CREDENTIALS_PROVIDER + = "fs.obs.credentials.provider"; + + /** + * OBS client security provider. + */ + static final String OBS_SECURITY_PROVIDER = "fs.obs.security.provider"; + + /** + * Extra set of security credentials which will be prepended to that set in + * {@code "hadoop.security.credential.provider.path"}. This extra option + * allows for per-bucket overrides. + */ + static final String OBS_SECURITY_CREDENTIAL_PROVIDER_PATH = + "fs.obs.security.credential.provider.path"; + + /** + * Session token for when using TemporaryOBSCredentialsProvider. + */ + static final String SESSION_TOKEN = "fs.obs.session.token"; + + /** + * Maximum number of simultaneous connections to obs. + */ + static final String MAXIMUM_CONNECTIONS = "fs.obs.connection.maximum"; + + /** + * Default value of {@link #MAXIMUM_CONNECTIONS}. + */ + static final int DEFAULT_MAXIMUM_CONNECTIONS = 1000; + + /** + * Connect to obs over ssl. + */ + static final String SECURE_CONNECTIONS = "fs.obs.connection.ssl.enabled"; + + /** + * Default value of {@link #SECURE_CONNECTIONS}. + */ + static final boolean DEFAULT_SECURE_CONNECTIONS = false; + + /** + * Use a custom endpoint. + */ + static final String ENDPOINT = "fs.obs.endpoint"; + + /** + * Host for connecting to OBS through proxy server. + */ + static final String PROXY_HOST = "fs.obs.proxy.host"; + + /** + * Port for connecting to OBS through proxy server. + */ + static final String PROXY_PORT = "fs.obs.proxy.port"; + + /** + * User name for connecting to OBS through proxy server. + */ + static final String PROXY_USERNAME = "fs.obs.proxy.username"; + + /** + * Password for connecting to OBS through proxy server. + */ + static final String PROXY_PASSWORD = "fs.obs.proxy.password"; + + /** + * Default port for HTTPS. + */ + static final int DEFAULT_HTTPS_PORT = 443; + + /** + * Default port for HTTP. + */ + static final int DEFAULT_HTTP_PORT = 80; + + /** + * Number of times we should retry errors. + */ + static final String MAX_ERROR_RETRIES = "fs.obs.attempts.maximum"; + + /** + * Default value of {@link #MAX_ERROR_RETRIES}. + */ + static final int DEFAULT_MAX_ERROR_RETRIES = 3; + + /** + * Seconds until we give up trying to establish a connection to obs. + */ + static final String ESTABLISH_TIMEOUT + = "fs.obs.connection.establish.timeout"; + + /** + * Default value of {@link #ESTABLISH_TIMEOUT}. + */ + static final int DEFAULT_ESTABLISH_TIMEOUT = 120000; + + /** + * Seconds until we give up on a connection to obs. + */ + static final String SOCKET_TIMEOUT = "fs.obs.connection.timeout"; + + /** + * Default value of {@link #SOCKET_TIMEOUT}. + */ + static final int DEFAULT_SOCKET_TIMEOUT = 120000; + + /** + * Socket send buffer to be used in OBS SDK. + */ + static final String SOCKET_SEND_BUFFER = "fs.obs.socket.send.buffer"; + + /** + * Default value of {@link #SOCKET_SEND_BUFFER}. + */ + static final int DEFAULT_SOCKET_SEND_BUFFER = 256 * 1024; + + /** + * Socket receive buffer to be used in OBS SDK. + */ + static final String SOCKET_RECV_BUFFER = "fs.obs.socket.recv.buffer"; + + /** + * Default value of {@link #SOCKET_RECV_BUFFER}. + */ + static final int DEFAULT_SOCKET_RECV_BUFFER = 256 * 1024; + + /** + * Number of records to get while paging through a directory listing. + */ + static final String MAX_PAGING_KEYS = "fs.obs.paging.maximum"; + + /** + * Default value of {@link #MAX_PAGING_KEYS}. + */ + static final int DEFAULT_MAX_PAGING_KEYS = 1000; + + /** + * Maximum number of threads to allow in the pool used by TransferManager. + */ + static final String MAX_THREADS = "fs.obs.threads.max"; + + /** + * Default value of {@link #MAX_THREADS}. + */ + static final int DEFAULT_MAX_THREADS = 20; + + /** + * Maximum number of tasks cached if all threads are already uploading. + */ + static final String MAX_TOTAL_TASKS = "fs.obs.max.total.tasks"; + + /** + * Default value of {@link #MAX_TOTAL_TASKS}. + */ + static final int DEFAULT_MAX_TOTAL_TASKS = 20; + + /** + * Max number of copy threads. + */ + static final String MAX_COPY_THREADS = "fs.obs.copy.threads.max"; + + /** + * Default value of {@link #MAX_COPY_THREADS}. + */ + static final int DEFAULT_MAX_COPY_THREADS = 40; + + /** + * Max number of delete threads. + */ + static final String MAX_DELETE_THREADS = "fs.obs.delete.threads.max"; + + /** + * Default value of {@link #MAX_DELETE_THREADS}. + */ + static final int DEFAULT_MAX_DELETE_THREADS = 20; + + /** + * Unused option: maintained for compile-time compatibility. If set, a warning + * is logged in OBS during init. + */ + @Deprecated + static final String CORE_THREADS = "fs.obs.threads.core"; + + /** + * The time that an idle thread waits before terminating. + */ + static final String KEEPALIVE_TIME = "fs.obs.threads.keepalivetime"; + + /** + * Default value of {@link #KEEPALIVE_TIME}. + */ + static final int DEFAULT_KEEPALIVE_TIME = 60; + + /** + * Size of each of or multipart pieces in bytes. + */ + static final String MULTIPART_SIZE = "fs.obs.multipart.size"; + + /** + * Default value of {@link #MULTIPART_SIZE}. + */ + static final long DEFAULT_MULTIPART_SIZE = 104857600; // 100 MB + + /** + * Enable multi-object delete calls. + */ + static final String ENABLE_MULTI_DELETE = "fs.obs.multiobjectdelete.enable"; + + /** + * Max number of objects in one multi-object delete call. This option takes + * effect only when the option 'ENABLE_MULTI_DELETE' is set to 'true'. + */ + static final String MULTI_DELETE_MAX_NUMBER + = "fs.obs.multiobjectdelete.maximum"; + + /** + * Default value of {@link #MULTI_DELETE_MAX_NUMBER}. + */ + static final int DEFAULT_MULTI_DELETE_MAX_NUMBER = 1000; + + /** + * Delete recursively or not. + */ + static final String MULTI_DELETE_RECURSION + = "fs.obs.multiobjectdelete.recursion"; + + /** + * Minimum number of objects in one multi-object delete call. + */ + static final String MULTI_DELETE_THRESHOLD + = "fs.obs.multiobjectdelete.threshold"; + + /** + * Default value of {@link #MULTI_DELETE_THRESHOLD}. + */ + static final int MULTI_DELETE_DEFAULT_THRESHOLD = 3; + + /** + * Comma separated list of directories. + */ + static final String BUFFER_DIR = "fs.obs.buffer.dir"; + + /** + * Switch to the fast block-by-block upload mechanism. + */ + static final String FAST_UPLOAD = "fs.obs.fast.upload"; + + /** + * What buffer to use. Default is {@link #FAST_UPLOAD_BUFFER_DISK} Value: + * {@value} + */ + @InterfaceStability.Unstable + static final String FAST_UPLOAD_BUFFER = "fs.obs.fast.upload.buffer"; + + /** + * Buffer blocks to disk: {@value}. Capacity is limited to available disk + * space. + */ + @InterfaceStability.Unstable + static final String FAST_UPLOAD_BUFFER_DISK = "disk"; + + /** + * Use an in-memory array. Fast but will run of heap rapidly: {@value}. + */ + @InterfaceStability.Unstable + static final String FAST_UPLOAD_BUFFER_ARRAY = "array"; + + /** + * Use a byte buffer. May be more memory efficient than the {@link + * #FAST_UPLOAD_BUFFER_ARRAY}: {@value}. + */ + @InterfaceStability.Unstable + static final String FAST_UPLOAD_BYTEBUFFER = "bytebuffer"; + + /** + * Maximum number of blocks a single output stream can have active (uploading, + * or queued to the central FileSystem instance's pool of queued operations. + * )This stops a single stream overloading the shared thread pool. {@value} + * + *

Default is {@link #DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS} + */ + @InterfaceStability.Unstable + static final String FAST_UPLOAD_ACTIVE_BLOCKS + = "fs.obs.fast.upload.active.blocks"; + + /** + * Limit of queued block upload operations before writes block. Value: + * {@value} + */ + @InterfaceStability.Unstable + static final int DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS = 4; + + /** + * Canned acl options: Private | PublicRead | PublicReadWrite | + * AuthenticatedRead | LogDeliveryWrite | BucketOwnerRead | + * BucketOwnerFullControl. + */ + static final String CANNED_ACL = "fs.obs.acl.default"; + + /** + * Default value of {@link #CANNED_ACL}. + */ + static final String DEFAULT_CANNED_ACL = ""; + + /** + * Should we try to purge old multipart uploads when starting up. + */ + static final String PURGE_EXISTING_MULTIPART = "fs.obs.multipart.purge"; + + /** + * Default value of {@link #PURGE_EXISTING_MULTIPART}. + */ + static final boolean DEFAULT_PURGE_EXISTING_MULTIPART = false; + + /** + * Purge any multipart uploads older than this number of seconds. + */ + static final String PURGE_EXISTING_MULTIPART_AGE + = "fs.obs.multipart.purge.age"; + + /** + * Default value of {@link #PURGE_EXISTING_MULTIPART_AGE}. + */ + static final long DEFAULT_PURGE_EXISTING_MULTIPART_AGE = 86400; + + /** + * OBS folder suffix. + */ + static final String OBS_FOLDER_SUFFIX = "_$folder$"; + + /** + * Block size for + * {@link org.apache.hadoop.fs.FileSystem#getDefaultBlockSize()}. + */ + static final String FS_OBS_BLOCK_SIZE = "fs.obs.block.size"; + + /** + * Default value of {@link #FS_OBS_BLOCK_SIZE}. + */ + static final int DEFAULT_FS_OBS_BLOCK_SIZE = 128 * 1024 * 1024; + + /** + * OBS scheme. + */ + static final String OBS_SCHEME = "obs"; + + /** + * Prefix for all OBS properties: {@value}. + */ + static final String FS_OBS_PREFIX = "fs.obs."; + + /** + * Prefix for OBS bucket-specific properties: {@value}. + */ + static final String FS_OBS_BUCKET_PREFIX = "fs.obs.bucket."; + + /** + * OBS default port. + */ + static final int OBS_DEFAULT_PORT = -1; + + /** + * User agent prefix. + */ + static final String USER_AGENT_PREFIX = "fs.obs.user.agent.prefix"; + + /** + * Read ahead buffer size to prevent connection re-establishments. + */ + static final String READAHEAD_RANGE = "fs.obs.readahead.range"; + + /** + * Default value of {@link #READAHEAD_RANGE}. + */ + static final long DEFAULT_READAHEAD_RANGE = 1024 * 1024; + + /** + * Flag indicating if {@link OBSInputStream#read(long, byte[], int, int)} will + * use the implementation of + * {@link org.apache.hadoop.fs.FSInputStream#read(long, + * byte[], int, int)}. + */ + static final String READ_TRANSFORM_ENABLE = "fs.obs.read.transform.enable"; + + /** + * OBS client factory implementation class. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + static final String OBS_CLIENT_FACTORY_IMPL + = "fs.obs.client.factory.impl"; + + /** + * Default value of {@link #OBS_CLIENT_FACTORY_IMPL}. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + static final Class + DEFAULT_OBS_CLIENT_FACTORY_IMPL = + DefaultOBSClientFactory.class; + + /** + * Maximum number of partitions in a multipart upload: {@value}. + */ + @InterfaceAudience.Private + static final int MAX_MULTIPART_COUNT = 10000; + + // OBS Client configuration + + /** + * Idle connection time. + */ + static final String IDLE_CONNECTION_TIME = "fs.obs.idle.connection.time"; + + /** + * Default value of {@link #IDLE_CONNECTION_TIME}. + */ + static final int DEFAULT_IDLE_CONNECTION_TIME = 30000; + + /** + * Maximum number of idle connections. + */ + static final String MAX_IDLE_CONNECTIONS = "fs.obs.max.idle.connections"; + + /** + * Default value of {@link #MAX_IDLE_CONNECTIONS}. + */ + static final int DEFAULT_MAX_IDLE_CONNECTIONS = 1000; + + /** + * Keep alive. + */ + static final String KEEP_ALIVE = "fs.obs.keep.alive"; + + /** + * Default value of {@link #KEEP_ALIVE}. + */ + static final boolean DEFAULT_KEEP_ALIVE = true; + + /** + * Validate certificate. + */ + static final String VALIDATE_CERTIFICATE = "fs.obs.validate.certificate"; + + /** + * Default value of {@link #VALIDATE_CERTIFICATE}. + */ + static final boolean DEFAULT_VALIDATE_CERTIFICATE = false; + + /** + * Verify response content type. + */ + static final String VERIFY_RESPONSE_CONTENT_TYPE + = "fs.obs.verify.response.content.type"; + + /** + * Default value of {@link #VERIFY_RESPONSE_CONTENT_TYPE}. + */ + static final boolean DEFAULT_VERIFY_RESPONSE_CONTENT_TYPE = true; + + /** + * UploadStreamRetryBufferSize. + */ + static final String UPLOAD_STREAM_RETRY_SIZE + = "fs.obs.upload.stream.retry.buffer.size"; + + /** + * Default value of {@link #UPLOAD_STREAM_RETRY_SIZE}. + */ + static final int DEFAULT_UPLOAD_STREAM_RETRY_SIZE = 512 * 1024; + + /** + * Read buffer size. + */ + static final String READ_BUFFER_SIZE = "fs.obs.read.buffer.size"; + + /** + * Default value of {@link #READ_BUFFER_SIZE}. + */ + static final int DEFAULT_READ_BUFFER_SIZE = 256 * 1024; + + /** + * Write buffer size. + */ + static final String WRITE_BUFFER_SIZE = "fs.obs.write.buffer.size"; + + /** + * Default value of {@link #WRITE_BUFFER_SIZE}. + */ + static final int DEFAULT_WRITE_BUFFER_SIZE = 256 * 1024; + + /** + * Canonical name. + */ + static final String CNAME = "fs.obs.cname"; + + /** + * Default value of {@link #CNAME}. + */ + static final boolean DEFAULT_CNAME = false; + + /** + * Strict host name verification. + */ + static final String STRICT_HOSTNAME_VERIFICATION + = "fs.obs.strict.hostname.verification"; + + /** + * Default value of {@link #STRICT_HOSTNAME_VERIFICATION}. + */ + static final boolean DEFAULT_STRICT_HOSTNAME_VERIFICATION = false; + + /** + * Size of object copy part pieces in bytes. + */ + static final String COPY_PART_SIZE = "fs.obs.copypart.size"; + + /** + * Maximum value of {@link #COPY_PART_SIZE}. + */ + static final long MAX_COPY_PART_SIZE = 5368709120L; // 5GB + + /** + * Default value of {@link #COPY_PART_SIZE}. + */ + static final long DEFAULT_COPY_PART_SIZE = 104857600L; // 100MB + + /** + * Maximum number of copy part threads. + */ + static final String MAX_COPY_PART_THREADS = "fs.obs.copypart.threads.max"; + + /** + * Default value of {@link #MAX_COPY_PART_THREADS}. + */ + static final int DEFAULT_MAX_COPY_PART_THREADS = 40; + + /** + * Number of core list threads. + */ + static final String CORE_LIST_THREADS = "fs.obs.list.threads.core"; + + /** + * Default value of {@link #CORE_LIST_THREADS}. + */ + static final int DEFAULT_CORE_LIST_THREADS = 30; + + /** + * Maximum number of list threads. + */ + static final String MAX_LIST_THREADS = "fs.obs.list.threads.max"; + + /** + * Default value of {@link #MAX_LIST_THREADS}. + */ + static final int DEFAULT_MAX_LIST_THREADS = 60; + + /** + * Capacity of list work queue. + */ + static final String LIST_WORK_QUEUE_CAPACITY + = "fs.obs.list.workqueue.capacity"; + + /** + * Default value of {@link #LIST_WORK_QUEUE_CAPACITY}. + */ + static final int DEFAULT_LIST_WORK_QUEUE_CAPACITY = 1024; + + /** + * List parallel factor. + */ + static final String LIST_PARALLEL_FACTOR = "fs.obs.list.parallel.factor"; + + /** + * Default value of {@link #LIST_PARALLEL_FACTOR}. + */ + static final int DEFAULT_LIST_PARALLEL_FACTOR = 30; + + /** + * Switch for the fast delete. + */ + static final String TRASH_ENABLE = "fs.obs.trash.enable"; + + /** + * Enable obs content summary or not. + */ + static final String OBS_CONTENT_SUMMARY_ENABLE + = "fs.obs.content.summary.enable"; + + /** + * Enable obs client dfs list or not. + */ + static final String OBS_CLIENT_DFS_LIST_ENABLE + = "fs.obs.client.dfs.list.enable"; + + /** + * Default trash : false. + */ + static final boolean DEFAULT_TRASH = false; + + /** + * The fast delete recycle directory. + */ + static final String TRASH_DIR = "fs.obs.trash.dir"; + + /** + * Encryption type is sse-kms or sse-c. + */ + static final String SSE_TYPE = "fs.obs.server-side-encryption-type"; + + /** + * Kms key id for sse-kms, while key base64 encoded content for sse-c. + */ + static final String SSE_KEY = "fs.obs.server-side-encryption-key"; + + /** + * Array first block size. + */ + static final String FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE + = "fs.obs.fast.upload.array.first.buffer"; + + /** + * The fast upload buffer array first block default size. + */ + static final int FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT = 1024 + * 1024; + + /** + * Auth Type Negotiation Enable Switch. + */ + static final String SDK_AUTH_TYPE_NEGOTIATION_ENABLE + = "fs.obs.authtype.negotiation.enable"; + + /** + * Default value of {@link #SDK_AUTH_TYPE_NEGOTIATION_ENABLE}. + */ + static final boolean DEFAULT_SDK_AUTH_TYPE_NEGOTIATION_ENABLE = false; + + /** + * Okhttp retryOnConnectionFailure switch. + */ + static final String SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE + = "fs.obs.connection.retry.enable"; + + /** + * Default value of {@link #SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE}. + */ + static final boolean DEFAULT_SDK_RETRY_ON_CONNECTION_FAILURE_ENABLE = true; + + /** + * Sdk max retry times on unexpected end of stream. exception, default: -1, + * don't retry + */ + static final String SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION + = "fs.obs.unexpectedend.retrytime"; + + /** + * Default value of {@link #SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION}. + */ + static final int DEFAULT_SDK_RETRY_TIMES_ON_UNEXPECTED_END_EXCEPTION = -1; + + /** + * Maximum sdk connection retry times, default : 2000. + */ + static final int DEFAULT_MAX_SDK_CONNECTION_RETRY_TIMES = 2000; + + /** + * Second to millisecond factor. + */ + static final int SEC2MILLISEC_FACTOR = 1000; + + private OBSConstants() { + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java new file mode 100644 index 00000000000..5e413e6841a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java @@ -0,0 +1,1020 @@ +/* + * 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.obs; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.util.DirectBufferPool; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedOutputStream; +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.Closeable; +import java.io.EOFException; +import java.io.File; +import java.io.FileNotFoundException; +import java.io.FileOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; + +/** + * Set of classes to support output streaming into blocks which are then + * uploaded as to OBS as a single PUT, or as part of a multipart request. + */ +final class OBSDataBlocks { + + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + OBSDataBlocks.class); + + private OBSDataBlocks() { + } + + /** + * Validate args to a write command. These are the same validation checks + * expected for any implementation of {@code OutputStream.write()}. + * + * @param b byte array containing data + * @param off offset in array where to start + * @param len number of bytes to be written + * @throws NullPointerException for a null buffer + * @throws IndexOutOfBoundsException if indices are out of range + */ + static void validateWriteArgs(final byte[] b, final int off, + final int len) { + Preconditions.checkNotNull(b); + if (off < 0 || off > b.length || len < 0 || off + len > b.length + || off + len < 0) { + throw new IndexOutOfBoundsException( + "write (b[" + b.length + "], " + off + ", " + len + ')'); + } + } + + /** + * Create a factory. + * + * @param owner factory owner + * @param name factory name -the option from {@link OBSConstants}. + * @return the factory, ready to be initialized. + * @throws IllegalArgumentException if the name is unknown. + */ + static BlockFactory createFactory(final OBSFileSystem owner, + final String name) { + switch (name) { + case OBSConstants.FAST_UPLOAD_BUFFER_ARRAY: + return new ByteArrayBlockFactory(owner); + case OBSConstants.FAST_UPLOAD_BUFFER_DISK: + return new DiskBlockFactory(owner); + case OBSConstants.FAST_UPLOAD_BYTEBUFFER: + return new ByteBufferBlockFactory(owner); + default: + throw new IllegalArgumentException( + "Unsupported block buffer" + " \"" + name + '"'); + } + } + + /** + * Base class for block factories. + */ + abstract static class BlockFactory { + /** + * OBS file system type. + */ + private final OBSFileSystem owner; + + protected BlockFactory(final OBSFileSystem obsFileSystem) { + this.owner = obsFileSystem; + } + + /** + * Create a block. + * + * @param index index of block + * @param limit limit of the block. + * @return a new block. + * @throws IOException on any failure to create block + */ + abstract DataBlock create(long index, int limit) throws IOException; + + /** + * Owner. + * + * @return obsFileSystem instance + */ + protected OBSFileSystem getOwner() { + return owner; + } + } + + /** + * This represents a block being uploaded. + */ + abstract static class DataBlock implements Closeable { + + /** + * Data block index. + */ + private final long index; + + /** + * Dest state can be : writing/upload/closed. + */ + private volatile DestState state = DestState.Writing; + + protected DataBlock(final long dataIndex) { + this.index = dataIndex; + } + + /** + * Atomically enter a state, verifying current state. + * + * @param current current state. null means "no check" + * @param next next state + * @throws IllegalStateException if the current state is not as expected + */ + protected final synchronized void enterState(final DestState current, + final DestState next) + throws IllegalStateException { + verifyState(current); + LOG.debug("{}: entering state {}", this, next); + state = next; + } + + /** + * Verify that the block is in the declared state. + * + * @param expected expected state. + * @throws IllegalStateException if the DataBlock is in the wrong state + */ + protected final void verifyState(final DestState expected) + throws IllegalStateException { + if (expected != null && state != expected) { + throw new IllegalStateException( + "Expected stream state " + expected + + " -but actual state is " + state + " in " + this); + } + } + + /** + * Current state. + * + * @return the current state. + */ + protected final DestState getState() { + return state; + } + + protected long getIndex() { + return index; + } + + /** + * Return the current data size. + * + * @return the size of the data + */ + abstract int dataSize(); + + /** + * Predicate to verify that the block has the capacity to write the given + * set of bytes. + * + * @param bytes number of bytes desired to be written. + * @return true if there is enough space. + */ + abstract boolean hasCapacity(long bytes); + + /** + * Predicate to check if there is data in the block. + * + * @return true if there is + */ + boolean hasData() { + return dataSize() > 0; + } + + /** + * The remaining capacity in the block before it is full. + * + * @return the number of bytes remaining. + */ + abstract int remainingCapacity(); + + /** + * Write a series of bytes from the buffer, from the offset. Returns the + * number of bytes written. Only valid in the state {@code Writing}. Base + * class verifies the state but does no writing. + * + * @param buffer buffer + * @param offset offset + * @param length length of write + * @return number of bytes written + * @throws IOException trouble + */ + int write(final byte[] buffer, final int offset, final int length) + throws IOException { + verifyState(DestState.Writing); + Preconditions.checkArgument(buffer != null, "Null buffer"); + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(offset >= 0, "offset is negative"); + Preconditions.checkArgument( + !(buffer.length - offset < length), + "buffer shorter than amount of data to write"); + return 0; + } + + /** + * Flush the output. Only valid in the state {@code Writing}. In the base + * class, this is a no-op + * + * @throws IOException any IO problem. + */ + void flush() throws IOException { + verifyState(DestState.Writing); + } + + /** + * Switch to the upload state and return a stream for uploading. Base class + * calls {@link #enterState(DestState, DestState)} to manage the state + * machine. + * + * @return the stream + * @throws IOException trouble + */ + Object startUpload() throws IOException { + LOG.debug("Start datablock[{}] upload", index); + enterState(DestState.Writing, DestState.Upload); + return null; + } + + /** + * Enter the closed state. + * + * @return true if the class was in any other state, implying that the + * subclass should do its close operations + */ + protected synchronized boolean enterClosedState() { + if (!state.equals(DestState.Closed)) { + enterState(null, DestState.Closed); + return true; + } else { + return false; + } + } + + @Override + public void close() throws IOException { + if (enterClosedState()) { + LOG.debug("Closed {}", this); + innerClose(); + } + } + + /** + * Inner close logic for subclasses to implement. + * + * @throws IOException on any failure to close + */ + protected abstract void innerClose() throws IOException; + + /** + * Destination state definition for a data block. + */ + enum DestState { + /** + * destination state : writing. + */ + Writing, + /** + * destination state : upload. + */ + Upload, + /** + * destination state : closed. + */ + Closed + } + } + + /** + * Use byte arrays on the heap for storage. + */ + static class ByteArrayBlockFactory extends BlockFactory { + ByteArrayBlockFactory(final OBSFileSystem owner) { + super(owner); + } + + @Override + DataBlock create(final long index, final int limit) { + int firstBlockSize = super.owner.getConf() + .getInt(OBSConstants.FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE, + OBSConstants + .FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT); + return new ByteArrayBlock(0, limit, firstBlockSize); + } + } + + /** + * OBS specific byte array output stream. + */ + static class OBSByteArrayOutputStream extends ByteArrayOutputStream { + OBSByteArrayOutputStream(final int size) { + super(size); + } + + /** + * InputStream backed by the internal byte array. + * + * @return input stream + */ + ByteArrayInputStream getInputStream() { + ByteArrayInputStream bin = new ByteArrayInputStream(this.buf, 0, + count); + this.reset(); + this.buf = null; + return bin; + } + } + + /** + * Stream to memory via a {@code ByteArrayOutputStream}. + * + *

This was taken from {@code OBSBlockOutputStream} and has the same + * problem which surfaced there: it can consume a lot of heap space + * proportional to the mismatch between writes to the stream and the JVM-wide + * upload bandwidth to the OBS endpoint. The memory consumption can be limited + * by tuning the filesystem settings to restrict the number of queued/active + * uploads. + */ + static class ByteArrayBlock extends DataBlock { + /** + * Memory limit. + */ + private final int limit; + + /** + * Output stream. + */ + private OBSByteArrayOutputStream buffer; + + /** + * Cache data size so that it is consistent after the buffer is reset. + */ + private Integer dataSize; + + /** + * Block first size. + */ + private int firstBlockSize; + + /** + * Input stream. + */ + private ByteArrayInputStream inputStream = null; + + ByteArrayBlock(final long index, final int limitBlockSize, + final int blockSize) { + super(index); + this.limit = limitBlockSize; + this.buffer = new OBSByteArrayOutputStream(blockSize); + this.firstBlockSize = blockSize; + } + + /** + * Returns the block first block size. + * + * @return the block first block size + */ + @VisibleForTesting + public int firstBlockSize() { + return this.firstBlockSize; + } + + /** + * Get the amount of data; if there is no buffer then the size is 0. + * + * @return the amount of data available to upload. + */ + @Override + int dataSize() { + return dataSize != null ? dataSize : buffer.size(); + } + + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = buffer.size(); + inputStream = buffer.getInputStream(); + return inputStream; + } + + @Override + boolean hasCapacity(final long bytes) { + return dataSize() + bytes <= limit; + } + + @Override + int remainingCapacity() { + return limit - dataSize(); + } + + @Override + int write(final byte[] b, final int offset, final int len) + throws IOException { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + buffer.write(b, offset, written); + return written; + } + + @Override + protected void innerClose() throws IOException { + if (buffer != null) { + buffer.close(); + buffer = null; + } + + if (inputStream != null) { + inputStream.close(); + inputStream = null; + } + } + + @Override + public String toString() { + return "ByteArrayBlock{" + + "index=" + + getIndex() + + ", state=" + + getState() + + ", limit=" + + limit + + ", dataSize=" + + dataSize + + '}'; + } + } + + /** + * Stream via Direct ByteBuffers; these are allocated off heap via {@link + * DirectBufferPool}. + */ + static class ByteBufferBlockFactory extends BlockFactory { + + /** + * The directory buffer pool. + */ + private static final DirectBufferPool BUFFER_POOL + = new DirectBufferPool(); + + /** + * Count of outstanding buffers. + */ + private static final AtomicInteger BUFFERS_OUTSTANDING + = new AtomicInteger(0); + + ByteBufferBlockFactory(final OBSFileSystem owner) { + super(owner); + } + + @Override + ByteBufferBlock create(final long index, final int limit) { + return new ByteBufferBlock(index, limit); + } + + public static ByteBuffer requestBuffer(final int limit) { + LOG.debug("Requesting buffer of size {}", limit); + BUFFERS_OUTSTANDING.incrementAndGet(); + return BUFFER_POOL.getBuffer(limit); + } + + public static void releaseBuffer(final ByteBuffer buffer) { + LOG.debug("Releasing buffer"); + BUFFER_POOL.returnBuffer(buffer); + BUFFERS_OUTSTANDING.decrementAndGet(); + } + + /** + * Get count of outstanding buffers. + * + * @return the current buffer count + */ + public int getOutstandingBufferCount() { + return BUFFERS_OUTSTANDING.get(); + } + + @Override + public String toString() { + return "ByteBufferBlockFactory{" + "buffersOutstanding=" + + BUFFERS_OUTSTANDING + '}'; + } + } + + /** + * A DataBlock which requests a buffer from pool on creation; returns it when + * it is closed. + */ + static class ByteBufferBlock extends DataBlock { + /** + * Set the buffer size. + */ + private final int bufferSize; + + /** + * Create block buffer. + */ + private ByteBuffer blockBuffer; + + /** + * Cache data size so that it is consistent after the buffer is reset. + */ + private Integer dataSize; + + /** + * Create input stream. + */ + private ByteBufferInputStream inputStream; + + /** + * Instantiate. This will request a ByteBuffer of the desired size. + * + * @param index block index + * @param initBufferSize buffer size + */ + ByteBufferBlock(final long index, final int initBufferSize) { + super(index); + this.bufferSize = initBufferSize; + blockBuffer = ByteBufferBlockFactory.requestBuffer(initBufferSize); + } + + /** + * Get the amount of data; if there is no buffer then the size is 0. + * + * @return the amount of data available to upload. + */ + @Override + int dataSize() { + return dataSize != null ? dataSize : bufferCapacityUsed(); + } + + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = bufferCapacityUsed(); + // set the buffer up from reading from the beginning + blockBuffer.limit(blockBuffer.position()); + blockBuffer.position(0); + inputStream = new ByteBufferInputStream(dataSize, blockBuffer); + return inputStream; + } + + @Override + public boolean hasCapacity(final long bytes) { + return bytes <= remainingCapacity(); + } + + @Override + public int remainingCapacity() { + return blockBuffer != null ? blockBuffer.remaining() : 0; + } + + private int bufferCapacityUsed() { + return blockBuffer.capacity() - blockBuffer.remaining(); + } + + @Override + int write(final byte[] b, final int offset, final int len) + throws IOException { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + blockBuffer.put(b, offset, written); + return written; + } + + /** + * Closing the block will release the buffer. + */ + @Override + protected void innerClose() { + if (blockBuffer != null) { + ByteBufferBlockFactory.releaseBuffer(blockBuffer); + blockBuffer = null; + } + if (inputStream != null) { + inputStream.close(); + inputStream = null; + } + } + + @Override + public String toString() { + return "ByteBufferBlock{" + + "index=" + + getIndex() + + ", state=" + + getState() + + ", dataSize=" + + dataSize() + + ", limit=" + + bufferSize + + ", remainingCapacity=" + + remainingCapacity() + + '}'; + } + + /** + * Provide an input stream from a byte buffer; supporting {@link + * #mark(int)}, which is required to enable replay of failed PUT attempts. + */ + class ByteBufferInputStream extends InputStream { + + /** + * Set the input stream size. + */ + private final int size; + + /** + * Set the byte buffer. + */ + private ByteBuffer byteBuffer; + + ByteBufferInputStream(final int streamSize, + final ByteBuffer streamByteBuffer) { + LOG.debug("Creating ByteBufferInputStream of size {}", + streamSize); + this.size = streamSize; + this.byteBuffer = streamByteBuffer; + } + + /** + * After the stream is closed, set the local reference to the byte buffer + * to null; this guarantees that future attempts to use stream methods + * will fail. + */ + @Override + public synchronized void close() { + LOG.debug("ByteBufferInputStream.close() for {}", + ByteBufferBlock.super.toString()); + byteBuffer = null; + } + + /** + * Verify that the stream is open. + * + * @throws IOException if the stream is closed + */ + private void verifyOpen() throws IOException { + if (byteBuffer == null) { + throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + public synchronized int read() { + if (available() > 0) { + return byteBuffer.get() & OBSCommonUtils.BYTE_TO_INT_MASK; + } else { + return -1; + } + } + + @Override + public synchronized long skip(final long offset) + throws IOException { + verifyOpen(); + long newPos = position() + offset; + if (newPos < 0) { + throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK); + } + if (newPos > size) { + throw new EOFException( + FSExceptionMessages.CANNOT_SEEK_PAST_EOF); + } + byteBuffer.position((int) newPos); + return newPos; + } + + @Override + public synchronized int available() { + Preconditions.checkState(byteBuffer != null, + FSExceptionMessages.STREAM_IS_CLOSED); + return byteBuffer.remaining(); + } + + /** + * Get the current buffer position. + * + * @return the buffer position + */ + public synchronized int position() { + return byteBuffer.position(); + } + + /** + * Check if there is data left. + * + * @return true if there is data remaining in the buffer. + */ + public synchronized boolean hasRemaining() { + return byteBuffer.hasRemaining(); + } + + @Override + public synchronized void mark(final int readlimit) { + LOG.debug("mark at {}", position()); + byteBuffer.mark(); + } + + @Override + public synchronized void reset() { + LOG.debug("reset"); + byteBuffer.reset(); + } + + @Override + public boolean markSupported() { + return true; + } + + /** + * Read in data. + * + * @param b destination buffer + * @param offset offset within the buffer + * @param length length of bytes to read + * @return read size + * @throws EOFException if the position is negative + * @throws IndexOutOfBoundsException if there isn't space for the amount + * of data requested. + * @throws IllegalArgumentException other arguments are invalid. + */ + public synchronized int read(final byte[] b, final int offset, + final int length) + throws IOException { + Preconditions.checkArgument(length >= 0, "length is negative"); + Preconditions.checkArgument(b != null, "Null buffer"); + if (b.length - offset < length) { + throw new IndexOutOfBoundsException( + FSExceptionMessages.TOO_MANY_BYTES_FOR_DEST_BUFFER + + ": request length =" + + length + + ", with offset =" + + offset + + "; buffer capacity =" + + (b.length - offset)); + } + verifyOpen(); + if (!hasRemaining()) { + return -1; + } + + int toRead = Math.min(length, available()); + byteBuffer.get(b, offset, toRead); + return toRead; + } + + @Override + public String toString() { + final StringBuilder sb = new StringBuilder( + "ByteBufferInputStream{"); + sb.append("size=").append(size); + ByteBuffer buf = this.byteBuffer; + if (buf != null) { + sb.append(", available=").append(buf.remaining()); + } + sb.append(", ").append(ByteBufferBlock.super.toString()); + sb.append('}'); + return sb.toString(); + } + } + } + + /** + * Buffer blocks to disk. + */ + static class DiskBlockFactory extends BlockFactory { + /** + * Allocator the local directory. + */ + private static LocalDirAllocator directoryAllocator; + + DiskBlockFactory(final OBSFileSystem owner) { + super(owner); + } + + /** + * Create a temp file and a {@link DiskBlock} instance to manage it. + * + * @param index block index + * @param limit limit of the block. + * @return the new block + * @throws IOException IO problems + */ + @Override + DataBlock create(final long index, final int limit) throws IOException { + File destFile = createTmpFileForWrite( + String.format("obs-block-%04d-", index), limit, + getOwner().getConf()); + return new DiskBlock(destFile, limit, index); + } + + /** + * Demand create the directory allocator, then create a temporary file. + * {@link LocalDirAllocator#createTmpFileForWrite(String, long, + * Configuration)}. + * + * @param pathStr prefix for the temporary file + * @param size the size of the file that is going to be written + * @param conf the Configuration object + * @return a unique temporary file + * @throws IOException IO problems + */ + static synchronized File createTmpFileForWrite(final String pathStr, + final long size, final Configuration conf) + throws IOException { + if (directoryAllocator == null) { + String bufferDir = conf.get(OBSConstants.BUFFER_DIR) != null + ? OBSConstants.BUFFER_DIR + : "hadoop.tmp.dir"; + directoryAllocator = new LocalDirAllocator(bufferDir); + } + return directoryAllocator.createTmpFileForWrite(pathStr, size, + conf); + } + } + + /** + * Stream to a file. This will stop at the limit; the caller is expected to + * create a new block. + */ + static class DiskBlock extends DataBlock { + + /** + * Create buffer file. + */ + private final File bufferFile; + + /** + * Buffer size limit. + */ + private final int limit; + + /** + * Verify block has closed or not. + */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** + * Written bytes count. + */ + private int bytesWritten; + + /** + * Out put stream buffer. + */ + private BufferedOutputStream out; + + DiskBlock(final File destBufferFile, final int limitSize, + final long index) + throws FileNotFoundException { + super(index); + this.limit = limitSize; + this.bufferFile = destBufferFile; + out = new BufferedOutputStream( + new FileOutputStream(destBufferFile)); + } + + @Override + int dataSize() { + return bytesWritten; + } + + @Override + boolean hasCapacity(final long bytes) { + return dataSize() + bytes <= limit; + } + + @Override + int remainingCapacity() { + return limit - bytesWritten; + } + + @Override + int write(final byte[] b, final int offset, final int len) + throws IOException { + super.write(b, offset, len); + int written = Math.min(remainingCapacity(), len); + out.write(b, offset, written); + bytesWritten += written; + return written; + } + + @Override + File startUpload() throws IOException { + super.startUpload(); + try { + out.flush(); + } finally { + out.close(); + out = null; + } + return bufferFile; + } + + /** + * The close operation will delete the destination file if it still exists. + */ + @Override + protected void innerClose() { + final DestState state = getState(); + LOG.debug("Closing {}", this); + switch (state) { + case Writing: + if (bufferFile.exists()) { + // file was not uploaded + LOG.debug( + "Block[{}]: Deleting buffer file as upload " + + "did not start", + getIndex()); + closeBlock(); + } + break; + + case Upload: + LOG.debug( + "Block[{}]: Buffer file {} exists close upload stream", + getIndex(), bufferFile); + break; + + case Closed: + closeBlock(); + break; + + default: + // this state can never be reached, but checkstyle + // complains, so it is here. + } + } + + /** + * Flush operation will flush to disk. + * + * @throws IOException IOE raised on FileOutputStream + */ + @Override + void flush() throws IOException { + super.flush(); + out.flush(); + } + + @Override + public String toString() { + return "FileBlock{index=" + getIndex() + ", destFile=" + bufferFile + + ", state=" + getState() + ", dataSize=" + + dataSize() + ", limit=" + limit + '}'; + } + + /** + * Close the block. This will delete the block's buffer file if the block + * has not previously been closed. + */ + void closeBlock() { + LOG.debug("block[{}]: closeBlock()", getIndex()); + if (!closed.getAndSet(true)) { + if (!bufferFile.delete() && bufferFile.exists()) { + LOG.warn("delete({}) returned false", + bufferFile.getAbsoluteFile()); + } + } else { + LOG.debug("block[{}]: skipping re-entrant closeBlock()", + getIndex()); + } + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java new file mode 100644 index 00000000000..448115554f8 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java @@ -0,0 +1,92 @@ +/* + * 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.obs; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; + +/** + * File status for an OBS file. + * + *

The subclass is private as it should not be created directly. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +class OBSFileStatus extends FileStatus { + /** + * Create a directory status. + * + * @param path the path + * @param owner the owner + */ + OBSFileStatus(final Path path, final String owner) { + super(0, true, 1, 0, 0, path); + setOwner(owner); + setGroup(owner); + } + + /** + * Create a directory status. + * + * @param modificationTime modification time + * @param path the path + * @param owner the owner + */ + OBSFileStatus(final Path path, final long modificationTime, + final String owner) { + super(0, true, 1, 0, modificationTime, path); + setOwner(owner); + setGroup(owner); + } + + /** + * Create a directory status. + * + * @param modificationTime modification time + * @param accessTime access time + * @param path the path + * @param owner the owner + */ + OBSFileStatus(final Path path, final long modificationTime, + final long accessTime, + final String owner) { + super(0, true, 1, 0, modificationTime, accessTime, null, owner, owner, + path); + } + + /** + * A simple file. + * + * @param length file length + * @param modificationTime mod time + * @param path path + * @param blockSize block size + * @param owner owner + */ + OBSFileStatus( + final long length, final long modificationTime, final Path path, + final long blockSize, + final String owner) { + super(length, false, 1, blockSize, modificationTime, path); + setOwner(owner); + setGroup(owner); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java new file mode 100644 index 00000000000..dd8be1e1fe9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java @@ -0,0 +1,1562 @@ +/* + * 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.obs; + +import org.apache.hadoop.thirdparty.com.google.common.annotations.VisibleForTesting; +import org.apache.hadoop.thirdparty.com.google.common.util.concurrent.ListeningExecutorService; +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import com.obs.services.model.AccessControlList; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Options.ChecksumOpt; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.ReflectionUtils; +import org.apache.hadoop.util.SemaphoredDelegatingExecutor; +import org.apache.hadoop.util.BlockingThreadPoolExecutorService; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.util.EnumSet; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; + +/** + * The core OBS Filesystem implementation. + * + *

This subclass is marked as private as code should not be creating it + * directly; use {@link FileSystem#get(Configuration)} and variants to create + * one. + * + *

If cast to {@code OBSFileSystem}, extra methods and features may be + * accessed. Consider those private and unstable. + * + *

Because it prints some of the state of the instrumentation, the output of + * {@link #toString()} must also be considered unstable. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public final class OBSFileSystem extends FileSystem { + /** + * Class logger. + */ + public static final Logger LOG = LoggerFactory.getLogger( + OBSFileSystem.class); + + /** + * Flag indicating if the filesystem instance is closed. + */ + private final AtomicBoolean closed = new AtomicBoolean(false); + + /** + * URI of the filesystem. + */ + private URI uri; + + /** + * Current working directory of the filesystem. + */ + private Path workingDir; + + /** + * Short name of the user who instantiated the filesystem. + */ + private String username; + + /** + * OBS client instance. + */ + private ObsClient obs; + + /** + * Flag indicating if posix bucket is used. + */ + private boolean enablePosix = false; + + /** + * Flag indicating if multi-object delete recursion is enabled. + */ + private boolean enableMultiObjectDeleteRecursion = true; + + /** + * Flag indicating if OBS specific content summary is enabled. + */ + private boolean obsContentSummaryEnable = true; + + /** + * Flag indicating if OBS client specific depth first search (DFS) list is + * enabled. + */ + private boolean obsClientDFSListEnable = true; + + /** + * Bucket name. + */ + private String bucket; + + /** + * Max number of keys to get while paging through a directory listing. + */ + private int maxKeys; + + /** + * OBSListing instance. + */ + private OBSListing obsListing; + + /** + * Helper for an ongoing write operation. + */ + private OBSWriteOperationHelper writeHelper; + + /** + * Part size for multipart upload. + */ + private long partSize; + + /** + * Flag indicating if multi-object delete is enabled. + */ + private boolean enableMultiObjectDelete; + + /** + * Minimum number of objects in one multi-object delete call. + */ + private int multiDeleteThreshold; + + /** + * Maximum number of entries in one multi-object delete call. + */ + private int maxEntriesToDelete; + + /** + * Bounded thread pool for multipart upload. + */ + private ListeningExecutorService boundedMultipartUploadThreadPool; + + /** + * Bounded thread pool for copy. + */ + private ThreadPoolExecutor boundedCopyThreadPool; + + /** + * Bounded thread pool for delete. + */ + private ThreadPoolExecutor boundedDeleteThreadPool; + + /** + * Bounded thread pool for copy part. + */ + private ThreadPoolExecutor boundedCopyPartThreadPool; + + /** + * Bounded thread pool for list. + */ + private ThreadPoolExecutor boundedListThreadPool; + + /** + * List parallel factor. + */ + private int listParallelFactor; + + /** + * Read ahead range. + */ + private long readAheadRange; + + /** + * Flag indicating if {@link OBSInputStream#read(long, byte[], int, int)} will + * be transformed into {@link org.apache.hadoop.fs.FSInputStream#read(long, + * byte[], int, int)}. + */ + private boolean readTransformEnable = true; + + /** + * Factory for creating blocks. + */ + private OBSDataBlocks.BlockFactory blockFactory; + + /** + * Maximum Number of active blocks a single output stream can submit to {@link + * #boundedMultipartUploadThreadPool}. + */ + private int blockOutputActiveBlocks; + + /** + * Copy part size. + */ + private long copyPartSize; + + /** + * Flag indicating if fast delete is enabled. + */ + private boolean enableTrash = false; + + /** + * Trash directory for fast delete. + */ + private String trashDir; + + /** + * OBS redefined access control list. + */ + private AccessControlList cannedACL; + + /** + * Server-side encryption wrapper. + */ + private SseWrapper sse; + + /** + * Block size for {@link FileSystem#getDefaultBlockSize()}. + */ + private long blockSize; + + /** + * Initialize a FileSystem. Called after a new FileSystem instance is + * constructed. + * + * @param name a URI whose authority section names the host, port, + * etc. for this FileSystem + * @param originalConf the configuration to use for the FS. The + * bucket-specific options are patched over the base ones + * before any use is made of the config. + */ + @Override + public void initialize(final URI name, final Configuration originalConf) + throws IOException { + uri = URI.create(name.getScheme() + "://" + name.getAuthority()); + bucket = name.getAuthority(); + // clone the configuration into one with propagated bucket options + Configuration conf = OBSCommonUtils.propagateBucketOptions(originalConf, + bucket); + OBSCommonUtils.patchSecurityCredentialProviders(conf); + super.initialize(name, conf); + setConf(conf); + try { + + // Username is the current user at the time the FS was instantiated. + username = UserGroupInformation.getCurrentUser().getShortUserName(); + workingDir = new Path("/user", username).makeQualified(this.uri, + this.getWorkingDirectory()); + + Class obsClientFactoryClass = + conf.getClass( + OBSConstants.OBS_CLIENT_FACTORY_IMPL, + OBSConstants.DEFAULT_OBS_CLIENT_FACTORY_IMPL, + OBSClientFactory.class); + obs = ReflectionUtils.newInstance(obsClientFactoryClass, conf) + .createObsClient(name); + sse = new SseWrapper(conf); + + OBSCommonUtils.verifyBucketExists(this); + enablePosix = OBSCommonUtils.getBucketFsStatus(obs, bucket); + + maxKeys = OBSCommonUtils.intOption(conf, + OBSConstants.MAX_PAGING_KEYS, + OBSConstants.DEFAULT_MAX_PAGING_KEYS, 1); + obsListing = new OBSListing(this); + partSize = OBSCommonUtils.getMultipartSizeProperty(conf, + OBSConstants.MULTIPART_SIZE, + OBSConstants.DEFAULT_MULTIPART_SIZE); + + // check but do not store the block size + blockSize = OBSCommonUtils.longBytesOption(conf, + OBSConstants.FS_OBS_BLOCK_SIZE, + OBSConstants.DEFAULT_FS_OBS_BLOCK_SIZE, 1); + enableMultiObjectDelete = conf.getBoolean( + OBSConstants.ENABLE_MULTI_DELETE, true); + maxEntriesToDelete = conf.getInt( + OBSConstants.MULTI_DELETE_MAX_NUMBER, + OBSConstants.DEFAULT_MULTI_DELETE_MAX_NUMBER); + enableMultiObjectDeleteRecursion = conf.getBoolean( + OBSConstants.MULTI_DELETE_RECURSION, true); + obsContentSummaryEnable = conf.getBoolean( + OBSConstants.OBS_CONTENT_SUMMARY_ENABLE, true); + readAheadRange = OBSCommonUtils.longBytesOption(conf, + OBSConstants.READAHEAD_RANGE, + OBSConstants.DEFAULT_READAHEAD_RANGE, 0); + readTransformEnable = conf.getBoolean( + OBSConstants.READ_TRANSFORM_ENABLE, true); + multiDeleteThreshold = conf.getInt( + OBSConstants.MULTI_DELETE_THRESHOLD, + OBSConstants.MULTI_DELETE_DEFAULT_THRESHOLD); + + initThreadPools(conf); + + writeHelper = new OBSWriteOperationHelper(this); + + initCannedAcls(conf); + + OBSCommonUtils.initMultipartUploads(this, conf); + + String blockOutputBuffer = conf.getTrimmed( + OBSConstants.FAST_UPLOAD_BUFFER, + OBSConstants.FAST_UPLOAD_BUFFER_DISK); + partSize = OBSCommonUtils.ensureOutputParameterInRange( + OBSConstants.MULTIPART_SIZE, partSize); + blockFactory = OBSDataBlocks.createFactory(this, blockOutputBuffer); + blockOutputActiveBlocks = + OBSCommonUtils.intOption(conf, + OBSConstants.FAST_UPLOAD_ACTIVE_BLOCKS, + OBSConstants.DEFAULT_FAST_UPLOAD_ACTIVE_BLOCKS, 1); + LOG.debug( + "Using OBSBlockOutputStream with buffer = {}; block={};" + + " queue limit={}", + blockOutputBuffer, + partSize, + blockOutputActiveBlocks); + + enableTrash = conf.getBoolean(OBSConstants.TRASH_ENABLE, + OBSConstants.DEFAULT_TRASH); + if (enableTrash) { + if (!isFsBucket()) { + String errorMsg = String.format( + "The bucket [%s] is not posix. not supported for " + + "trash.", bucket); + LOG.warn(errorMsg); + enableTrash = false; + trashDir = null; + } else { + trashDir = conf.get(OBSConstants.TRASH_DIR); + if (StringUtils.isEmpty(trashDir)) { + String errorMsg = + String.format( + "The trash feature(fs.obs.trash.enable) is " + + "enabled, but the " + + "configuration(fs.obs.trash.dir [%s]) " + + "is empty.", + trashDir); + LOG.error(errorMsg); + throw new ObsException(errorMsg); + } + trashDir = OBSCommonUtils.maybeAddBeginningSlash(trashDir); + trashDir = OBSCommonUtils.maybeAddTrailingSlash(trashDir); + } + } + } catch (ObsException e) { + throw OBSCommonUtils.translateException("initializing ", + new Path(name), e); + } + } + + private void initThreadPools(final Configuration conf) { + long keepAliveTime = OBSCommonUtils.longOption(conf, + OBSConstants.KEEPALIVE_TIME, + OBSConstants.DEFAULT_KEEPALIVE_TIME, 0); + + int maxThreads = conf.getInt(OBSConstants.MAX_THREADS, + OBSConstants.DEFAULT_MAX_THREADS); + if (maxThreads < 2) { + LOG.warn(OBSConstants.MAX_THREADS + + " must be at least 2: forcing to 2."); + maxThreads = 2; + } + int totalTasks = OBSCommonUtils.intOption(conf, + OBSConstants.MAX_TOTAL_TASKS, + OBSConstants.DEFAULT_MAX_TOTAL_TASKS, 1); + boundedMultipartUploadThreadPool = + BlockingThreadPoolExecutorService.newInstance( + maxThreads, + maxThreads + totalTasks, + keepAliveTime, + TimeUnit.SECONDS, + "obs-transfer-shared"); + + int maxDeleteThreads = conf.getInt(OBSConstants.MAX_DELETE_THREADS, + OBSConstants.DEFAULT_MAX_DELETE_THREADS); + if (maxDeleteThreads < 2) { + LOG.warn(OBSConstants.MAX_DELETE_THREADS + + " must be at least 2: forcing to 2."); + maxDeleteThreads = 2; + } + int coreDeleteThreads = (int) Math.ceil(maxDeleteThreads / 2.0); + boundedDeleteThreadPool = + new ThreadPoolExecutor( + coreDeleteThreads, + maxDeleteThreads, + keepAliveTime, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + BlockingThreadPoolExecutorService.newDaemonThreadFactory( + "obs-delete-transfer-shared")); + boundedDeleteThreadPool.allowCoreThreadTimeOut(true); + + if (enablePosix) { + obsClientDFSListEnable = conf.getBoolean( + OBSConstants.OBS_CLIENT_DFS_LIST_ENABLE, true); + if (obsClientDFSListEnable) { + int coreListThreads = conf.getInt( + OBSConstants.CORE_LIST_THREADS, + OBSConstants.DEFAULT_CORE_LIST_THREADS); + int maxListThreads = conf.getInt(OBSConstants.MAX_LIST_THREADS, + OBSConstants.DEFAULT_MAX_LIST_THREADS); + int listWorkQueueCapacity = conf.getInt( + OBSConstants.LIST_WORK_QUEUE_CAPACITY, + OBSConstants.DEFAULT_LIST_WORK_QUEUE_CAPACITY); + listParallelFactor = conf.getInt( + OBSConstants.LIST_PARALLEL_FACTOR, + OBSConstants.DEFAULT_LIST_PARALLEL_FACTOR); + if (listParallelFactor < 1) { + LOG.warn(OBSConstants.LIST_PARALLEL_FACTOR + + " must be at least 1: forcing to 1."); + listParallelFactor = 1; + } + boundedListThreadPool = + new ThreadPoolExecutor( + coreListThreads, + maxListThreads, + keepAliveTime, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(listWorkQueueCapacity), + BlockingThreadPoolExecutorService + .newDaemonThreadFactory( + "obs-list-transfer-shared")); + boundedListThreadPool.allowCoreThreadTimeOut(true); + } + } else { + int maxCopyThreads = conf.getInt(OBSConstants.MAX_COPY_THREADS, + OBSConstants.DEFAULT_MAX_COPY_THREADS); + if (maxCopyThreads < 2) { + LOG.warn(OBSConstants.MAX_COPY_THREADS + + " must be at least 2: forcing to 2."); + maxCopyThreads = 2; + } + int coreCopyThreads = (int) Math.ceil(maxCopyThreads / 2.0); + boundedCopyThreadPool = + new ThreadPoolExecutor( + coreCopyThreads, + maxCopyThreads, + keepAliveTime, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + BlockingThreadPoolExecutorService.newDaemonThreadFactory( + "obs-copy-transfer-shared")); + boundedCopyThreadPool.allowCoreThreadTimeOut(true); + + copyPartSize = OBSCommonUtils.longOption(conf, + OBSConstants.COPY_PART_SIZE, + OBSConstants.DEFAULT_COPY_PART_SIZE, 0); + if (copyPartSize > OBSConstants.MAX_COPY_PART_SIZE) { + LOG.warn( + "obs: {} capped to ~5GB (maximum allowed part size with " + + "current output mechanism)", + OBSConstants.COPY_PART_SIZE); + copyPartSize = OBSConstants.MAX_COPY_PART_SIZE; + } + + int maxCopyPartThreads = conf.getInt( + OBSConstants.MAX_COPY_PART_THREADS, + OBSConstants.DEFAULT_MAX_COPY_PART_THREADS); + if (maxCopyPartThreads < 2) { + LOG.warn(OBSConstants.MAX_COPY_PART_THREADS + + " must be at least 2: forcing to 2."); + maxCopyPartThreads = 2; + } + int coreCopyPartThreads = (int) Math.ceil(maxCopyPartThreads / 2.0); + boundedCopyPartThreadPool = + new ThreadPoolExecutor( + coreCopyPartThreads, + maxCopyPartThreads, + keepAliveTime, + TimeUnit.SECONDS, + new LinkedBlockingQueue<>(), + BlockingThreadPoolExecutorService.newDaemonThreadFactory( + "obs-copy-part-transfer-shared")); + boundedCopyPartThreadPool.allowCoreThreadTimeOut(true); + } + } + + /** + * Is posix bucket or not. + * + * @return is it posix bucket + */ + boolean isFsBucket() { + return enablePosix; + } + + /** + * Get read transform switch stat. + * + * @return is read transform enabled + */ + boolean isReadTransformEnabled() { + return readTransformEnable; + } + + /** + * Initialize bucket acl for upload, write operation. + * + * @param conf the configuration to use for the FS. + */ + private void initCannedAcls(final Configuration conf) { + // No canned acl in obs + String cannedACLName = conf.get(OBSConstants.CANNED_ACL, + OBSConstants.DEFAULT_CANNED_ACL); + if (!cannedACLName.isEmpty()) { + switch (cannedACLName) { + case "Private": + case "PublicRead": + case "PublicReadWrite": + case "AuthenticatedRead": + case "LogDeliveryWrite": + case "BucketOwnerRead": + case "BucketOwnerFullControl": + cannedACL = new AccessControlList(); + break; + default: + cannedACL = null; + } + } else { + cannedACL = null; + } + } + + /** + * Get the bucket acl of user setting. + * + * @return bucket acl {@link AccessControlList} + */ + AccessControlList getCannedACL() { + return cannedACL; + } + + /** + * Return the protocol scheme for the FileSystem. + * + * @return "obs" + */ + @Override + public String getScheme() { + return "obs"; + } + + /** + * Return a URI whose scheme and authority identify this FileSystem. + * + * @return the URI of this filesystem. + */ + @Override + public URI getUri() { + return uri; + } + + /** + * Return the default port for this FileSystem. + * + * @return -1 to indicate the port is undefined, which agrees with the + * contract of {@link URI#getPort()} + */ + @Override + public int getDefaultPort() { + return OBSConstants.OBS_DEFAULT_PORT; + } + + /** + * Return the OBS client used by this filesystem. + * + * @return OBS client + */ + @VisibleForTesting + ObsClient getObsClient() { + return obs; + } + + /** + * Return the read ahead range used by this filesystem. + * + * @return read ahead range + */ + @VisibleForTesting + long getReadAheadRange() { + return readAheadRange; + } + + /** + * Return the bucket of this filesystem. + * + * @return the bucket + */ + String getBucket() { + return bucket; + } + + /** + * Check that a Path belongs to this FileSystem. Unlike the superclass, this + * version does not look at authority, but only hostname. + * + * @param path the path to check + * @throws IllegalArgumentException if there is an FS mismatch + */ + @Override + public void checkPath(final Path path) { + OBSLoginHelper.checkPath(getConf(), getUri(), path, getDefaultPort()); + } + + /** + * Canonicalize the given URI. + * + * @param rawUri the URI to be canonicalized + * @return the canonicalized URI + */ + @Override + protected URI canonicalizeUri(final URI rawUri) { + return OBSLoginHelper.canonicalizeUri(rawUri, getDefaultPort()); + } + + /** + * Open an FSDataInputStream at the indicated Path. + * + * @param f the file path to open + * @param bufferSize the size of the buffer to be used + * @return the FSDataInputStream for the file + * @throws IOException on any failure to open the file + */ + @Override + public FSDataInputStream open(final Path f, final int bufferSize) + throws IOException { + LOG.debug("Opening '{}' for reading.", f); + final FileStatus fileStatus = getFileStatus(f); + if (fileStatus.isDirectory()) { + throw new FileNotFoundException( + "Can't open " + f + " because it is a directory"); + } + + return new FSDataInputStream( + new OBSInputStream(bucket, OBSCommonUtils.pathToKey(this, f), + fileStatus.getLen(), + obs, statistics, readAheadRange, this)); + } + + /** + * Create an FSDataOutputStream at the indicated Path with write-progress + * reporting. + * + * @param f the file path to create + * @param permission the permission to set + * @param overwrite if a file with this name already exists, then if true, + * the file will be overwritten, and if false an error will + * be thrown + * @param bufferSize the size of the buffer to be used + * @param replication required block replication for the file + * @param blkSize the requested block size + * @param progress the progress reporter + * @throws IOException on any failure to create the file + * @see #setPermission(Path, FsPermission) + */ + @Override + public FSDataOutputStream create( + final Path f, + final FsPermission permission, + final boolean overwrite, + final int bufferSize, + final short replication, + final long blkSize, + final Progressable progress) + throws IOException { + String key = OBSCommonUtils.pathToKey(this, f); + FileStatus status; + long objectLen = 0; + try { + // get the status or throw an exception + status = getFileStatus(f); + objectLen = status.getLen(); + // if the thread reaches here, there is something at the path + if (status.isDirectory()) { + // path references a directory: automatic error + throw new FileAlreadyExistsException(f + " is a directory"); + } + if (!overwrite) { + // path references a file and overwrite is disabled + throw new FileAlreadyExistsException(f + " already exists"); + } + LOG.debug("create: Overwriting file {}", f); + } catch (FileNotFoundException e) { + // this means the file is not found + LOG.debug("create: Creating new file {}", f); + } + return new FSDataOutputStream( + new OBSBlockOutputStream( + this, + key, + objectLen, + new SemaphoredDelegatingExecutor( + boundedMultipartUploadThreadPool, + blockOutputActiveBlocks, true), + false), + null); + } + + /** + * Return the part size for multipart upload used by {@link + * OBSBlockOutputStream}. + * + * @return the part size + */ + long getPartSize() { + return partSize; + } + + /** + * Return the block factory used by {@link OBSBlockOutputStream}. + * + * @return the block factory + */ + OBSDataBlocks.BlockFactory getBlockFactory() { + return blockFactory; + } + + /** + * Return the write helper used by {@link OBSBlockOutputStream}. + * + * @return the write helper + */ + OBSWriteOperationHelper getWriteHelper() { + return writeHelper; + } + + /** + * Create an FSDataOutputStream at the indicated Path with write-progress + * reporting. + * + * @param f the file name to create + * @param permission permission of + * @param flags {@link CreateFlag}s to use for this stream + * @param bufferSize the size of the buffer to be used + * @param replication required block replication for the file + * @param blkSize block size + * @param progress progress + * @param checksumOpt check sum option + * @throws IOException io exception + */ + @Override + @SuppressWarnings("checkstyle:parameternumber") + public FSDataOutputStream create( + final Path f, + final FsPermission permission, + final EnumSet flags, + final int bufferSize, + final short replication, + final long blkSize, + final Progressable progress, + final ChecksumOpt checksumOpt) + throws IOException { + LOG.debug("create: Creating new file {}, flags:{}, isFsBucket:{}", f, + flags, isFsBucket()); + if (null != flags && flags.contains(CreateFlag.APPEND)) { + if (!isFsBucket()) { + throw new UnsupportedOperationException( + "non-posix bucket. Append is not supported by " + + "OBSFileSystem"); + } + String key = OBSCommonUtils.pathToKey(this, f); + FileStatus status; + long objectLen = 0; + try { + // get the status or throw an FNFE + status = getFileStatus(f); + objectLen = status.getLen(); + // if the thread reaches here, there is something at the path + if (status.isDirectory()) { + // path references a directory: automatic error + throw new FileAlreadyExistsException(f + " is a directory"); + } + } catch (FileNotFoundException e) { + LOG.debug("FileNotFoundException, create: Creating new file {}", + f); + } + + return new FSDataOutputStream( + new OBSBlockOutputStream( + this, + key, + objectLen, + new SemaphoredDelegatingExecutor( + boundedMultipartUploadThreadPool, + blockOutputActiveBlocks, true), + true), + null); + } else { + return create( + f, + permission, + flags == null || flags.contains(CreateFlag.OVERWRITE), + bufferSize, + replication, + blkSize, + progress); + } + } + + /** + * Open an FSDataOutputStream at the indicated Path with write-progress + * reporting. Same as create(), except fails if parent directory doesn't + * already exist. + * + * @param path the file path to create + * @param permission file permission + * @param flags {@link CreateFlag}s to use for this stream + * @param bufferSize the size of the buffer to be used + * @param replication required block replication for the file + * @param blkSize block size + * @param progress the progress reporter + * @throws IOException IO failure + */ + @Override + public FSDataOutputStream createNonRecursive( + final Path path, + final FsPermission permission, + final EnumSet flags, + final int bufferSize, + final short replication, + final long blkSize, + final Progressable progress) + throws IOException { + Path parent = path.getParent(); + if (parent != null && !getFileStatus(parent).isDirectory()) { + // expect this to raise an exception if there is no parent + throw new FileAlreadyExistsException("Not a directory: " + parent); + } + return create( + path, + permission, + flags.contains(CreateFlag.OVERWRITE), + bufferSize, + replication, + blkSize, + progress); + } + + /** + * Append to an existing file (optional operation). + * + * @param f the existing file to be appended + * @param bufferSize the size of the buffer to be used + * @param progress for reporting progress if it is not null + * @throws IOException indicating that append is not supported + */ + @Override + public FSDataOutputStream append(final Path f, final int bufferSize, + final Progressable progress) + throws IOException { + if (!isFsBucket()) { + throw new UnsupportedOperationException( + "non-posix bucket. Append is not supported " + + "by OBSFileSystem"); + } + LOG.debug("append: Append file {}.", f); + String key = OBSCommonUtils.pathToKey(this, f); + + // get the status or throw an FNFE + FileStatus status = getFileStatus(f); + long objectLen = status.getLen(); + // if the thread reaches here, there is something at the path + if (status.isDirectory()) { + // path references a directory: automatic error + throw new FileAlreadyExistsException(f + " is a directory"); + } + + return new FSDataOutputStream( + new OBSBlockOutputStream( + this, + key, + objectLen, + new SemaphoredDelegatingExecutor( + boundedMultipartUploadThreadPool, + blockOutputActiveBlocks, true), + true), + null); + } + + /** + * Check if a path exists. + * + * @param f source path + * @return true if the path exists + * @throws IOException IO failure + */ + @Override + public boolean exists(final Path f) throws IOException { + try { + return getFileStatus(f) != null; + } catch (FileNotFoundException | FileConflictException e) { + return false; + } + } + + /** + * Rename Path src to Path dst. + * + * @param src path to be renamed + * @param dst new path after rename + * @return true if rename is successful + * @throws IOException on IO failure + */ + @Override + public boolean rename(final Path src, final Path dst) throws IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + LOG.debug("Rename path {} to {} start", src, dst); + try { + if (enablePosix) { + return OBSPosixBucketUtils.renameBasedOnPosix(this, src, dst); + } else { + return OBSObjectBucketUtils.renameBasedOnObject(this, src, dst); + } + } catch (ObsException e) { + throw OBSCommonUtils.translateException( + "rename(" + src + ", " + dst + ")", src, e); + } catch (RenameFailedException e) { + LOG.error(e.getMessage()); + return e.getExitCode(); + } catch (FileNotFoundException e) { + LOG.error(e.toString()); + return false; + } finally { + long endTime = System.currentTimeMillis(); + LOG.debug( + "Rename path {} to {} finished, thread:{}, " + + "timeUsedInMilliSec:{}.", src, dst, threadId, + endTime - startTime); + } + } + + /** + * Return maximum number of entries in one multi-object delete call. + * + * @return the maximum number of entries in one multi-object delete call + */ + int getMaxEntriesToDelete() { + return maxEntriesToDelete; + } + + /** + * Return list parallel factor. + * + * @return the list parallel factor + */ + int getListParallelFactor() { + return listParallelFactor; + } + + /** + * Return bounded thread pool for list. + * + * @return bounded thread pool for list + */ + ThreadPoolExecutor getBoundedListThreadPool() { + return boundedListThreadPool; + } + + /** + * Return a flag that indicates if OBS client specific depth first search + * (DFS) list is enabled. + * + * @return the flag + */ + boolean isObsClientDFSListEnable() { + return obsClientDFSListEnable; + } + + /** + * Return the {@link Statistics} instance used by this filesystem. + * + * @return the used {@link Statistics} instance + */ + Statistics getSchemeStatistics() { + return statistics; + } + + /** + * Return the minimum number of objects in one multi-object delete call. + * + * @return the minimum number of objects in one multi-object delete call + */ + int getMultiDeleteThreshold() { + return multiDeleteThreshold; + } + + /** + * Return a flag that indicates if multi-object delete is enabled. + * + * @return the flag + */ + boolean isEnableMultiObjectDelete() { + return enableMultiObjectDelete; + } + + /** + * Delete a Path. This operation is at least {@code O(files)}, with added + * overheads to enumerate the path. It is also not atomic. + * + * @param f the path to delete + * @param recursive if path is a directory and set to true, the directory is + * deleted else throws an exception. In case of a file the + * recursive can be set to either true or false + * @return true if delete is successful else false + * @throws IOException due to inability to delete a directory or file + */ + @Override + public boolean delete(final Path f, final boolean recursive) + throws IOException { + try { + FileStatus status = getFileStatus(f); + LOG.debug("delete: path {} - recursive {}", status.getPath(), + recursive); + + if (enablePosix) { + return OBSPosixBucketUtils.fsDelete(this, status, recursive); + } + + return OBSObjectBucketUtils.objectDelete(this, status, recursive); + } catch (FileNotFoundException e) { + LOG.warn("Couldn't delete {} - does not exist", f); + return false; + } catch (ObsException e) { + throw OBSCommonUtils.translateException("delete", f, e); + } + } + + /** + * Return a flag that indicates if fast delete is enabled. + * + * @return the flag + */ + boolean isEnableTrash() { + return enableTrash; + } + + /** + * Return trash directory for fast delete. + * + * @return the trash directory + */ + String getTrashDir() { + return trashDir; + } + + /** + * Return a flag that indicates if multi-object delete recursion is enabled. + * + * @return the flag + */ + boolean isEnableMultiObjectDeleteRecursion() { + return enableMultiObjectDeleteRecursion; + } + + /** + * List the statuses of the files/directories in the given path if the path is + * a directory. + * + * @param f given path + * @return the statuses of the files/directories in the given patch + * @throws FileNotFoundException when the path does not exist + * @throws IOException see specific implementation + */ + @Override + public FileStatus[] listStatus(final Path f) + throws FileNotFoundException, IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + try { + FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f, + false); + long endTime = System.currentTimeMillis(); + LOG.debug( + "List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f, + threadId, endTime - startTime); + return statuses; + } catch (ObsException e) { + throw OBSCommonUtils.translateException("listStatus", f, e); + } + } + + /** + * This public interface is provided specially for Huawei MRS. List the + * statuses of the files/directories in the given path if the path is a + * directory. When recursive is true, iterator all objects in the given path + * and its sub directories. + * + * @param f given path + * @param recursive whether to iterator objects in sub direcotries + * @return the statuses of the files/directories in the given patch + * @throws FileNotFoundException when the path does not exist + * @throws IOException see specific implementation + */ + public FileStatus[] listStatus(final Path f, final boolean recursive) + throws FileNotFoundException, IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + try { + FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f, + recursive); + long endTime = System.currentTimeMillis(); + LOG.debug( + "List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f, + threadId, endTime - startTime); + return statuses; + } catch (ObsException e) { + throw OBSCommonUtils.translateException( + "listStatus with recursive flag[" + + (recursive ? "true] " : "false] "), f, e); + } + } + + /** + * Return the OBSListing instance used by this filesystem. + * + * @return the OBSListing instance + */ + OBSListing getObsListing() { + return obsListing; + } + + /** + * Return the current working directory for the given file system. + * + * @return the directory pathname + */ + @Override + public Path getWorkingDirectory() { + return workingDir; + } + + /** + * Set the current working directory for the file system. All relative paths + * will be resolved relative to it. + * + * @param newDir the new working directory + */ + @Override + public void setWorkingDirectory(final Path newDir) { + workingDir = newDir; + } + + /** + * Return the username of the filesystem. + * + * @return the short name of the user who instantiated the filesystem + */ + String getUsername() { + return username; + } + + /** + * Make the given path and all non-existent parents into directories. Has the + * semantics of Unix {@code 'mkdir -p'}. Existence of the directory hierarchy + * is not an error. + * + * @param path path to create + * @param permission to apply to f + * @return true if a directory was created + * @throws FileAlreadyExistsException there is a file at the path specified + * @throws IOException other IO problems + */ + @Override + public boolean mkdirs(final Path path, final FsPermission permission) + throws IOException, FileAlreadyExistsException { + try { + return OBSCommonUtils.innerMkdirs(this, path); + } catch (ObsException e) { + throw OBSCommonUtils.translateException("mkdirs", path, e); + } + } + + /** + * Return a file status object that represents the path. + * + * @param f the path we want information from + * @return a FileStatus object + * @throws FileNotFoundException when the path does not exist + * @throws IOException on other problems + */ + @Override + public FileStatus getFileStatus(final Path f) + throws FileNotFoundException, IOException { + for (int retryTime = 1; + retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) { + try { + return innerGetFileStatus(f); + } catch (FileNotFoundException | FileConflictException e) { + throw e; + } catch (IOException e) { + LOG.warn("Failed to get file status for [{}], retry time [{}], " + + "exception [{}]", f, retryTime, e); + + try { + Thread.sleep(OBSCommonUtils.DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + } + + return innerGetFileStatus(f); + } + + /** + * Inner implementation without retry for {@link #getFileStatus(Path)}. + * + * @param f the path we want information from + * @return a FileStatus object + * @throws IOException on IO failure + */ + @VisibleForTesting + OBSFileStatus innerGetFileStatus(final Path f) throws IOException { + if (enablePosix) { + return OBSPosixBucketUtils.innerFsGetObjectStatus(this, f); + } + + return OBSObjectBucketUtils.innerGetObjectStatus(this, f); + } + + /** + * Return the {@link ContentSummary} of a given {@link Path}. + * + * @param f path to use + * @return the {@link ContentSummary} + * @throws FileNotFoundException if the path does not resolve + * @throws IOException IO failure + */ + @Override + public ContentSummary getContentSummary(final Path f) + throws FileNotFoundException, IOException { + if (!obsContentSummaryEnable) { + return super.getContentSummary(f); + } + + FileStatus status = getFileStatus(f); + if (status.isFile()) { + // f is a file + long length = status.getLen(); + return new ContentSummary.Builder().length(length) + .fileCount(1).directoryCount(0).spaceConsumed(length).build(); + } + + // f is a directory + if (enablePosix) { + return OBSPosixBucketUtils.fsGetDirectoryContentSummary(this, + OBSCommonUtils.pathToKey(this, f)); + } else { + return OBSObjectBucketUtils.getDirectoryContentSummary(this, + OBSCommonUtils.pathToKey(this, f)); + } + } + + /** + * Copy the {@code src} file on the local disk to the filesystem at the given + * {@code dst} name. + * + * @param delSrc whether to delete the src + * @param overwrite whether to overwrite an existing file + * @param src path + * @param dst path + * @throws FileAlreadyExistsException if the destination file exists and + * overwrite == false + * @throws IOException IO problem + */ + @Override + public void copyFromLocalFile(final boolean delSrc, final boolean overwrite, + final Path src, final Path dst) throws FileAlreadyExistsException, + IOException { + try { + super.copyFromLocalFile(delSrc, overwrite, src, dst); + } catch (ObsException e) { + throw OBSCommonUtils.translateException( + "copyFromLocalFile(" + src + ", " + dst + ")", src, e); + } + } + + /** + * Close the filesystem. This shuts down all transfers. + * + * @throws IOException IO problem + */ + @Override + public void close() throws IOException { + LOG.debug("This Filesystem closed by user, clear resource."); + if (closed.getAndSet(true)) { + // already closed + return; + } + + try { + super.close(); + } finally { + OBSCommonUtils.shutdownAll( + boundedMultipartUploadThreadPool, + boundedCopyThreadPool, + boundedDeleteThreadPool, + boundedCopyPartThreadPool, + boundedListThreadPool); + } + } + + /** + * Override {@code getCanonicalServiceName} and return {@code null} since + * delegation token is not supported. + */ + @Override + public String getCanonicalServiceName() { + // Does not support Token + return null; + } + + /** + * Return copy part size. + * + * @return copy part size + */ + long getCopyPartSize() { + return copyPartSize; + } + + /** + * Return bounded thread pool for copy part. + * + * @return the bounded thread pool for copy part + */ + ThreadPoolExecutor getBoundedCopyPartThreadPool() { + return boundedCopyPartThreadPool; + } + + /** + * Return bounded thread pool for copy. + * + * @return the bounded thread pool for copy + */ + ThreadPoolExecutor getBoundedCopyThreadPool() { + return boundedCopyThreadPool; + } + + /** + * Imitate HDFS to return the number of bytes that large input files should be + * optimally split into to minimize I/O time for compatibility. + * + * @deprecated use {@link #getDefaultBlockSize(Path)} instead + */ + @Override + public long getDefaultBlockSize() { + return blockSize; + } + + /** + * Imitate HDFS to return the number of bytes that large input files should be + * optimally split into to minimize I/O time. The given path will be used to + * locate the actual filesystem. The full path does not have to exist. + * + * @param f path of file + * @return the default block size for the path's filesystem + */ + @Override + public long getDefaultBlockSize(final Path f) { + return blockSize; + } + + /** + * Return a string that describes this filesystem instance. + * + * @return the string + */ + @Override + public String toString() { + final StringBuilder sb = new StringBuilder("OBSFileSystem{"); + sb.append("uri=").append(uri); + sb.append(", workingDir=").append(workingDir); + sb.append(", partSize=").append(partSize); + sb.append(", enableMultiObjectsDelete=") + .append(enableMultiObjectDelete); + sb.append(", maxKeys=").append(maxKeys); + if (cannedACL != null) { + sb.append(", cannedACL=").append(cannedACL.toString()); + } + sb.append(", readAheadRange=").append(readAheadRange); + sb.append(", blockSize=").append(getDefaultBlockSize()); + if (blockFactory != null) { + sb.append(", blockFactory=").append(blockFactory); + } + sb.append(", boundedMultipartUploadThreadPool=") + .append(boundedMultipartUploadThreadPool); + sb.append(", statistics {").append(statistics).append("}"); + sb.append(", metrics {").append("}"); + sb.append('}'); + return sb.toString(); + } + + /** + * Return the maximum number of keys to get while paging through a directory + * listing. + * + * @return the maximum number of keys + */ + int getMaxKeys() { + return maxKeys; + } + + /** + * List the statuses and block locations of the files in the given path. Does + * not guarantee to return the iterator that traverses statuses of the files + * in a sorted order. + * + *

+   * If the path is a directory,
+   *   if recursive is false, returns files in the directory;
+   *   if recursive is true, return files in the subtree rooted at the path.
+   * If the path is a file, return the file's status and block locations.
+   * 
+ * + * @param f a path + * @param recursive if the subdirectories need to be traversed recursively + * @return an iterator that traverses statuses of the files/directories in the + * given path + * @throws FileNotFoundException if {@code path} does not exist + * @throws IOException if any I/O error occurred + */ + @Override + public RemoteIterator listFiles(final Path f, + final boolean recursive) + throws FileNotFoundException, IOException { + Path path = OBSCommonUtils.qualify(this, f); + LOG.debug("listFiles({}, {})", path, recursive); + try { + // lookup dir triggers existence check + final FileStatus fileStatus = getFileStatus(path); + if (fileStatus.isFile()) { + // simple case: File + LOG.debug("Path is a file"); + return new OBSListing + .SingleStatusRemoteIterator( + OBSCommonUtils.toLocatedFileStatus(this, fileStatus)); + } else { + LOG.debug( + "listFiles: doing listFiles of directory {} - recursive {}", + path, recursive); + // directory: do a bulk operation + String key = OBSCommonUtils.maybeAddTrailingSlash( + OBSCommonUtils.pathToKey(this, path)); + String delimiter = recursive ? null : "/"; + LOG.debug("Requesting all entries under {} with delimiter '{}'", + key, delimiter); + return obsListing.createLocatedFileStatusIterator( + obsListing.createFileStatusListingIterator( + path, + OBSCommonUtils.createListObjectsRequest(this, key, + delimiter), + OBSListing.ACCEPT_ALL, + new OBSListing.AcceptFilesOnly(path))); + } + } catch (ObsException e) { + throw OBSCommonUtils.translateException("listFiles", path, e); + } + } + + /** + * List the statuses of the files/directories in the given path if the path is + * a directory. Return the file's status and block locations If the path is a + * file. + *

+ * If a returned status is a file, it contains the file's block locations. + * + * @param f is the path + * @return an iterator that traverses statuses of the files/directories in the + * given path + * @throws FileNotFoundException If f does not exist + * @throws IOException If an I/O error occurred + */ + @Override + public RemoteIterator listLocatedStatus(final Path f) + throws FileNotFoundException, IOException { + return listLocatedStatus(f, + OBSListing.ACCEPT_ALL); + } + + /** + * List a directory. The returned results include its block location if it is + * a file The results are filtered by the given path filter + * + * @param f a path + * @param filter a path filter + * @return an iterator that traverses statuses of the files/directories in the + * given path + * @throws FileNotFoundException if f does not exist + * @throws IOException if any I/O error occurred + */ + @Override + public RemoteIterator listLocatedStatus(final Path f, + final PathFilter filter) + throws FileNotFoundException, IOException { + Path path = OBSCommonUtils.qualify(this, f); + LOG.debug("listLocatedStatus({}, {}", path, filter); + try { + // lookup dir triggers existence check + final FileStatus fileStatus = getFileStatus(path); + if (fileStatus.isFile()) { + // simple case: File + LOG.debug("Path is a file"); + return new OBSListing.SingleStatusRemoteIterator( + filter.accept(path) ? OBSCommonUtils.toLocatedFileStatus( + this, fileStatus) : null); + } else { + // directory: trigger a lookup + String key = OBSCommonUtils.maybeAddTrailingSlash( + OBSCommonUtils.pathToKey(this, path)); + return obsListing.createLocatedFileStatusIterator( + obsListing.createFileStatusListingIterator( + path, + OBSCommonUtils.createListObjectsRequest(this, key, "/"), + filter, + new OBSListing.AcceptAllButSelfAndS3nDirs(path))); + } + } catch (ObsException e) { + throw OBSCommonUtils.translateException("listLocatedStatus", path, + e); + } + } + + /** + * Return server-side encryption wrapper used by this filesystem instance. + * + * @return the server-side encryption wrapper + */ + SseWrapper getSse() { + return sse; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java new file mode 100644 index 00000000000..bbf29df14f3 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java @@ -0,0 +1,744 @@ +/* + * 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.obs; + +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObjectMetadata; +import com.obs.services.model.ObsObject; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; +import java.util.ListIterator; +import java.util.Queue; +import java.util.Stack; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * OBS depth first search listing implementation for posix bucket. + */ +class OBSFsDFSListing extends ObjectListing { + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + OBSFsDFSListing.class); + + static void increaseLevelStats(final List levelStatsList, + final int level, + final boolean isDir) { + int currMaxLevel = levelStatsList.size() - 1; + if (currMaxLevel < level) { + for (int i = 0; i < level - currMaxLevel; i++) { + levelStatsList.add(new LevelStats(currMaxLevel + 1 + i)); + } + } + + if (isDir) { + levelStatsList.get(level).increaseDirNum(); + } else { + levelStatsList.get(level).increaseFileNum(); + } + } + + static String fsDFSListNextBatch(final OBSFileSystem owner, + final Stack listStack, + final Queue resultQueue, + final String marker, + final int maxKeyNum, + final List objectSummaries, + final List levelStatsList) throws IOException { + // 0. check if marker matches with the peek of result queue when marker + // is given + if (marker != null) { + if (resultQueue.isEmpty()) { + throw new IllegalArgumentException( + "result queue is empty, but marker is not empty: " + + marker); + } else if (resultQueue.peek().getType() + == ListEntityType.LIST_TAIL) { + throw new RuntimeException( + "cannot put list tail (" + resultQueue.peek() + + ") into result queue"); + } else if (!marker.equals( + resultQueue.peek().getType() == ListEntityType.COMMON_PREFIX + ? resultQueue.peek().getCommonPrefix() + : resultQueue.peek().getObjectSummary().getObjectKey())) { + throw new IllegalArgumentException("marker (" + marker + + ") does not match with result queue peek (" + + resultQueue.peek() + ")"); + } + } + + // 1. fetch some list results from local result queue + int resultNum = fetchListResultLocally(owner.getBucket(), resultQueue, + maxKeyNum, objectSummaries, + levelStatsList); + + // 2. fetch more list results by doing one-level lists in parallel + fetchListResultRemotely(owner, listStack, resultQueue, maxKeyNum, + objectSummaries, levelStatsList, resultNum); + + // 3. check if list operation ends + if (!listStack.empty() && resultQueue.isEmpty()) { + throw new RuntimeException( + "result queue is empty, but list stack is not empty: " + + listStack); + } + + String nextMarker = null; + if (!resultQueue.isEmpty()) { + if (resultQueue.peek().getType() == ListEntityType.LIST_TAIL) { + throw new RuntimeException( + "cannot put list tail (" + resultQueue.peek() + + ") into result queue"); + } else { + nextMarker = + resultQueue.peek().getType() == ListEntityType.COMMON_PREFIX + ? resultQueue + .peek().getCommonPrefix() + : resultQueue.peek().getObjectSummary().getObjectKey(); + } + } + return nextMarker; + } + + static void fetchListResultRemotely(final OBSFileSystem owner, + final Stack listStack, + final Queue resultQueue, final int maxKeyNum, + final List objectSummaries, + final List levelStatsList, + final int resultNum) throws IOException { + int newResultNum = resultNum; + while (!listStack.empty() && (newResultNum < maxKeyNum + || resultQueue.isEmpty())) { + List oneLevelListRequests = new ArrayList<>(); + List> oneLevelListFutures = new ArrayList<>(); + List levels = new ArrayList<>(); + List oneLevelObjectListings = new ArrayList<>(); + // a. submit some one-level list tasks in parallel + submitOneLevelListTasks(owner, listStack, maxKeyNum, + oneLevelListRequests, oneLevelListFutures, levels); + + // b. wait these tasks to complete + waitForOneLevelListTasksFinished(oneLevelListRequests, + oneLevelListFutures, oneLevelObjectListings); + + // c. put subdir/file into result commonPrefixes and + // objectSummaries;if the number of results reaches maxKeyNum, + // cache it into resultQueue for next list batch note: unlike + // standard DFS, we put subdir directly into result list to avoid + // caching it using more space + newResultNum = handleOneLevelListTaskResult(resultQueue, maxKeyNum, + objectSummaries, levelStatsList, newResultNum, + oneLevelListRequests, levels, oneLevelObjectListings); + + // d. push subdirs and list continuing tail/end into list stack in + // reversed order,so that we can pop them from the stack in order + // later + addNewListStackEntities(listStack, oneLevelListRequests, levels, + oneLevelObjectListings); + } + } + + @SuppressWarnings("checkstyle:parameternumber") + static int handleOneLevelListTaskResult(final Queue resultQueue, + final int maxKeyNum, + final List objectSummaries, + final List levelStatsList, + final int resultNum, + final List oneLevelListRequests, + final List levels, + final List oneLevelObjectListings) { + int newResultNum = resultNum; + for (int i = 0; i < oneLevelObjectListings.size(); i++) { + LOG.debug( + "one level listing with prefix=" + oneLevelListRequests.get(i) + .getPrefix() + + ", marker=" + ( + oneLevelListRequests.get(i).getMarker() != null + ? oneLevelListRequests.get(i) + .getMarker() + : "")); + + ObjectListing oneLevelObjectListing = oneLevelObjectListings.get(i); + LOG.debug("# of CommonPrefixes/Objects: {}/{}", + oneLevelObjectListing.getCommonPrefixes().size(), + oneLevelObjectListing.getObjects().size()); + + if (oneLevelObjectListing.getCommonPrefixes().isEmpty() + && oneLevelObjectListing.getObjects().isEmpty()) { + continue; + } + + for (String commonPrefix + : oneLevelObjectListing.getCommonPrefixes()) { + if (commonPrefix.equals( + oneLevelListRequests.get(i).getPrefix())) { + // skip prefix itself + continue; + } + + LOG.debug("common prefix: " + commonPrefix); + if (newResultNum < maxKeyNum) { + addCommonPrefixIntoObjectList( + oneLevelListRequests.get(i).getBucketName(), + objectSummaries, + commonPrefix); + increaseLevelStats(levelStatsList, levels.get(i), true); + newResultNum++; + } else { + resultQueue.add( + new ListEntity(commonPrefix, levels.get(i))); + } + } + + for (ObsObject obj : oneLevelObjectListing.getObjects()) { + if (obj.getObjectKey() + .equals(oneLevelListRequests.get(i).getPrefix())) { + // skip prefix itself + continue; + } + + LOG.debug("object: {}, size: {}", obj.getObjectKey(), + obj.getMetadata().getContentLength()); + if (newResultNum < maxKeyNum) { + objectSummaries.add(obj); + increaseLevelStats(levelStatsList, levels.get(i), + obj.getObjectKey().endsWith("/")); + newResultNum++; + } else { + resultQueue.add(new ListEntity(obj, levels.get(i))); + } + } + } + return newResultNum; + } + + static void waitForOneLevelListTasksFinished( + final List oneLevelListRequests, + final List> oneLevelListFutures, + final List oneLevelObjectListings) + throws IOException { + for (int i = 0; i < oneLevelListFutures.size(); i++) { + try { + oneLevelObjectListings.add(oneLevelListFutures.get(i).get()); + } catch (InterruptedException e) { + LOG.warn("Interrupted while listing using DFS, prefix=" + + oneLevelListRequests.get(i).getPrefix() + ", marker=" + + (oneLevelListRequests.get(i).getMarker() != null + ? oneLevelListRequests.get(i).getMarker() + : "")); + throw new InterruptedIOException( + "Interrupted while listing using DFS, prefix=" + + oneLevelListRequests.get(i).getPrefix() + ", marker=" + + (oneLevelListRequests.get(i).getMarker() != null + ? oneLevelListRequests.get(i).getMarker() + : "")); + } catch (ExecutionException e) { + LOG.error("Exception while listing using DFS, prefix=" + + oneLevelListRequests.get(i).getPrefix() + ", marker=" + + (oneLevelListRequests.get(i).getMarker() != null + ? oneLevelListRequests.get(i).getMarker() + : ""), + e); + for (Future future : oneLevelListFutures) { + future.cancel(true); + } + + throw OBSCommonUtils.extractException( + "Listing using DFS with exception, marker=" + + (oneLevelListRequests.get(i).getMarker() != null + ? oneLevelListRequests.get(i).getMarker() + : ""), + oneLevelListRequests.get(i).getPrefix(), e); + } + } + } + + static void submitOneLevelListTasks(final OBSFileSystem owner, + final Stack listStack, final int maxKeyNum, + final List oneLevelListRequests, + final List> oneLevelListFutures, + final List levels) { + for (int i = 0; + i < owner.getListParallelFactor() && !listStack.empty(); i++) { + ListEntity listEntity = listStack.pop(); + if (listEntity.getType() == ListEntityType.LIST_TAIL) { + if (listEntity.getNextMarker() != null) { + ListObjectsRequest oneLevelListRequest + = new ListObjectsRequest(); + oneLevelListRequest.setBucketName(owner.getBucket()); + oneLevelListRequest.setPrefix(listEntity.getPrefix()); + oneLevelListRequest.setMarker(listEntity.getNextMarker()); + oneLevelListRequest.setMaxKeys( + Math.min(maxKeyNum, owner.getMaxKeys())); + oneLevelListRequest.setDelimiter("/"); + oneLevelListRequests.add(oneLevelListRequest); + oneLevelListFutures.add(owner.getBoundedListThreadPool() + .submit(() -> OBSCommonUtils.commonContinueListObjects( + owner, oneLevelListRequest))); + levels.add(listEntity.getLevel()); + } + + // avoid adding list tasks in different levels later + break; + } else { + String oneLevelListPrefix = + listEntity.getType() == ListEntityType.COMMON_PREFIX + ? listEntity.getCommonPrefix() + : listEntity.getObjectSummary().getObjectKey(); + ListObjectsRequest oneLevelListRequest = OBSCommonUtils + .createListObjectsRequest(owner, oneLevelListPrefix, "/", + maxKeyNum); + oneLevelListRequests.add(oneLevelListRequest); + oneLevelListFutures.add(owner.getBoundedListThreadPool() + .submit(() -> OBSCommonUtils.commonListObjects(owner, + oneLevelListRequest))); + levels.add(listEntity.getLevel() + 1); + } + } + } + + static void addNewListStackEntities(final Stack listStack, + final List oneLevelListRequests, + final List levels, + final List oneLevelObjectListings) { + for (int i = oneLevelObjectListings.size() - 1; i >= 0; i--) { + ObjectListing oneLevelObjectListing = oneLevelObjectListings.get(i); + + if (oneLevelObjectListing.getCommonPrefixes().isEmpty() + && oneLevelObjectListing.getObjects() + .isEmpty()) { + continue; + } + + listStack.push(new ListEntity(oneLevelObjectListing.getPrefix(), + oneLevelObjectListing.isTruncated() + ? oneLevelObjectListing.getNextMarker() + : null, + levels.get(i))); + + ListIterator commonPrefixListIterator + = oneLevelObjectListing.getCommonPrefixes() + .listIterator(oneLevelObjectListing.getCommonPrefixes().size()); + while (commonPrefixListIterator.hasPrevious()) { + String commonPrefix = commonPrefixListIterator.previous(); + + if (commonPrefix.equals( + oneLevelListRequests.get(i).getPrefix())) { + // skip prefix itself + continue; + } + + listStack.push(new ListEntity(commonPrefix, levels.get(i))); + } + + ListIterator objectSummaryListIterator + = oneLevelObjectListing.getObjects() + .listIterator(oneLevelObjectListing.getObjects().size()); + while (objectSummaryListIterator.hasPrevious()) { + ObsObject objectSummary = objectSummaryListIterator.previous(); + + if (objectSummary.getObjectKey() + .equals(oneLevelListRequests.get(i).getPrefix())) { + // skip prefix itself + continue; + } + + if (objectSummary.getObjectKey().endsWith("/")) { + listStack.push( + new ListEntity(objectSummary, levels.get(i))); + } + } + } + } + + static int fetchListResultLocally(final String bucketName, + final Queue resultQueue, final int maxKeyNum, + final List objectSummaries, + final List levelStatsList) { + int resultNum = 0; + while (!resultQueue.isEmpty() && resultNum < maxKeyNum) { + ListEntity listEntity = resultQueue.poll(); + if (listEntity.getType() == ListEntityType.LIST_TAIL) { + throw new RuntimeException("cannot put list tail (" + listEntity + + ") into result queue"); + } else if (listEntity.getType() == ListEntityType.COMMON_PREFIX) { + addCommonPrefixIntoObjectList(bucketName, objectSummaries, + listEntity.getCommonPrefix()); + increaseLevelStats(levelStatsList, listEntity.getLevel(), true); + resultNum++; + } else { + objectSummaries.add(listEntity.getObjectSummary()); + increaseLevelStats(levelStatsList, listEntity.getLevel(), + listEntity.getObjectSummary().getObjectKey().endsWith("/")); + resultNum++; + } + } + return resultNum; + } + + static void addCommonPrefixIntoObjectList(final String bucketName, + final List objectSummaries, + final String commonPrefix) { + ObsObject objectSummary = new ObsObject(); + ObjectMetadata objectMetadata = new ObjectMetadata(); + objectMetadata.setContentLength(0L); + objectSummary.setBucketName(bucketName); + objectSummary.setObjectKey(commonPrefix); + objectSummary.setMetadata(objectMetadata); + objectSummaries.add(objectSummary); + } + + static OBSFsDFSListing fsDFSListObjects(final OBSFileSystem owner, + final ListObjectsRequest request) throws IOException { + List objectSummaries = new ArrayList<>(); + List commonPrefixes = new ArrayList<>(); + String bucketName = owner.getBucket(); + String prefix = request.getPrefix(); + int maxKeyNum = request.getMaxKeys(); + if (request.getDelimiter() != null) { + throw new IllegalArgumentException( + "illegal delimiter: " + request.getDelimiter()); + } + if (request.getMarker() != null) { + throw new IllegalArgumentException( + "illegal marker: " + request.getMarker()); + } + + Stack listStack = new Stack<>(); + Queue resultQueue = new LinkedList<>(); + List levelStatsList = new ArrayList<>(); + + listStack.push(new ListEntity(prefix, 0)); + increaseLevelStats(levelStatsList, 0, true); + + String nextMarker = fsDFSListNextBatch(owner, listStack, resultQueue, + null, maxKeyNum, objectSummaries, + levelStatsList); + + if (nextMarker == null) { + StringBuilder levelStatsStringBuilder = new StringBuilder(); + levelStatsStringBuilder.append("bucketName=").append(bucketName) + .append(", prefix=").append(prefix).append(": "); + for (LevelStats levelStats : levelStatsList) { + levelStatsStringBuilder.append("level=") + .append(levelStats.getLevel()) + .append(", dirNum=") + .append(levelStats.getDirNum()) + .append(", fileNum=") + .append(levelStats.getFileNum()) + .append("; "); + } + LOG.debug("[list level statistics info] " + + levelStatsStringBuilder.toString()); + } + + return new OBSFsDFSListing(request, + objectSummaries, + commonPrefixes, + nextMarker, + listStack, + resultQueue, + levelStatsList); + } + + static OBSFsDFSListing fsDFSContinueListObjects(final OBSFileSystem owner, + final OBSFsDFSListing obsFsDFSListing) + throws IOException { + List objectSummaries = new ArrayList<>(); + List commonPrefixes = new ArrayList<>(); + String bucketName = owner.getBucket(); + String prefix = obsFsDFSListing.getPrefix(); + String marker = obsFsDFSListing.getNextMarker(); + int maxKeyNum = obsFsDFSListing.getMaxKeys(); + if (obsFsDFSListing.getDelimiter() != null) { + throw new IllegalArgumentException( + "illegal delimiter: " + obsFsDFSListing.getDelimiter()); + } + + Stack listStack = obsFsDFSListing.getListStack(); + Queue resultQueue = obsFsDFSListing.getResultQueue(); + List levelStatsList = obsFsDFSListing.getLevelStatsList(); + + String nextMarker = fsDFSListNextBatch(owner, listStack, resultQueue, + marker, maxKeyNum, objectSummaries, + levelStatsList); + + if (nextMarker == null) { + StringBuilder levelStatsStringBuilder = new StringBuilder(); + levelStatsStringBuilder.append("bucketName=").append(bucketName) + .append(", prefix=").append(prefix).append(": "); + for (LevelStats levelStats : levelStatsList) { + levelStatsStringBuilder.append("level=") + .append(levelStats.getLevel()) + .append(", dirNum=") + .append(levelStats.getDirNum()) + .append(", fileNum=") + .append(levelStats.getFileNum()) + .append("; "); + } + LOG.debug("[list level statistics info] " + + levelStatsStringBuilder.toString()); + } + + return new OBSFsDFSListing(obsFsDFSListing, + objectSummaries, + commonPrefixes, + nextMarker, + listStack, + resultQueue, + levelStatsList); + } + + /** + * List entity type definition. + */ + enum ListEntityType { + /** + * Common prefix. + */ + COMMON_PREFIX, + /** + * Object summary. + */ + OBJECT_SUMMARY, + /** + * List tail. + */ + LIST_TAIL + } + + /** + * List entity for OBS depth first search listing. + */ + static class ListEntity { + /** + * List entity type. + */ + private ListEntityType type; + + /** + * Entity level. + */ + private final int level; + + /** + * For COMMON_PREFIX. + */ + private String commonPrefix = null; + + /** + * For OBJECT_SUMMARY. + */ + private ObsObject objectSummary = null; + + /** + * For LIST_TAIL. + */ + private String prefix = null; + + /** + * Next marker. + */ + private String nextMarker = null; + + ListEntity(final String comPrefix, final int entityLevel) { + this.type = ListEntityType.COMMON_PREFIX; + this.commonPrefix = comPrefix; + this.level = entityLevel; + } + + ListEntity(final ObsObject summary, final int entityLevel) { + this.type = ListEntityType.OBJECT_SUMMARY; + this.objectSummary = summary; + this.level = entityLevel; + } + + ListEntity(final String pf, final String nextMk, + final int entityLevel) { + this.type = ListEntityType.LIST_TAIL; + this.prefix = pf; + this.nextMarker = nextMk; + this.level = entityLevel; + } + + ListEntityType getType() { + return type; + } + + int getLevel() { + return level; + } + + String getCommonPrefix() { + return commonPrefix; + } + + ObsObject getObjectSummary() { + return objectSummary; + } + + public String getPrefix() { + return prefix; + } + + String getNextMarker() { + return nextMarker; + } + + @Override + public String toString() { + return "type: " + type + + ", commonPrefix: " + (commonPrefix != null + ? commonPrefix + : "") + + ", objectSummary: " + (objectSummary != null + ? objectSummary + : "") + + ", prefix: " + (prefix != null ? prefix : "") + + ", nextMarker: " + (nextMarker != null ? nextMarker : ""); + } + } + + /** + * Level statistics for OBS depth first search listing. + */ + static class LevelStats { + /** + * Entity level. + */ + private int level; + + /** + * Directory num. + */ + private long dirNum; + + /** + * File num. + */ + private long fileNum; + + LevelStats(final int entityLevel) { + this.level = entityLevel; + this.dirNum = 0; + this.fileNum = 0; + } + + void increaseDirNum() { + dirNum++; + } + + void increaseFileNum() { + fileNum++; + } + + int getLevel() { + return level; + } + + long getDirNum() { + return dirNum; + } + + long getFileNum() { + return fileNum; + } + } + + /** + * Stack of entity list.. + */ + private Stack listStack; + + /** + * Queue of entity list. + */ + private Queue resultQueue; + + /** + * List of levelStats. + */ + private List levelStatsList; + + OBSFsDFSListing(final ListObjectsRequest request, + final List objectSummaries, + final List commonPrefixes, + final String nextMarker, + final Stack listEntityStack, + final Queue listEntityQueue, + final List listLevelStats) { + super(objectSummaries, + commonPrefixes, + request.getBucketName(), + nextMarker != null, + request.getPrefix(), + null, + request.getMaxKeys(), + null, + nextMarker, + null); + this.listStack = listEntityStack; + this.resultQueue = listEntityQueue; + this.levelStatsList = listLevelStats; + } + + OBSFsDFSListing(final OBSFsDFSListing obsFsDFSListing, + final List objectSummaries, + final List commonPrefixes, + final String nextMarker, + final Stack listEntityStack, + final Queue listEntityQueue, + final List listLevelStats) { + super(objectSummaries, + commonPrefixes, + obsFsDFSListing.getBucketName(), + nextMarker != null, + obsFsDFSListing.getPrefix(), + obsFsDFSListing.getNextMarker(), + obsFsDFSListing.getMaxKeys(), + null, + nextMarker, + null); + this.listStack = listEntityStack; + this.resultQueue = listEntityQueue; + this.levelStatsList = listLevelStats; + } + + Stack getListStack() { + return listStack; + } + + Queue getResultQueue() { + return resultQueue; + } + + List getLevelStatsList() { + return levelStatsList; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java new file mode 100644 index 00000000000..29a92c71919 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.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.hadoop.fs.obs; + +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import com.obs.services.exception.ObsException; + +import java.io.IOException; + +/** + * IOException equivalent to {@link ObsException}. + */ +class OBSIOException extends IOException { + private static final long serialVersionUID = -1582681108285856259L; + + /** + * Peration message. + */ + private final String operation; + + OBSIOException(final String operationMsg, final ObsException cause) { + super(cause); + Preconditions.checkArgument(operationMsg != null, + "Null 'operation' argument"); + Preconditions.checkArgument(cause != null, "Null 'cause' argument"); + this.operation = operationMsg; + } + + public ObsException getCause() { + return (ObsException) super.getCause(); + } + + @Override + public String getMessage() { + return operation + ": " + getCause().getErrorMessage() + + ", detailMessage: " + super.getMessage(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java new file mode 100644 index 00000000000..e94565a4d76 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInputStream.java @@ -0,0 +1,1047 @@ +/* + * 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.obs; + +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import com.obs.services.model.GetObjectRequest; +import com.sun.istack.NotNull; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.ByteBufferReadable; +import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +import static org.apache.hadoop.fs.obs.OBSCommonUtils.translateException; + +/** + * Input stream for an OBS object. + * + *

As this stream seeks withing an object, it may close then re-open the + * stream. When this happens, any updated stream data may be retrieved, and, + * given the consistency model of Huawei OBS, outdated data may in fact be + * picked up. + * + *

As a result, the outcome of reading from a stream of an object which is + * actively manipulated during the read process is "undefined". + * + *

The class is marked as private as code should not be creating instances + * themselves. Any extra feature (e.g instrumentation) should be considered + * unstable. + * + *

Because it prints some of the state of the instrumentation, the output of + * {@link #toString()} must also be considered unstable. + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +class OBSInputStream extends FSInputStream + implements CanSetReadahead, ByteBufferReadable { + /** + * Class logger. + */ + public static final Logger LOG = LoggerFactory.getLogger( + OBSInputStream.class); + + /** + * Read retry times. + */ + private static final int READ_RETRY_TIME = 3; + + /** + * Seek retry times. + */ + private static final int SEEK_RETRY_TIME = 9; + + /** + * Delay times. + */ + private static final long DELAY_TIME = 10; + + /** + * The statistics for OBS file system. + */ + private final FileSystem.Statistics statistics; + + /** + * Obs client. + */ + private final ObsClient client; + + /** + * Bucket name. + */ + private final String bucket; + + /** + * Bucket key. + */ + private final String key; + + /** + * Content length. + */ + private final long contentLength; + + /** + * Object uri. + */ + private final String uri; + + /** + * Obs file system instance. + */ + private OBSFileSystem fs; + + /** + * This is the public position; the one set in {@link #seek(long)} and + * returned in {@link #getPos()}. + */ + private long streamCurrentPos; + + /** + * Closed bit. Volatile so reads are non-blocking. Updates must be in a + * synchronized block to guarantee an atomic check and set + */ + private volatile boolean closed; + + /** + * Input stream. + */ + private InputStream wrappedStream = null; + + /** + * Read ahead range. + */ + private long readAheadRange = OBSConstants.DEFAULT_READAHEAD_RANGE; + + /** + * This is the actual position within the object, used by lazy seek to decide + * whether to seek on the next read or not. + */ + private long nextReadPos; + + /** + * The end of the content range of the last request. This is an absolute value + * of the range, not a length field. + */ + private long contentRangeFinish; + + /** + * The start of the content range of the last request. + */ + private long contentRangeStart; + + OBSInputStream( + final String bucketName, + final String bucketKey, + final long fileStatusLength, + final ObsClient obsClient, + final FileSystem.Statistics stats, + final long readaheadRange, + final OBSFileSystem obsFileSystem) { + Preconditions.checkArgument(StringUtils.isNotEmpty(bucketName), + "No Bucket"); + Preconditions.checkArgument(StringUtils.isNotEmpty(bucketKey), + "No Key"); + Preconditions.checkArgument(fileStatusLength >= 0, + "Negative content length"); + this.bucket = bucketName; + this.key = bucketKey; + this.contentLength = fileStatusLength; + this.client = obsClient; + this.statistics = stats; + this.uri = "obs://" + this.bucket + "/" + this.key; + this.fs = obsFileSystem; + setReadahead(readaheadRange); + } + + /** + * Calculate the limit for a get request, based on input policy and state of + * object. + * + * @param targetPos position of the read + * @param length length of bytes requested; if less than zero + * "unknown" + * @param contentLength total length of file + * @param readahead current readahead value + * @return the absolute value of the limit of the request. + */ + static long calculateRequestLimit( + final long targetPos, final long length, final long contentLength, + final long readahead) { + // cannot read past the end of the object + return Math.min(contentLength, length < 0 ? contentLength + : targetPos + Math.max(readahead, length)); + } + + /** + * Opens up the stream at specified target position and for given length. + * + * @param reason reason for reopen + * @param targetPos target position + * @param length length requested + * @throws IOException on any failure to open the object + */ + private synchronized void reopen(final String reason, final long targetPos, + final long length) + throws IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + if (wrappedStream != null) { + closeStream("reopen(" + reason + ")", contentRangeFinish); + } + + contentRangeFinish = + calculateRequestLimit(targetPos, length, contentLength, + readAheadRange); + + try { + GetObjectRequest request = new GetObjectRequest(bucket, key); + request.setRangeStart(targetPos); + request.setRangeEnd(contentRangeFinish); + if (fs.getSse().isSseCEnable()) { + request.setSseCHeader(fs.getSse().getSseCHeader()); + } + wrappedStream = client.getObject(request).getObjectContent(); + contentRangeStart = targetPos; + if (wrappedStream == null) { + throw new IOException( + "Null IO stream from reopen of (" + reason + ") " + uri); + } + } catch (ObsException e) { + throw translateException("Reopen at position " + targetPos, uri, e); + } + + this.streamCurrentPos = targetPos; + long endTime = System.currentTimeMillis(); + LOG.debug( + "reopen({}) for {} range[{}-{}], length={}," + + " streamPosition={}, nextReadPosition={}, thread={}, " + + "timeUsedInMilliSec={}", + uri, + reason, + targetPos, + contentRangeFinish, + length, + streamCurrentPos, + nextReadPos, + threadId, + endTime - startTime + ); + } + + @Override + public synchronized long getPos() { + return nextReadPos < 0 ? 0 : nextReadPos; + } + + @Override + public synchronized void seek(final long targetPos) throws IOException { + checkNotClosed(); + + // Do not allow negative seek + if (targetPos < 0) { + throw new EOFException( + FSExceptionMessages.NEGATIVE_SEEK + " " + targetPos); + } + + if (this.contentLength <= 0) { + return; + } + + // Lazy seek + nextReadPos = targetPos; + } + + /** + * Seek without raising any exception. This is for use in {@code finally} + * clauses + * + * @param positiveTargetPos a target position which must be positive. + */ + private void seekQuietly(final long positiveTargetPos) { + try { + seek(positiveTargetPos); + } catch (IOException ioe) { + LOG.debug("Ignoring IOE on seek of {} to {}", uri, + positiveTargetPos, ioe); + } + } + + /** + * Adjust the stream to a specific position. + * + * @param targetPos target seek position + * @throws IOException on any failure to seek + */ + private void seekInStream(final long targetPos) throws IOException { + checkNotClosed(); + if (wrappedStream == null) { + return; + } + // compute how much more to skip + long diff = targetPos - streamCurrentPos; + if (diff > 0) { + // forward seek -this is where data can be skipped + + int available = wrappedStream.available(); + // always seek at least as far as what is available + long forwardSeekRange = Math.max(readAheadRange, available); + // work out how much is actually left in the stream + // then choose whichever comes first: the range or the EOF + long remainingInCurrentRequest = remainingInCurrentRequest(); + + long forwardSeekLimit = Math.min(remainingInCurrentRequest, + forwardSeekRange); + boolean skipForward = remainingInCurrentRequest > 0 + && diff <= forwardSeekLimit; + if (skipForward) { + // the forward seek range is within the limits + LOG.debug("Forward seek on {}, of {} bytes", uri, diff); + long skippedOnce = wrappedStream.skip(diff); + while (diff > 0 && skippedOnce > 0) { + streamCurrentPos += skippedOnce; + diff -= skippedOnce; + incrementBytesRead(skippedOnce); + skippedOnce = wrappedStream.skip(diff); + } + + if (streamCurrentPos == targetPos) { + // all is well + return; + } else { + // log a warning; continue to attempt to re-open + LOG.info("Failed to seek on {} to {}. Current position {}", + uri, targetPos, streamCurrentPos); + } + } + } else if (diff == 0 && remainingInCurrentRequest() > 0) { + // targetPos == streamCurrentPos + // if there is data left in the stream, keep going + return; + } + + // if the code reaches here, the stream needs to be reopened. + // close the stream; if read the object will be opened at the + // new streamCurrentPos + closeStream("seekInStream()", this.contentRangeFinish); + streamCurrentPos = targetPos; + } + + @Override + public boolean seekToNewSource(final long targetPos) { + return false; + } + + /** + * Perform lazy seek and adjust stream to correct position for reading. + * + * @param targetPos position from where data should be read + * @param len length of the content that needs to be read + * @throws IOException on any failure to lazy seek + */ + private void lazySeek(final long targetPos, final long len) + throws IOException { + for (int i = 0; i < SEEK_RETRY_TIME; i++) { + try { + // For lazy seek + seekInStream(targetPos); + + // re-open at specific location if needed + if (wrappedStream == null) { + reopen("read from new offset", targetPos, len); + } + + break; + } catch (IOException e) { + if (wrappedStream != null) { + closeStream("lazySeek() seekInStream has exception ", + this.contentRangeFinish); + } + Throwable cause = e.getCause(); + if (cause instanceof ObsException) { + ObsException obsException = (ObsException) cause; + int status = obsException.getResponseCode(); + switch (status) { + case OBSCommonUtils.UNAUTHORIZED_CODE: + case OBSCommonUtils.FORBIDDEN_CODE: + case OBSCommonUtils.NOT_FOUND_CODE: + case OBSCommonUtils.GONE_CODE: + case OBSCommonUtils.EOF_CODE: + throw e; + default: + break; + } + } + + LOG.warn("IOException occurred in lazySeek, retry: {}", i, e); + if (i == SEEK_RETRY_TIME - 1) { + throw e; + } + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + } + } + + /** + * Increment the bytes read counter if there is a stats instance and the + * number of bytes read is more than zero. + * + * @param bytesRead number of bytes read + */ + private void incrementBytesRead(final long bytesRead) { + if (statistics != null && bytesRead > 0) { + statistics.incrementBytesRead(bytesRead); + } + } + + private void sleepInLock() throws InterruptedException { + long start = System.currentTimeMillis(); + long now = start; + while (now - start < OBSInputStream.DELAY_TIME) { + wait(start + OBSInputStream.DELAY_TIME - now); + now = System.currentTimeMillis(); + } + } + + @Override + public synchronized int read() throws IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + checkNotClosed(); + if (this.contentLength == 0 || nextReadPos >= contentLength) { + return -1; + } + + int byteRead = -1; + try { + lazySeek(nextReadPos, 1); + } catch (EOFException e) { + onReadFailure(e, 1); + return -1; + } + + IOException exception = null; + for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) { + try { + byteRead = wrappedStream.read(); + exception = null; + break; + } catch (EOFException e) { + onReadFailure(e, 1); + return -1; + } catch (IOException e) { + exception = e; + onReadFailure(e, 1); + LOG.warn( + "read of [{}] failed, retry time[{}], due to exception[{}]", + uri, retryTime, exception); + if (retryTime < READ_RETRY_TIME) { + try { + sleepInLock(); + } catch (InterruptedException ie) { + LOG.error( + "read of [{}] failed, retry time[{}], due to " + + "exception[{}]", + uri, retryTime, + exception); + throw exception; + } + } + } + } + + if (exception != null) { + LOG.error( + "read of [{}] failed, retry time[{}], due to exception[{}]", + uri, READ_RETRY_TIME, exception); + throw exception; + } + + if (byteRead >= 0) { + streamCurrentPos++; + nextReadPos++; + } + + if (byteRead >= 0) { + incrementBytesRead(1); + } + + long endTime = System.currentTimeMillis(); + LOG.debug( + "read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, " + + "thread:{}, timeUsedMilliSec:{}", + uri, contentLength, byteRead >= 0 ? nextReadPos - 1 : nextReadPos, + byteRead, threadId, + endTime - startTime); + return byteRead; + } + + /** + * Handle an IOE on a read by attempting to re-open the stream. The + * filesystem's readException count will be incremented. + * + * @param ioe exception caught. + * @param length length of data being attempted to read + * @throws IOException any exception thrown on the re-open attempt. + */ + private void onReadFailure(final IOException ioe, final int length) + throws IOException { + LOG.debug( + "Got exception while trying to read from stream {}" + + " trying to recover: " + ioe, uri); + int i = 1; + while (true) { + try { + reopen("failure recovery", streamCurrentPos, length); + return; + } catch (OBSIOException e) { + LOG.warn( + "OBSIOException occurred in reopen for failure recovery, " + + "the {} retry time", + i, e); + if (i == READ_RETRY_TIME) { + throw e; + } + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + i++; + } + } + + @Override + public synchronized int read(final ByteBuffer byteBuffer) + throws IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + LOG.debug("read byteBuffer: {}", byteBuffer.toString()); + checkNotClosed(); + + int len = byteBuffer.remaining(); + if (len == 0) { + return 0; + } + + byte[] buf = new byte[len]; + + if (this.contentLength == 0 || nextReadPos >= contentLength) { + return -1; + } + + try { + lazySeek(nextReadPos, len); + } catch (EOFException e) { + onReadFailure(e, len); + // the end of the file has moved + return -1; + } + + int bytesRead = 0; + IOException exception = null; + for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) { + try { + bytesRead = tryToReadFromInputStream(wrappedStream, buf, 0, + len); + if (bytesRead == -1) { + return -1; + } + exception = null; + break; + } catch (EOFException e) { + onReadFailure(e, len); + return -1; + } catch (IOException e) { + exception = e; + onReadFailure(e, len); + LOG.warn( + "read len[{}] of [{}] failed, retry time[{}], " + + "due to exception[{}]", + len, uri, retryTime, exception); + if (retryTime < READ_RETRY_TIME) { + try { + sleepInLock(); + } catch (InterruptedException ie) { + LOG.error( + "read len[{}] of [{}] failed, retry time[{}], " + + "due to exception[{}]", + len, uri, retryTime, exception); + throw exception; + } + } + } + } + + if (exception != null) { + LOG.error( + "read len[{}] of [{}] failed, retry time[{}], " + + "due to exception[{}]", + len, uri, READ_RETRY_TIME, exception); + throw exception; + } + + if (bytesRead > 0) { + streamCurrentPos += bytesRead; + nextReadPos += bytesRead; + byteBuffer.put(buf, 0, bytesRead); + } + incrementBytesRead(bytesRead); + + long endTime = System.currentTimeMillis(); + LOG.debug( + "Read-ByteBuffer uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", + uri, contentLength, len, bytesRead, + bytesRead >= 0 ? nextReadPos - bytesRead : nextReadPos, threadId, + endTime - startTime); + return bytesRead; + } + + private int tryToReadFromInputStream(final InputStream in, final byte[] buf, + final int off, final int len) throws IOException { + int bytesRead = 0; + while (bytesRead < len) { + int bytes = in.read(buf, off + bytesRead, len - bytesRead); + if (bytes == -1) { + if (bytesRead == 0) { + return -1; + } else { + break; + } + } + bytesRead += bytes; + } + + return bytesRead; + } + + /** + * {@inheritDoc} + * + *

This updates the statistics on read operations started and whether or + * not the read operation "completed", that is: returned the exact number of + * bytes requested. + * + * @throws IOException if there are other problems + */ + @Override + public synchronized int read(@NotNull final byte[] buf, final int off, + final int len) throws IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + checkNotClosed(); + validatePositionedReadArgs(nextReadPos, buf, off, len); + if (len == 0) { + return 0; + } + + if (this.contentLength == 0 || nextReadPos >= contentLength) { + return -1; + } + + try { + lazySeek(nextReadPos, len); + } catch (EOFException e) { + onReadFailure(e, len); + // the end of the file has moved + return -1; + } + + int bytesRead = 0; + IOException exception = null; + for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) { + try { + bytesRead = tryToReadFromInputStream(wrappedStream, buf, off, + len); + if (bytesRead == -1) { + return -1; + } + exception = null; + break; + } catch (EOFException e) { + onReadFailure(e, len); + return -1; + } catch (IOException e) { + exception = e; + onReadFailure(e, len); + LOG.warn( + "read offset[{}] len[{}] of [{}] failed, retry time[{}], " + + "due to exception[{}]", + off, len, uri, retryTime, exception); + if (retryTime < READ_RETRY_TIME) { + try { + sleepInLock(); + } catch (InterruptedException ie) { + LOG.error( + "read offset[{}] len[{}] of [{}] failed, " + + "retry time[{}], due to exception[{}]", + off, len, uri, retryTime, exception); + throw exception; + } + } + } + } + + if (exception != null) { + LOG.error( + "read offset[{}] len[{}] of [{}] failed, retry time[{}], " + + "due to exception[{}]", + off, len, uri, READ_RETRY_TIME, exception); + throw exception; + } + + if (bytesRead > 0) { + streamCurrentPos += bytesRead; + nextReadPos += bytesRead; + } + incrementBytesRead(bytesRead); + + long endTime = System.currentTimeMillis(); + LOG.debug( + "Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", + uri, contentLength, len, bytesRead, + bytesRead >= 0 ? nextReadPos - bytesRead : nextReadPos, threadId, + endTime - startTime); + return bytesRead; + } + + /** + * Verify that the input stream is open. Non blocking; this gives the last + * state of the volatile {@link #closed} field. + * + * @throws IOException if the connection is closed. + */ + private void checkNotClosed() throws IOException { + if (closed) { + throw new IOException( + uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + /** + * Close the stream. This triggers publishing of the stream statistics back to + * the filesystem statistics. This operation is synchronized, so that only one + * thread can attempt to close the connection; all later/blocked calls are + * no-ops. + * + * @throws IOException on any problem + */ + @Override + public synchronized void close() throws IOException { + if (!closed) { + closed = true; + // close or abort the stream + closeStream("close() operation", this.contentRangeFinish); + // this is actually a no-op + super.close(); + } + } + + /** + * Close a stream: decide whether to abort or close, based on the length of + * the stream and the current position. If a close() is attempted and fails, + * the operation escalates to an abort. + * + *

This does not set the {@link #closed} flag. + * + * @param reason reason for stream being closed; used in messages + * @param length length of the stream + * @throws IOException on any failure to close stream + */ + private synchronized void closeStream(final String reason, + final long length) + throws IOException { + if (wrappedStream != null) { + try { + wrappedStream.close(); + } catch (IOException e) { + // exception escalates to an abort + LOG.debug("When closing {} stream for {}", uri, reason, e); + throw e; + } + + LOG.debug( + "Stream {} : {}; streamPos={}, nextReadPos={}," + + " request range {}-{} length={}", + uri, + reason, + streamCurrentPos, + nextReadPos, + contentRangeStart, + contentRangeFinish, + length); + wrappedStream = null; + } + } + + @Override + public synchronized int available() throws IOException { + checkNotClosed(); + + long remaining = remainingInFile(); + if (remaining > Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } + return (int) remaining; + } + + /** + * Bytes left in stream. + * + * @return how many bytes are left to read + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + public synchronized long remainingInFile() { + return this.contentLength - this.streamCurrentPos; + } + + /** + * Bytes left in the current request. Only valid if there is an active + * request. + * + * @return how many bytes are left to read in the current GET. + */ + @InterfaceAudience.Private + @InterfaceStability.Unstable + public synchronized long remainingInCurrentRequest() { + return this.contentRangeFinish - this.streamCurrentPos; + } + + @Override + public boolean markSupported() { + return false; + } + + /** + * String value includes statistics as well as stream state. Important: + * there are no guarantees as to the stability of this value. + * + * @return a string value for printing in logs/diagnostics + */ + @Override + @InterfaceStability.Unstable + public String toString() { + synchronized (this) { + return "OBSInputStream{" + uri + + " wrappedStream=" + (wrappedStream != null + ? "open" + : "closed") + + " streamCurrentPos=" + streamCurrentPos + + " nextReadPos=" + nextReadPos + + " contentLength=" + contentLength + + " contentRangeStart=" + contentRangeStart + + " contentRangeFinish=" + contentRangeFinish + + " remainingInCurrentRequest=" + remainingInCurrentRequest() + + '}'; + } + } + + /** + * Subclass {@code readFully()} operation which only seeks at the start of the + * series of operations; seeking back at the end. + * + *

This is significantly higher performance if multiple read attempts + * are needed to fetch the data, as it does not break the HTTP connection. + * + *

To maintain thread safety requirements, this operation is + * synchronized for the duration of the sequence. {@inheritDoc} + */ + @Override + public void readFully(final long position, final byte[] buffer, + final int offset, + final int length) + throws IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + checkNotClosed(); + validatePositionedReadArgs(position, buffer, offset, length); + if (length == 0) { + return; + } + int nread = 0; + synchronized (this) { + long oldPos = getPos(); + try { + seek(position); + while (nread < length) { + int nbytes = read(buffer, offset + nread, length - nread); + if (nbytes < 0) { + throw new EOFException( + FSExceptionMessages.EOF_IN_READ_FULLY); + } + nread += nbytes; + } + } finally { + seekQuietly(oldPos); + } + } + + long endTime = System.currentTimeMillis(); + LOG.debug( + "ReadFully uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", + uri, contentLength, length, nread, position, threadId, + endTime - startTime); + } + + /** + * Read bytes starting from the specified position. + * + * @param position start read from this position + * @param buffer read buffer + * @param offset offset into buffer + * @param length number of bytes to read + * @return actual number of bytes read + * @throws IOException on any failure to read + */ + @Override + public int read(final long position, final byte[] buffer, final int offset, + final int length) + throws IOException { + int len = length; + checkNotClosed(); + validatePositionedReadArgs(position, buffer, offset, len); + if (position < 0 || position >= contentLength) { + return -1; + } + if ((position + len) > contentLength) { + len = (int) (contentLength - position); + } + + if (fs.isReadTransformEnabled()) { + return super.read(position, buffer, offset, len); + } + + return randomReadWithNewInputStream(position, buffer, offset, len); + } + + private int randomReadWithNewInputStream(final long position, + final byte[] buffer, final int offset, final int length) + throws IOException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + int bytesRead = 0; + InputStream inputStream = null; + IOException exception = null; + GetObjectRequest request = new GetObjectRequest(bucket, key); + request.setRangeStart(position); + request.setRangeEnd(position + length); + if (fs.getSse().isSseCEnable()) { + request.setSseCHeader(fs.getSse().getSseCHeader()); + } + + for (int retryTime = 1; retryTime <= READ_RETRY_TIME; retryTime++) { + try { + inputStream = client.getObject(request).getObjectContent(); + if (inputStream == null) { + break; + } + bytesRead = tryToReadFromInputStream(inputStream, buffer, + offset, length); + if (bytesRead == -1) { + return -1; + } + + exception = null; + break; + } catch (ObsException | IOException e) { + if (e instanceof ObsException) { + exception = translateException( + "Read at position " + position, uri, (ObsException) e); + } else { + exception = (IOException) e; + } + LOG.warn( + "read position[{}] destLen[{}] offset[{}] readLen[{}] " + + "of [{}] failed, retry time[{}], due to " + + "exception[{}] e[{}]", + position, length, offset, bytesRead, uri, retryTime, + exception, e); + if (retryTime < READ_RETRY_TIME) { + try { + Thread.sleep(DELAY_TIME); + } catch (InterruptedException ie) { + LOG.error( + "read position[{}] destLen[{}] offset[{}] " + + "readLen[{}] of [{}] failed, retry time[{}], " + + "due to exception[{}] e[{}]", + position, length, offset, bytesRead, uri, retryTime, + exception, e); + throw exception; + } + } + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + if (inputStream == null || exception != null) { + LOG.error( + "read position[{}] destLen[{}] offset[{}] len[{}] failed, " + + "retry time[{}], due to exception[{}]", + position, length, offset, bytesRead, READ_RETRY_TIME, + exception); + throw new IOException("read failed of " + uri + ", inputStream is " + + (inputStream == null ? "null" : "not null"), exception); + + } + + long endTime = System.currentTimeMillis(); + LOG.debug( + "Read-4args uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", + uri, contentLength, length, bytesRead, position, threadId, + endTime - startTime); + return bytesRead; + } + + @Override + public synchronized void setReadahead(final Long newReadaheadRange) { + if (newReadaheadRange == null) { + this.readAheadRange = OBSConstants.DEFAULT_READAHEAD_RANGE; + } else { + Preconditions.checkArgument(newReadaheadRange >= 0, + "Negative readahead value"); + this.readAheadRange = newReadaheadRange; + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java new file mode 100644 index 00000000000..4072feb2cac --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java @@ -0,0 +1,656 @@ +/* + * 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.obs; + +import com.obs.services.exception.ObsException; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObsObject; + +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.RemoteIterator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; +import java.util.ListIterator; +import java.util.NoSuchElementException; + +/** + * OBS listing implementation. + */ +class OBSListing { + /** + * A Path filter which accepts all filenames. + */ + static final PathFilter ACCEPT_ALL = + new PathFilter() { + @Override + public boolean accept(final Path file) { + return true; + } + + @Override + public String toString() { + return "ACCEPT_ALL"; + } + }; + + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger(OBSListing.class); + + /** + * OBS File System instance. + */ + private final OBSFileSystem owner; + + OBSListing(final OBSFileSystem ownerFS) { + this.owner = ownerFS; + } + + /** + * Create a FileStatus iterator against a path, with a given list object + * request. + * + * @param listPath path of the listing + * @param request initial request to make + * @param filter the filter on which paths to accept + * @param acceptor the class/predicate to decide which entries to accept in + * the listing based on the full file status. + * @return the iterator + * @throws IOException IO Problems + */ + FileStatusListingIterator createFileStatusListingIterator( + final Path listPath, + final ListObjectsRequest request, + final PathFilter filter, + final FileStatusAcceptor acceptor) + throws IOException { + return new FileStatusListingIterator( + new ObjectListingIterator(listPath, request), filter, acceptor); + } + + /** + * Create a located status iterator over a file status iterator. + * + * @param statusIterator an iterator over the remote status entries + * @return a new remote iterator + */ + LocatedFileStatusIterator createLocatedFileStatusIterator( + final RemoteIterator statusIterator) { + return new LocatedFileStatusIterator(statusIterator); + } + + /** + * Interface to implement by the logic deciding whether to accept a summary + * entry or path as a valid file or directory. + */ + interface FileStatusAcceptor { + + /** + * Predicate to decide whether or not to accept a summary entry. + * + * @param keyPath qualified path to the entry + * @param summary summary entry + * @return true if the entry is accepted (i.e. that a status entry should be + * generated. + */ + boolean accept(Path keyPath, ObsObject summary); + + /** + * Predicate to decide whether or not to accept a prefix. + * + * @param keyPath qualified path to the entry + * @param commonPrefix the prefix + * @return true if the entry is accepted (i.e. that a status entry should be + * generated.) + */ + boolean accept(Path keyPath, String commonPrefix); + } + + /** + * A remote iterator which only iterates over a single `LocatedFileStatus` + * value. + * + *

If the status value is null, the iterator declares that it has no + * data. This iterator is used to handle + * {@link OBSFileSystem#listStatus(Path)}calls where the path handed in + * refers to a file, not a directory: this is + * the iterator returned. + */ + static final class SingleStatusRemoteIterator + implements RemoteIterator { + + /** + * The status to return; set to null after the first iteration. + */ + private LocatedFileStatus status; + + /** + * Constructor. + * + * @param locatedFileStatus status value: may be null, in which case the + * iterator is empty. + */ + SingleStatusRemoteIterator(final LocatedFileStatus locatedFileStatus) { + this.status = locatedFileStatus; + } + + /** + * {@inheritDoc} + * + * @return true if there is a file status to return: this is always false + * for the second iteration, and may be false for the first. + */ + @Override + public boolean hasNext() { + return status != null; + } + + /** + * {@inheritDoc} + * + * @return the non-null status element passed in when the instance was + * constructed, if it ha not already been retrieved. + * @throws NoSuchElementException if this is the second call, or it is the + * first call and a null + * {@link LocatedFileStatus} + * entry was passed to the constructor. + */ + @Override + public LocatedFileStatus next() { + if (hasNext()) { + LocatedFileStatus s = this.status; + status = null; + return s; + } else { + throw new NoSuchElementException(); + } + } + } + + /** + * Accept all entries except the base path and those which map to OBS pseudo + * directory markers. + */ + static class AcceptFilesOnly implements FileStatusAcceptor { + /** + * path to qualify. + */ + private final Path qualifiedPath; + + AcceptFilesOnly(final Path path) { + this.qualifiedPath = path; + } + + /** + * Reject a summary entry if the key path is the qualified Path, or it ends + * with {@code "_$folder$"}. + * + * @param keyPath key path of the entry + * @param summary summary entry + * @return true if the entry is accepted (i.e. that a status entry should be + * generated. + */ + @Override + public boolean accept(final Path keyPath, final ObsObject summary) { + return !keyPath.equals(qualifiedPath) + && !summary.getObjectKey() + .endsWith(OBSConstants.OBS_FOLDER_SUFFIX) + && !OBSCommonUtils.objectRepresentsDirectory( + summary.getObjectKey(), + summary.getMetadata().getContentLength()); + } + + /** + * Accept no directory paths. + * + * @param keyPath qualified path to the entry + * @param prefix common prefix in listing. + * @return false, always. + */ + @Override + public boolean accept(final Path keyPath, final String prefix) { + return false; + } + } + + /** + * Accept all entries except the base path and those which map to OBS pseudo + * directory markers. + */ + static class AcceptAllButSelfAndS3nDirs implements FileStatusAcceptor { + + /** + * Base path. + */ + private final Path qualifiedPath; + + /** + * Constructor. + * + * @param path an already-qualified path. + */ + AcceptAllButSelfAndS3nDirs(final Path path) { + this.qualifiedPath = path; + } + + /** + * Reject a summary entry if the key path is the qualified Path, or it ends + * with {@code "_$folder$"}. + * + * @param keyPath key path of the entry + * @param summary summary entry + * @return true if the entry is accepted (i.e. that a status entry should be + * generated.) + */ + @Override + public boolean accept(final Path keyPath, final ObsObject summary) { + return !keyPath.equals(qualifiedPath) && !summary.getObjectKey() + .endsWith(OBSConstants.OBS_FOLDER_SUFFIX); + } + + /** + * Accept all prefixes except the one for the base path, "self". + * + * @param keyPath qualified path to the entry + * @param prefix common prefix in listing. + * @return true if the entry is accepted (i.e. that a status entry should be + * generated. + */ + @Override + public boolean accept(final Path keyPath, final String prefix) { + return !keyPath.equals(qualifiedPath); + } + } + + /** + * Wraps up object listing into a remote iterator which will ask for more + * listing data if needed. + * + *

This is a complex operation, especially the process to determine if + * there are more entries remaining. If there are no more results remaining in + * the (filtered) results of the current listing request, then another request + * is made + * and those results filtered before the iterator can declare that + * there is more data available. + * + *

The need to filter the results precludes the iterator from simply + * declaring that if the {@link ObjectListingIterator#hasNext()} is true then + * there are more results. Instead the next batch of results must be retrieved + * and filtered. + * + *

What does this mean? It means that remote requests to retrieve new + * batches of object listings are made in the {@link #hasNext()} call; the + * {@link #next()} call simply returns the filtered results of the last + * listing processed. However, do note that {@link #next()} calls {@link + * #hasNext()} during its operation. This is critical to ensure that a listing + * obtained through a sequence of {@link #next()} will complete with the same + * set of results as a classic {@code while(it.hasNext()} loop. + * + *

Thread safety: None. + */ + class FileStatusListingIterator implements RemoteIterator { + + /** + * Source of objects. + */ + private final ObjectListingIterator source; + + /** + * Filter of paths from API call. + */ + private final PathFilter filter; + + /** + * Filter of entries from file status. + */ + private final FileStatusAcceptor acceptor; + + /** + * Request batch size. + */ + private int batchSize; + + /** + * Iterator over the current set of results. + */ + private ListIterator statusBatchIterator; + + /** + * Create an iterator over file status entries. + * + * @param listPath the listing iterator from a listObjects call. + * @param pathFilter the filter on which paths to accept + * @param fileStatusAcceptor the class/predicate to decide which entries to + * accept in the listing based on the full file + * status. + * @throws IOException IO Problems + */ + FileStatusListingIterator( + final ObjectListingIterator listPath, final PathFilter pathFilter, + final FileStatusAcceptor fileStatusAcceptor) + throws IOException { + this.source = listPath; + this.filter = pathFilter; + this.acceptor = fileStatusAcceptor; + // build the first set of results. This will not trigger any + // remote IO, assuming the source iterator is in its initial + // iteration + requestNextBatch(); + } + + /** + * Report whether or not there is new data available. If there is data in + * the local filtered list, return true. Else: request more data util that + * condition is met, or there is no more remote listing data. + * + * @return true if a call to {@link #next()} will succeed. + * @throws IOException on any failure to request next batch + */ + @Override + public boolean hasNext() throws IOException { + return statusBatchIterator.hasNext() || requestNextBatch(); + } + + @Override + public FileStatus next() throws IOException { + if (!hasNext()) { + throw new NoSuchElementException(); + } + return statusBatchIterator.next(); + } + + /** + * Try to retrieve another batch. Note that for the initial batch, {@link + * ObjectListingIterator} does not generate a request; it simply returns the + * initial set. + * + * @return true if a new batch was created. + * @throws IOException IO problems + */ + private boolean requestNextBatch() throws IOException { + // look for more object listing batches being available + while (source.hasNext()) { + // if available, retrieve it and build the next status + if (buildNextStatusBatch(source.next())) { + // this batch successfully generated entries matching + // the filters/acceptors; + // declare that the request was successful + return true; + } else { + LOG.debug( + "All entries in batch were filtered...continuing"); + } + } + // if this code is reached, it means that all remaining + // object lists have been retrieved, and there are no new entries + // to return. + return false; + } + + /** + * Build the next status batch from a listing. + * + * @param objects the next object listing + * @return true if this added any entries after filtering + */ + private boolean buildNextStatusBatch(final ObjectListing objects) { + // counters for debug logs + int added = 0; + int ignored = 0; + // list to fill in with results. Initial size will be list maximum. + List stats = + new ArrayList<>( + objects.getObjects().size() + objects.getCommonPrefixes() + .size()); + // objects + for (ObsObject summary : objects.getObjects()) { + String key = summary.getObjectKey(); + Path keyPath = OBSCommonUtils.keyToQualifiedPath(owner, key); + if (LOG.isDebugEnabled()) { + LOG.debug("{}: {}", keyPath, + OBSCommonUtils.stringify(summary)); + } + // Skip over keys that are ourselves and old OBS _$folder$ files + if (acceptor.accept(keyPath, summary) && filter.accept( + keyPath)) { + FileStatus status = + OBSCommonUtils.createFileStatus( + keyPath, summary, + owner.getDefaultBlockSize(keyPath), + owner.getUsername()); + LOG.debug("Adding: {}", status); + stats.add(status); + added++; + } else { + LOG.debug("Ignoring: {}", keyPath); + ignored++; + } + } + + // prefixes: always directories + for (ObsObject prefix : objects.getExtenedCommonPrefixes()) { + String key = prefix.getObjectKey(); + Path keyPath = OBSCommonUtils.keyToQualifiedPath(owner, key); + if (acceptor.accept(keyPath, key) && filter.accept(keyPath)) { + long lastModified = + prefix.getMetadata().getLastModified() == null + ? System.currentTimeMillis() + : OBSCommonUtils.dateToLong( + prefix.getMetadata().getLastModified()); + FileStatus status = new OBSFileStatus(keyPath, lastModified, + lastModified, owner.getUsername()); + LOG.debug("Adding directory: {}", status); + added++; + stats.add(status); + } else { + LOG.debug("Ignoring directory: {}", keyPath); + ignored++; + } + } + + // finish up + batchSize = stats.size(); + statusBatchIterator = stats.listIterator(); + boolean hasNext = statusBatchIterator.hasNext(); + LOG.debug( + "Added {} entries; ignored {}; hasNext={}; hasMoreObjects={}", + added, + ignored, + hasNext, + objects.isTruncated()); + return hasNext; + } + + /** + * Get the number of entries in the current batch. + * + * @return a number, possibly zero. + */ + public int getBatchSize() { + return batchSize; + } + } + + /** + * Wraps up OBS `ListObjects` requests in a remote iterator which will ask for + * more listing data if needed. + * + *

That is: + * + *

1. The first invocation of the {@link #next()} call will return the + * results of the first request, the one created during the construction of + * the instance. + * + *

2. Second and later invocations will continue the ongoing listing, + * calling {@link OBSCommonUtils#continueListObjects} to request the next + * batch of results. + * + *

3. The {@link #hasNext()} predicate returns true for the initial call, + * where {@link #next()} will return the initial results. It declares that it + * has future results iff the last executed request was truncated. + * + *

Thread safety: none. + */ + class ObjectListingIterator implements RemoteIterator { + + /** + * The path listed. + */ + private final Path listPath; + + /** + * The most recent listing results. + */ + private ObjectListing objects; + + /** + * Indicator that this is the first listing. + */ + private boolean firstListing = true; + + /** + * Count of how many listings have been requested (including initial + * result). + */ + private int listingCount = 1; + + /** + * Maximum keys in a request. + */ + private int maxKeys; + + /** + * Constructor -calls {@link OBSCommonUtils#listObjects} on the request to + * populate the initial set of results/fail if there was a problem talking + * to the bucket. + * + * @param path path of the listing + * @param request initial request to make + * @throws IOException on any failure to list objects + */ + ObjectListingIterator(final Path path, + final ListObjectsRequest request) + throws IOException { + this.listPath = path; + this.maxKeys = owner.getMaxKeys(); + this.objects = OBSCommonUtils.listObjects(owner, request); + } + + /** + * Declare that the iterator has data if it is either is the initial + * iteration or it is a later one and the last listing obtained was + * incomplete. + */ + @Override + public boolean hasNext() { + return firstListing || objects.isTruncated(); + } + + /** + * Ask for the next listing. For the first invocation, this returns the + * initial set, with no remote IO. For later requests, OBS will be queried, + * hence the calls may block or fail. + * + * @return the next object listing. + * @throws IOException if a query made of OBS fails. + * @throws NoSuchElementException if there is no more data to list. + */ + @Override + public ObjectListing next() throws IOException { + if (firstListing) { + // on the first listing, don't request more data. + // Instead just clear the firstListing flag so that it future + // calls will request new data. + firstListing = false; + } else { + try { + if (!objects.isTruncated()) { + // nothing more to request: fail. + throw new NoSuchElementException( + "No more results in listing of " + listPath); + } + // need to request a new set of objects. + LOG.debug("[{}], Requesting next {} objects under {}", + listingCount, maxKeys, listPath); + objects = OBSCommonUtils.continueListObjects(owner, + objects); + listingCount++; + LOG.debug("New listing status: {}", this); + } catch (ObsException e) { + throw OBSCommonUtils.translateException("listObjects()", + listPath, e); + } + } + return objects; + } + + @Override + public String toString() { + return "Object listing iterator against " + + listPath + + "; listing count " + + listingCount + + "; isTruncated=" + + objects.isTruncated(); + } + + } + + /** + * Take a remote iterator over a set of {@link FileStatus} instances and + * return a remote iterator of {@link LocatedFileStatus} instances. + */ + class LocatedFileStatusIterator + implements RemoteIterator { + /** + * File status. + */ + private final RemoteIterator statusIterator; + + /** + * Constructor. + * + * @param statusRemoteIterator an iterator over the remote status entries + */ + LocatedFileStatusIterator( + final RemoteIterator statusRemoteIterator) { + this.statusIterator = statusRemoteIterator; + } + + @Override + public boolean hasNext() throws IOException { + return statusIterator.hasNext(); + } + + @Override + public LocatedFileStatus next() throws IOException { + return OBSCommonUtils.toLocatedFileStatus(owner, + statusIterator.next()); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java new file mode 100644 index 00000000000..cd9853369af --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java @@ -0,0 +1,350 @@ +/* + * 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.obs; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.net.URISyntaxException; +import java.net.URLDecoder; +import java.util.Objects; + +import static org.apache.commons.lang3.StringUtils.equalsIgnoreCase; + +/** + * Helper for OBS login. + */ +final class OBSLoginHelper { + /** + * login warning. + */ + public static final String LOGIN_WARNING = + "The Filesystem URI contains login details." + + " This is insecure and may be unsupported in future."; + + /** + * plus warning. + */ + public static final String PLUS_WARNING = + "Secret key contains a special character that should be URL encoded! " + + "Attempting to resolve..."; + + /** + * defined plus unencoded char. + */ + public static final String PLUS_UNENCODED = "+"; + + /** + * defined plus encoded char. + */ + public static final String PLUS_ENCODED = "%2B"; + + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + OBSLoginHelper.class); + + private OBSLoginHelper() { + } + + /** + * Build the filesystem URI. This can include stripping down of part of the + * URI. + * + * @param uri filesystem uri + * @return the URI to use as the basis for FS operation and qualifying paths. + * @throws IllegalArgumentException if the URI is in some way invalid. + */ + public static URI buildFSURI(final URI uri) { + Objects.requireNonNull(uri, "null uri"); + Objects.requireNonNull(uri.getScheme(), "null uri.getScheme()"); + if (uri.getHost() == null && uri.getAuthority() != null) { + Objects.requireNonNull( + uri.getHost(), + "null uri host." + + " This can be caused by unencoded / in the " + + "password string"); + } + Objects.requireNonNull(uri.getHost(), "null uri host."); + return URI.create(uri.getScheme() + "://" + uri.getHost()); + } + + /** + * Create a stripped down string value for error messages. + * + * @param pathUri URI + * @return a shortened schema://host/path value + */ + public static String toString(final URI pathUri) { + return pathUri != null + ? String.format("%s://%s/%s", pathUri.getScheme(), + pathUri.getHost(), pathUri.getPath()) + : "(null URI)"; + } + + /** + * Extract the login details from a URI, logging a warning if the URI contains + * these. + * + * @param name URI of the filesystem + * @return a login tuple, possibly empty. + */ + public static Login extractLoginDetailsWithWarnings(final URI name) { + Login login = extractLoginDetails(name); + if (login.hasLogin()) { + LOG.warn(LOGIN_WARNING); + } + return login; + } + + /** + * Extract the login details from a URI. + * + * @param name URI of the filesystem + * @return a login tuple, possibly empty. + */ + public static Login extractLoginDetails(final URI name) { + try { + String authority = name.getAuthority(); + if (authority == null) { + return Login.EMPTY; + } + int loginIndex = authority.indexOf('@'); + if (loginIndex < 0) { + // no login + return Login.EMPTY; + } + String login = authority.substring(0, loginIndex); + int loginSplit = login.indexOf(':'); + if (loginSplit > 0) { + String user = login.substring(0, loginSplit); + String encodedPassword = login.substring(loginSplit + 1); + if (encodedPassword.contains(PLUS_UNENCODED)) { + LOG.warn(PLUS_WARNING); + encodedPassword = encodedPassword.replaceAll( + "\\" + PLUS_UNENCODED, PLUS_ENCODED); + } + String password = URLDecoder.decode(encodedPassword, "UTF-8"); + return new Login(user, password); + } else if (loginSplit == 0) { + // there is no user, just a password. In this case, + // there's no login + return Login.EMPTY; + } else { + return new Login(login, ""); + } + } catch (UnsupportedEncodingException e) { + // this should never happen; translate it if it does. + throw new RuntimeException(e); + } + } + + /** + * Canonicalize the given URI. + * + *

This strips out login information. + * + * @param uri the URI to canonicalize + * @param defaultPort default port to use in canonicalized URI if the input + * URI has no port and this value is greater than 0 + * @return a new, canonicalized URI. + */ + public static URI canonicalizeUri(final URI uri, final int defaultPort) { + URI newUri = uri; + if (uri.getPort() == -1 && defaultPort > 0) { + // reconstruct the uri with the default port set + try { + newUri = + new URI( + newUri.getScheme(), + null, + newUri.getHost(), + defaultPort, + newUri.getPath(), + newUri.getQuery(), + newUri.getFragment()); + } catch (URISyntaxException e) { + // Should never happen! + throw new AssertionError( + "Valid URI became unparseable: " + newUri); + } + } + + return newUri; + } + + /** + * Check the path, ignoring authentication details. See {@link + * OBSFileSystem#checkPath(Path)} for the operation of this. + * + *

Essentially + * + *

    + *
  1. The URI is canonicalized. + *
  2. If the schemas match, the hosts are compared. + *
  3. If there is a mismatch between null/non-null host, + * the default FS values are used to patch in the host. + *
+ *

+ * That all originates in the core FS; the sole change here being to use + * {@link URI#getHost()}over {@link URI#getAuthority()}. Some of that code + * looks a relic of the code anti-pattern of using "hdfs:file.txt" to define + * the path without declaring the hostname. It's retained for compatibility. + * + * @param conf FS configuration + * @param fsUri the FS URI + * @param path path to check + * @param defaultPort default port of FS + */ + public static void checkPath(final Configuration conf, final URI fsUri, + final Path path, final int defaultPort) { + URI pathUri = path.toUri(); + String thatScheme = pathUri.getScheme(); + if (thatScheme == null) { + // fs is relative + return; + } + URI thisUri = canonicalizeUri(fsUri, defaultPort); + String thisScheme = thisUri.getScheme(); + // hostname and scheme are not case sensitive in these checks + if (equalsIgnoreCase(thisScheme, thatScheme)) { // schemes match + String thisHost = thisUri.getHost(); + String thatHost = pathUri.getHost(); + if (thatHost == null + && // path's host is null + thisHost != null) { // fs has a host + URI defaultUri = FileSystem.getDefaultUri(conf); + if (equalsIgnoreCase(thisScheme, defaultUri.getScheme())) { + pathUri + = defaultUri; // schemes match, so use this uri instead + } else { + pathUri = null; // can't determine auth of the path + } + } + if (pathUri != null) { + // canonicalize uri before comparing with this fs + pathUri = canonicalizeUri(pathUri, defaultPort); + thatHost = pathUri.getHost(); + if (equalsIgnoreCase(thisHost, thatHost)) { + return; + } + } + } + // make sure the exception strips out any auth details + throw new IllegalArgumentException( + "Wrong FS " + OBSLoginHelper.toString(pathUri) + " -expected " + + fsUri); + } + + /** + * Simple tuple of login details. + */ + public static class Login { + /** + * Defined empty login instance. + */ + public static final Login EMPTY = new Login(); + + /** + * Defined user name. + */ + private final String user; + + /** + * Defined password. + */ + private final String password; + + /** + * Login token. + */ + private final String token; + + /** + * Create an instance with no login details. Calls to {@link #hasLogin()} + * return false. + */ + Login() { + this("", ""); + } + + Login(final String userName, final String passwd) { + this(userName, passwd, null); + } + + Login(final String userName, final String passwd, + final String sessionToken) { + this.user = userName; + this.password = passwd; + this.token = sessionToken; + } + + /** + * Predicate to verify login details are defined. + * + * @return true if the username is defined (not null, not empty). + */ + public boolean hasLogin() { + return StringUtils.isNotEmpty(user); + } + + /** + * Equality test matches user and password. + * + * @param o other object + * @return true if the objects are considered equivalent. + */ + @Override + public boolean equals(final Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + Login that = (Login) o; + return Objects.equals(user, that.user) && Objects.equals(password, + that.password); + } + + @Override + public int hashCode() { + return Objects.hash(user, password); + } + + public String getUser() { + return user; + } + + public String getPassword() { + return password; + } + + public String getToken() { + return token; + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java new file mode 100644 index 00000000000..e632f61ca2d --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java @@ -0,0 +1,892 @@ +/* + * 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.obs; + +import com.obs.services.exception.ObsException; +import com.obs.services.model.AbortMultipartUploadRequest; +import com.obs.services.model.CompleteMultipartUploadRequest; +import com.obs.services.model.CopyObjectRequest; +import com.obs.services.model.CopyObjectResult; +import com.obs.services.model.CopyPartRequest; +import com.obs.services.model.CopyPartResult; +import com.obs.services.model.DeleteObjectsRequest; +import com.obs.services.model.GetObjectMetadataRequest; +import com.obs.services.model.InitiateMultipartUploadRequest; +import com.obs.services.model.InitiateMultipartUploadResult; +import com.obs.services.model.KeyAndVersion; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObjectMetadata; +import com.obs.services.model.ObsObject; +import com.obs.services.model.PartEtag; +import com.obs.services.model.PutObjectRequest; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.io.InterruptedIOException; +import java.util.ArrayList; +import java.util.Collections; +import java.util.Comparator; +import java.util.LinkedList; +import java.util.List; +import java.util.Set; +import java.util.TreeSet; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; + +/** + * Object bucket specific utils for {@link OBSFileSystem}. + */ +final class OBSObjectBucketUtils { + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + OBSObjectBucketUtils.class); + + private OBSObjectBucketUtils() { + + } + + /** + * The inner rename operation. + * + * @param owner OBS File System instance + * @param src path to be renamed + * @param dst new path after rename + * @return boolean + * @throws RenameFailedException if some criteria for a state changing rename + * was not met. This means work didn't happen; + * it's not something which is reported upstream + * to the FileSystem APIs, for which the + * semantics of "false" are pretty vague. + * @throws FileNotFoundException there's no source file. + * @throws IOException on IO failure. + * @throws ObsException on failures inside the OBS SDK + */ + static boolean renameBasedOnObject(final OBSFileSystem owner, + final Path src, final Path dst) throws RenameFailedException, + FileNotFoundException, IOException, + ObsException { + String srcKey = OBSCommonUtils.pathToKey(owner, src); + String dstKey = OBSCommonUtils.pathToKey(owner, dst); + + if (srcKey.isEmpty()) { + LOG.error("rename: src [{}] is root directory", src); + throw new IOException(src + " is root directory"); + } + + // get the source file status; this raises a FNFE if there is no source + // file. + FileStatus srcStatus = owner.getFileStatus(src); + + FileStatus dstStatus; + try { + dstStatus = owner.getFileStatus(dst); + // if there is no destination entry, an exception is raised. + // hence this code sequence can assume that there is something + // at the end of the path; the only detail being what it is and + // whether or not it can be the destination of the rename. + if (dstStatus.isDirectory()) { + String newDstKey = OBSCommonUtils.maybeAddTrailingSlash(dstKey); + String filename = srcKey.substring( + OBSCommonUtils.pathToKey(owner, src.getParent()).length() + + 1); + newDstKey = newDstKey + filename; + dstKey = newDstKey; + dstStatus = owner.getFileStatus( + OBSCommonUtils.keyToPath(dstKey)); + if (dstStatus.isDirectory()) { + throw new RenameFailedException(src, dst, + "new destination is an existed directory") + .withExitCode(false); + } else { + throw new RenameFailedException(src, dst, + "new destination is an existed file") + .withExitCode(false); + } + } else { + + if (srcKey.equals(dstKey)) { + LOG.warn( + "rename: src and dest refer to the same file or" + + " directory: {}", + dst); + return true; + } else { + throw new RenameFailedException(src, dst, + "destination is an existed file") + .withExitCode(false); + } + } + } catch (FileNotFoundException e) { + LOG.debug("rename: destination path {} not found", dst); + + // Parent must exist + checkDestinationParent(owner, src, dst); + } + + if (dstKey.startsWith(srcKey) + && dstKey.charAt(srcKey.length()) == Path.SEPARATOR_CHAR) { + LOG.error("rename: dest [{}] cannot be a descendant of src [{}]", + dst, src); + return false; + } + + // Ok! Time to start + if (srcStatus.isFile()) { + LOG.debug("rename: renaming file {} to {}", src, dst); + + renameFile(owner, srcKey, dstKey, srcStatus); + } else { + LOG.debug("rename: renaming directory {} to {}", src, dst); + + // This is a directory to directory copy + dstKey = OBSCommonUtils.maybeAddTrailingSlash(dstKey); + srcKey = OBSCommonUtils.maybeAddTrailingSlash(srcKey); + + renameFolder(owner, srcKey, dstKey); + } + + if (src.getParent() != dst.getParent()) { + // deleteUnnecessaryFakeDirectories(dst.getParent()); + createFakeDirectoryIfNecessary(owner, src.getParent()); + } + + return true; + } + + private static void checkDestinationParent(final OBSFileSystem owner, + final Path src, + final Path dst) throws IOException { + Path parent = dst.getParent(); + if (!OBSCommonUtils.pathToKey(owner, parent).isEmpty()) { + try { + FileStatus dstParentStatus = owner.getFileStatus( + dst.getParent()); + if (!dstParentStatus.isDirectory()) { + throw new ParentNotDirectoryException( + "destination parent [" + dst.getParent() + + "] is not a directory"); + } + } catch (FileNotFoundException e2) { + throw new RenameFailedException(src, dst, + "destination has no parent "); + } + } + } + + /** + * Implement rename file. + * + * @param owner OBS File System instance + * @param srcKey source object key + * @param dstKey destination object key + * @param srcStatus source object status + * @throws IOException any problem with rename operation + */ + private static void renameFile(final OBSFileSystem owner, + final String srcKey, + final String dstKey, + final FileStatus srcStatus) + throws IOException { + long startTime = System.nanoTime(); + + copyFile(owner, srcKey, dstKey, srcStatus.getLen()); + objectDelete(owner, srcStatus, false); + + if (LOG.isDebugEnabled()) { + long delay = System.nanoTime() - startTime; + LOG.debug("OBSFileSystem rename: " + + ", {src=" + + srcKey + + ", dst=" + + dstKey + + ", delay=" + + delay + + "}"); + } + } + + static boolean objectDelete(final OBSFileSystem owner, + final FileStatus status, + final boolean recursive) throws IOException { + Path f = status.getPath(); + String key = OBSCommonUtils.pathToKey(owner, f); + + if (status.isDirectory()) { + LOG.debug("delete: Path is a directory: {} - recursive {}", f, + recursive); + + key = OBSCommonUtils.maybeAddTrailingSlash(key); + if (!key.endsWith("/")) { + key = key + "/"; + } + + boolean isEmptyDir = OBSCommonUtils.isFolderEmpty(owner, key); + if (key.equals("/")) { + return OBSCommonUtils.rejectRootDirectoryDelete( + owner.getBucket(), isEmptyDir, recursive); + } + + if (!recursive && !isEmptyDir) { + throw new PathIsNotEmptyDirectoryException(f.toString()); + } + + if (isEmptyDir) { + LOG.debug( + "delete: Deleting fake empty directory {} - recursive {}", + f, recursive); + OBSCommonUtils.deleteObject(owner, key); + } else { + LOG.debug( + "delete: Deleting objects for directory prefix {} " + + "- recursive {}", + f, recursive); + deleteNonEmptyDir(owner, recursive, key); + } + + } else { + LOG.debug("delete: Path is a file"); + OBSCommonUtils.deleteObject(owner, key); + } + + Path parent = f.getParent(); + if (parent != null) { + createFakeDirectoryIfNecessary(owner, parent); + } + return true; + } + + /** + * Implement rename folder. + * + * @param owner OBS File System instance + * @param srcKey source folder key + * @param dstKey destination folder key + * @throws IOException any problem with rename folder + */ + static void renameFolder(final OBSFileSystem owner, final String srcKey, + final String dstKey) + throws IOException { + long startTime = System.nanoTime(); + + List keysToDelete = new ArrayList<>(); + + createFakeDirectory(owner, dstKey); + + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(owner.getBucket()); + request.setPrefix(srcKey); + request.setMaxKeys(owner.getMaxKeys()); + + ObjectListing objects = OBSCommonUtils.listObjects(owner, request); + + List> copyfutures = new LinkedList<>(); + while (true) { + for (ObsObject summary : objects.getObjects()) { + if (summary.getObjectKey().equals(srcKey)) { + // skip prefix itself + continue; + } + + keysToDelete.add(new KeyAndVersion(summary.getObjectKey())); + String newDstKey = dstKey + summary.getObjectKey() + .substring(srcKey.length()); + // copyFile(summary.getObjectKey(), newDstKey, + // summary.getMetadata().getContentLength()); + copyfutures.add( + copyFileAsync(owner, summary.getObjectKey(), newDstKey, + summary.getMetadata().getContentLength())); + + if (keysToDelete.size() == owner.getMaxEntriesToDelete()) { + waitAllCopyFinished(copyfutures); + copyfutures.clear(); + } + } + + if (!objects.isTruncated()) { + if (!keysToDelete.isEmpty()) { + waitAllCopyFinished(copyfutures); + copyfutures.clear(); + } + break; + } + objects = OBSCommonUtils.continueListObjects(owner, objects); + } + + keysToDelete.add(new KeyAndVersion(srcKey)); + + DeleteObjectsRequest deleteObjectsRequest = new DeleteObjectsRequest( + owner.getBucket()); + deleteObjectsRequest.setKeyAndVersions( + keysToDelete.toArray(new KeyAndVersion[0])); + OBSCommonUtils.deleteObjects(owner, deleteObjectsRequest); + + if (LOG.isDebugEnabled()) { + long delay = System.nanoTime() - startTime; + LOG.debug( + "OBSFileSystem rename: " + + ", {src=" + + srcKey + + ", dst=" + + dstKey + + ", delay=" + + delay + + "}"); + } + } + + private static void waitAllCopyFinished( + final List> copyFutures) + throws IOException { + try { + for (Future copyFuture : copyFutures) { + copyFuture.get(); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while copying objects (copy)"); + throw new InterruptedIOException( + "Interrupted while copying objects (copy)"); + } catch (ExecutionException e) { + for (Future future : copyFutures) { + future.cancel(true); + } + + throw OBSCommonUtils.extractException( + "waitAllCopyFinished", copyFutures.toString(), e); + } + } + + /** + * Request object metadata; increments counters in the process. + * + * @param owner OBS File System instance + * @param key key + * @return the metadata + */ + protected static ObjectMetadata getObjectMetadata(final OBSFileSystem owner, + final String key) { + GetObjectMetadataRequest request = new GetObjectMetadataRequest(); + request.setBucketName(owner.getBucket()); + request.setObjectKey(key); + if (owner.getSse().isSseCEnable()) { + request.setSseCHeader(owner.getSse().getSseCHeader()); + } + ObjectMetadata meta = owner.getObsClient().getObjectMetadata(request); + owner.getSchemeStatistics().incrementReadOps(1); + return meta; + } + + /** + * Create a new object metadata instance. Any standard metadata headers are + * added here, for example: encryption. + * + * @param length length of data to set in header. + * @return a new metadata instance + */ + static ObjectMetadata newObjectMetadata(final long length) { + final ObjectMetadata om = new ObjectMetadata(); + if (length >= 0) { + om.setContentLength(length); + } + return om; + } + + private static void deleteNonEmptyDir(final OBSFileSystem owner, + final boolean recursive, final String key) throws IOException { + String delimiter = recursive ? null : "/"; + ListObjectsRequest request = OBSCommonUtils.createListObjectsRequest( + owner, key, delimiter); + + ObjectListing objects = OBSCommonUtils.listObjects(owner, request); + List keys = new ArrayList<>(objects.getObjects().size()); + while (true) { + for (ObsObject summary : objects.getObjects()) { + if (summary.getObjectKey().equals(key)) { + // skip prefix itself + continue; + } + + keys.add(new KeyAndVersion(summary.getObjectKey())); + LOG.debug("Got object to delete {}", summary.getObjectKey()); + + if (keys.size() == owner.getMaxEntriesToDelete()) { + OBSCommonUtils.removeKeys(owner, keys, true, true); + } + } + + if (!objects.isTruncated()) { + keys.add(new KeyAndVersion(key)); + OBSCommonUtils.removeKeys(owner, keys, false, true); + + break; + } + objects = OBSCommonUtils.continueListObjects(owner, objects); + } + } + + static void createFakeDirectoryIfNecessary(final OBSFileSystem owner, + final Path f) + throws IOException, ObsException { + + String key = OBSCommonUtils.pathToKey(owner, f); + if (!key.isEmpty() && !owner.exists(f)) { + LOG.debug("Creating new fake directory at {}", f); + createFakeDirectory(owner, key); + } + } + + static void createFakeDirectory(final OBSFileSystem owner, + final String objectName) + throws ObsException, IOException { + String newObjectName = objectName; + newObjectName = OBSCommonUtils.maybeAddTrailingSlash(newObjectName); + createEmptyObject(owner, newObjectName); + } + + // Used to create an empty file that represents an empty directory + private static void createEmptyObject(final OBSFileSystem owner, + final String objectName) + throws ObsException, IOException { + for (int retryTime = 1; + retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) { + try { + innerCreateEmptyObject(owner, objectName); + return; + } catch (ObsException e) { + LOG.warn("Failed to create empty object [{}], retry time [{}], " + + "exception [{}]", objectName, retryTime, e); + try { + Thread.sleep(OBSCommonUtils.DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + } + + innerCreateEmptyObject(owner, objectName); + } + + // Used to create an empty file that represents an empty directory + private static void innerCreateEmptyObject(final OBSFileSystem owner, + final String objectName) + throws ObsException, IOException { + final InputStream im = + new InputStream() { + @Override + public int read() { + return -1; + } + }; + + PutObjectRequest putObjectRequest = OBSCommonUtils + .newPutObjectRequest(owner, objectName, newObjectMetadata(0L), im); + + long len; + if (putObjectRequest.getFile() != null) { + len = putObjectRequest.getFile().length(); + } else { + len = putObjectRequest.getMetadata().getContentLength(); + } + + try { + owner.getObsClient().putObject(putObjectRequest); + owner.getSchemeStatistics().incrementWriteOps(1); + owner.getSchemeStatistics().incrementBytesWritten(len); + } finally { + im.close(); + } + } + + /** + * Copy a single object in the bucket via a COPY operation. + * + * @param owner OBS File System instance + * @param srcKey source object path + * @param dstKey destination object path + * @param size object size + * @throws InterruptedIOException the operation was interrupted + * @throws IOException Other IO problems + */ + private static void copyFile(final OBSFileSystem owner, final String srcKey, + final String dstKey, final long size) + throws IOException, InterruptedIOException { + for (int retryTime = 1; + retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) { + try { + innerCopyFile(owner, srcKey, dstKey, size); + return; + } catch (InterruptedIOException e) { + throw e; + } catch (IOException e) { + LOG.warn( + "Failed to copy file from [{}] to [{}] with size [{}], " + + "retry time [{}], exception [{}]", srcKey, dstKey, + size, retryTime, e); + try { + Thread.sleep(OBSCommonUtils.DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + } + + innerCopyFile(owner, srcKey, dstKey, size); + } + + private static void innerCopyFile(final OBSFileSystem owner, + final String srcKey, + final String dstKey, final long size) + throws IOException { + LOG.debug("copyFile {} -> {} ", srcKey, dstKey); + try { + // 100MB per part + if (size > owner.getCopyPartSize()) { + // initial copy part task + InitiateMultipartUploadRequest request + = new InitiateMultipartUploadRequest(owner.getBucket(), + dstKey); + request.setAcl(owner.getCannedACL()); + if (owner.getSse().isSseCEnable()) { + request.setSseCHeader(owner.getSse().getSseCHeader()); + } else if (owner.getSse().isSseKmsEnable()) { + request.setSseKmsHeader(owner.getSse().getSseKmsHeader()); + } + InitiateMultipartUploadResult result = owner.getObsClient() + .initiateMultipartUpload(request); + + final String uploadId = result.getUploadId(); + LOG.debug("Multipart copy file, uploadId: {}", uploadId); + // count the parts + long partCount = calPartCount(owner.getCopyPartSize(), size); + + final List partEtags = + getCopyFilePartEtags(owner, srcKey, dstKey, size, uploadId, + partCount); + // merge the copy parts + CompleteMultipartUploadRequest completeMultipartUploadRequest = + new CompleteMultipartUploadRequest(owner.getBucket(), + dstKey, uploadId, partEtags); + owner.getObsClient() + .completeMultipartUpload(completeMultipartUploadRequest); + } else { + ObjectMetadata srcom = getObjectMetadata(owner, srcKey); + ObjectMetadata dstom = cloneObjectMetadata(srcom); + final CopyObjectRequest copyObjectRequest = + new CopyObjectRequest(owner.getBucket(), srcKey, + owner.getBucket(), dstKey); + copyObjectRequest.setAcl(owner.getCannedACL()); + copyObjectRequest.setNewObjectMetadata(dstom); + if (owner.getSse().isSseCEnable()) { + copyObjectRequest.setSseCHeader( + owner.getSse().getSseCHeader()); + copyObjectRequest.setSseCHeaderSource( + owner.getSse().getSseCHeader()); + } else if (owner.getSse().isSseKmsEnable()) { + copyObjectRequest.setSseKmsHeader( + owner.getSse().getSseKmsHeader()); + } + owner.getObsClient().copyObject(copyObjectRequest); + } + + owner.getSchemeStatistics().incrementWriteOps(1); + } catch (ObsException e) { + throw OBSCommonUtils.translateException( + "copyFile(" + srcKey + ", " + dstKey + ")", srcKey, e); + } + } + + static int calPartCount(final long partSize, final long cloudSize) { + // get user setting of per copy part size ,default is 100MB + // calculate the part count + long partCount = cloudSize % partSize == 0 + ? cloudSize / partSize + : cloudSize / partSize + 1; + return (int) partCount; + } + + static List getCopyFilePartEtags(final OBSFileSystem owner, + final String srcKey, + final String dstKey, + final long objectSize, + final String uploadId, + final long partCount) + throws IOException { + final List partEtags = Collections.synchronizedList( + new ArrayList<>()); + final List> partCopyFutures = new ArrayList<>(); + submitCopyPartTasks(owner, srcKey, dstKey, objectSize, uploadId, + partCount, partEtags, partCopyFutures); + + // wait the tasks for completing + try { + for (Future partCopyFuture : partCopyFutures) { + partCopyFuture.get(); + } + } catch (InterruptedException e) { + LOG.warn("Interrupted while copying objects (copy)"); + throw new InterruptedIOException( + "Interrupted while copying objects (copy)"); + } catch (ExecutionException e) { + LOG.error("Multipart copy file exception.", e); + for (Future future : partCopyFutures) { + future.cancel(true); + } + + owner.getObsClient() + .abortMultipartUpload( + new AbortMultipartUploadRequest(owner.getBucket(), dstKey, + uploadId)); + + throw OBSCommonUtils.extractException( + "Multi-part copy with id '" + uploadId + "' from " + srcKey + + "to " + dstKey, dstKey, e); + } + + // Make part numbers in ascending order + partEtags.sort(Comparator.comparingInt(PartEtag::getPartNumber)); + return partEtags; + } + + @SuppressWarnings("checkstyle:parameternumber") + private static void submitCopyPartTasks(final OBSFileSystem owner, + final String srcKey, + final String dstKey, + final long objectSize, + final String uploadId, + final long partCount, + final List partEtags, + final List> partCopyFutures) { + for (int i = 0; i < partCount; i++) { + final long rangeStart = i * owner.getCopyPartSize(); + final long rangeEnd = (i + 1 == partCount) + ? objectSize - 1 + : rangeStart + owner.getCopyPartSize() - 1; + final int partNumber = i + 1; + partCopyFutures.add( + owner.getBoundedCopyPartThreadPool().submit(() -> { + CopyPartRequest request = new CopyPartRequest(); + request.setUploadId(uploadId); + request.setSourceBucketName(owner.getBucket()); + request.setSourceObjectKey(srcKey); + request.setDestinationBucketName(owner.getBucket()); + request.setDestinationObjectKey(dstKey); + request.setByteRangeStart(rangeStart); + request.setByteRangeEnd(rangeEnd); + request.setPartNumber(partNumber); + if (owner.getSse().isSseCEnable()) { + request.setSseCHeaderSource( + owner.getSse().getSseCHeader()); + request.setSseCHeaderDestination( + owner.getSse().getSseCHeader()); + } + CopyPartResult result = owner.getObsClient() + .copyPart(request); + partEtags.add( + new PartEtag(result.getEtag(), result.getPartNumber())); + LOG.debug( + "Multipart copy file, uploadId: {}, Part#{} done.", + uploadId, partNumber); + })); + } + } + + /** + * Creates a copy of the passed {@link ObjectMetadata}. Does so without using + * the {@link ObjectMetadata#clone()} method, to avoid copying unnecessary + * headers. + * + * @param source the {@link ObjectMetadata} to copy + * @return a copy of {@link ObjectMetadata} with only relevant attributes + */ + private static ObjectMetadata cloneObjectMetadata( + final ObjectMetadata source) { + // This approach may be too brittle, especially if + // in future there are new attributes added to ObjectMetadata + // that we do not explicitly call to set here + ObjectMetadata ret = newObjectMetadata(source.getContentLength()); + + if (source.getContentEncoding() != null) { + ret.setContentEncoding(source.getContentEncoding()); + } + return ret; + } + + static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, + final Path f) + throws IOException { + final Path path = OBSCommonUtils.qualify(owner, f); + String key = OBSCommonUtils.pathToKey(owner, path); + LOG.debug("Getting path status for {} ({})", path, key); + if (!StringUtils.isEmpty(key)) { + try { + ObjectMetadata meta = getObjectMetadata(owner, key); + + if (OBSCommonUtils.objectRepresentsDirectory(key, + meta.getContentLength())) { + LOG.debug("Found exact file: fake directory"); + return new OBSFileStatus(path, owner.getUsername()); + } else { + LOG.debug("Found exact file: normal file"); + return new OBSFileStatus(meta.getContentLength(), + OBSCommonUtils.dateToLong(meta.getLastModified()), + path, owner.getDefaultBlockSize(path), + owner.getUsername()); + } + } catch (ObsException e) { + if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) { + throw OBSCommonUtils.translateException("getFileStatus", + path, e); + } + } + + if (!key.endsWith("/")) { + String newKey = key + "/"; + try { + ObjectMetadata meta = getObjectMetadata(owner, newKey); + + if (OBSCommonUtils.objectRepresentsDirectory(newKey, + meta.getContentLength())) { + LOG.debug("Found file (with /): fake directory"); + return new OBSFileStatus(path, owner.getUsername()); + } else { + LOG.debug( + "Found file (with /): real file? should not " + + "happen: {}", + key); + + return new OBSFileStatus(meta.getContentLength(), + OBSCommonUtils.dateToLong(meta.getLastModified()), + path, + owner.getDefaultBlockSize(path), + owner.getUsername()); + } + } catch (ObsException e) { + if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) { + throw OBSCommonUtils.translateException("getFileStatus", + newKey, e); + } + } + } + } + + try { + boolean isEmpty = OBSCommonUtils.innerIsFolderEmpty(owner, key); + LOG.debug("Is dir ({}) empty? {}", path, isEmpty); + return new OBSFileStatus(path, owner.getUsername()); + } catch (ObsException e) { + if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) { + throw OBSCommonUtils.translateException("getFileStatus", key, + e); + } + } + + LOG.debug("Not Found: {}", path); + throw new FileNotFoundException("No such file or directory: " + path); + } + + static ContentSummary getDirectoryContentSummary(final OBSFileSystem owner, + final String key) throws IOException { + String newKey = key; + newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey); + long[] summary = {0, 0, 1}; + LOG.debug("Summary key {}", newKey); + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(owner.getBucket()); + request.setPrefix(newKey); + Set directories = new TreeSet<>(); + request.setMaxKeys(owner.getMaxKeys()); + ObjectListing objects = OBSCommonUtils.listObjects(owner, request); + while (true) { + if (!objects.getCommonPrefixes().isEmpty() || !objects.getObjects() + .isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Found path as directory (with /): {}/{}", + objects.getCommonPrefixes().size(), + objects.getObjects().size()); + } + for (String prefix : objects.getCommonPrefixes()) { + LOG.debug("Objects in folder [" + prefix + "]:"); + getDirectories(prefix, newKey, directories); + } + + for (ObsObject obj : objects.getObjects()) { + LOG.debug("Summary: {} {}", obj.getObjectKey(), + obj.getMetadata().getContentLength()); + if (!obj.getObjectKey().endsWith("/")) { + summary[0] += obj.getMetadata().getContentLength(); + summary[1] += 1; + } + getDirectories(obj.getObjectKey(), newKey, directories); + } + } + if (!objects.isTruncated()) { + break; + } + objects = OBSCommonUtils.continueListObjects(owner, objects); + } + summary[2] += directories.size(); + LOG.debug(String.format( + "file size [%d] - file count [%d] - directory count [%d] - " + + "file path [%s]", + summary[0], + summary[1], summary[2], newKey)); + return new ContentSummary.Builder().length(summary[0]) + .fileCount(summary[1]).directoryCount(summary[2]) + .spaceConsumed(summary[0]).build(); + } + + private static void getDirectories(final String key, final String sourceKey, + final Set directories) { + Path p = new Path(key); + Path sourcePath = new Path(sourceKey); + // directory must add first + if (key.endsWith("/") && p.compareTo(sourcePath) > 0) { + directories.add(p.toString()); + } + while (p.compareTo(sourcePath) > 0) { + p = p.getParent(); + if (p.isRoot() || p.compareTo(sourcePath) == 0) { + break; + } + directories.add(p.toString()); + } + } + + private static Future copyFileAsync( + final OBSFileSystem owner, + final String srcKey, + final String dstKey, final long size) { + return owner.getBoundedCopyThreadPool().submit(() -> { + copyFile(owner, srcKey, dstKey, size); + return null; + }); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java new file mode 100644 index 00000000000..d6afd456969 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java @@ -0,0 +1,745 @@ +/* + * 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.obs; + +import com.obs.services.exception.ObsException; +import com.obs.services.model.KeyAndVersion; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObsObject; +import com.obs.services.model.fs.GetAttributeRequest; +import com.obs.services.model.fs.NewFolderRequest; +import com.obs.services.model.fs.ObsFSAttribute; +import com.obs.services.model.fs.RenameRequest; + +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; + +/** + * Posix bucket specific utils for {@link OBSFileSystem}. + */ +final class OBSPosixBucketUtils { + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + OBSPosixBucketUtils.class); + + private OBSPosixBucketUtils() { + } + + /** + * Get the depth of an absolute path, that is the number of '/' in the path. + * + * @param key object key + * @return depth + */ + static int fsGetObjectKeyDepth(final String key) { + int depth = 0; + for (int idx = key.indexOf('/'); + idx >= 0; idx = key.indexOf('/', idx + 1)) { + depth++; + } + return key.endsWith("/") ? depth - 1 : depth; + } + + /** + * Used to judge that an object is a file or folder. + * + * @param attr posix object attribute + * @return is posix folder + */ + static boolean fsIsFolder(final ObsFSAttribute attr) { + final int ifDir = 0x004000; + int mode = attr.getMode(); + // object mode is -1 when the object is migrated from + // object bucket to posix bucket. + // -1 is a file, not folder. + if (mode < 0) { + return false; + } + + return (mode & ifDir) != 0; + } + + /** + * The inner rename operation based on Posix bucket. + * + * @param owner OBS File System instance + * @param src source path to be renamed from + * @param dst destination path to be renamed to + * @return boolean + * @throws RenameFailedException if some criteria for a state changing rename + * was not met. This means work didn't happen; + * it's not something which is reported upstream + * to the FileSystem APIs, for which the + * semantics of "false" are pretty vague. + * @throws IOException on IO failure. + */ + static boolean renameBasedOnPosix(final OBSFileSystem owner, final Path src, + final Path dst) throws IOException { + Path dstPath = dst; + String srcKey = OBSCommonUtils.pathToKey(owner, src); + String dstKey = OBSCommonUtils.pathToKey(owner, dstPath); + + if (srcKey.isEmpty()) { + LOG.error("rename: src [{}] is root directory", src); + return false; + } + + try { + FileStatus dstStatus = owner.getFileStatus(dstPath); + if (dstStatus.isDirectory()) { + String newDstString = OBSCommonUtils.maybeAddTrailingSlash( + dstPath.toString()); + String filename = srcKey.substring( + OBSCommonUtils.pathToKey(owner, src.getParent()) + .length() + 1); + dstPath = new Path(newDstString + filename); + dstKey = OBSCommonUtils.pathToKey(owner, dstPath); + LOG.debug( + "rename: dest is an existing directory and will be " + + "changed to [{}]", dstPath); + + if (owner.exists(dstPath)) { + LOG.error("rename: failed to rename " + src + " to " + + dstPath + + " because destination exists"); + return false; + } + } else { + if (srcKey.equals(dstKey)) { + LOG.warn( + "rename: src and dest refer to the same " + + "file or directory: {}", dstPath); + return true; + } else { + LOG.error("rename: failed to rename " + src + " to " + + dstPath + + " because destination exists"); + return false; + } + } + } catch (FileNotFoundException e) { + // if destination does not exist, do not change the + // destination key, and just do rename. + LOG.debug("rename: dest [{}] does not exist", dstPath); + } catch (FileConflictException e) { + Path parent = dstPath.getParent(); + if (!OBSCommonUtils.pathToKey(owner, parent).isEmpty()) { + FileStatus dstParentStatus = owner.getFileStatus(parent); + if (!dstParentStatus.isDirectory()) { + throw new ParentNotDirectoryException( + parent + " is not a directory"); + } + } + } + + if (dstKey.startsWith(srcKey) && (dstKey.equals(srcKey) + || dstKey.charAt(srcKey.length()) == Path.SEPARATOR_CHAR)) { + LOG.error("rename: dest [{}] cannot be a descendant of src [{}]", + dstPath, src); + return false; + } + + return innerFsRenameWithRetry(owner, src, dstPath, srcKey, dstKey); + } + + private static boolean innerFsRenameWithRetry(final OBSFileSystem owner, + final Path src, + final Path dst, final String srcKey, final String dstKey) + throws IOException { + boolean renameResult = true; + int retryTime = 1; + while (retryTime <= OBSCommonUtils.MAX_RETRY_TIME) { + try { + LOG.debug("rename: {}-st rename from [{}] to [{}] ...", + retryTime, srcKey, dstKey); + innerFsRenameFile(owner, srcKey, dstKey); + renameResult = true; + break; + } catch (FileNotFoundException e) { + if (owner.exists(dst)) { + LOG.warn( + "rename: successfully {}-st rename src [{}] " + + "to dest [{}] with SDK retry", + retryTime, src, dst, e); + renameResult = true; + } else { + LOG.error( + "rename: failed {}-st rename src [{}] to dest [{}]", + retryTime, src, dst, e); + renameResult = false; + } + break; + } catch (IOException e) { + if (retryTime == OBSCommonUtils.MAX_RETRY_TIME) { + LOG.error( + "rename: failed {}-st rename src [{}] to dest [{}]", + retryTime, src, dst, e); + throw e; + } else { + LOG.warn( + "rename: failed {}-st rename src [{}] to dest [{}]", + retryTime, src, dst, e); + if (owner.exists(dst) && owner.exists(src)) { + LOG.warn( + "rename: failed {}-st rename src [{}] to " + + "dest [{}] with SDK retry", retryTime, src, + dst, e); + renameResult = false; + break; + } + + try { + Thread.sleep(OBSCommonUtils.DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + } + + retryTime++; + } + + return renameResult; + } + + /** + * Used to rename a source folder to a destination folder that is not existed + * before rename. + * + * @param owner OBS File System instance + * @param src source folder key + * @param dst destination folder key that not existed before rename + * @throws IOException any io exception + * @throws ObsException any obs operation exception + */ + static void fsRenameToNewFolder(final OBSFileSystem owner, final String src, + final String dst) + throws IOException, ObsException { + LOG.debug("RenameFolder path {} to {}", src, dst); + + try { + RenameRequest renameObjectRequest = new RenameRequest(); + renameObjectRequest.setBucketName(owner.getBucket()); + renameObjectRequest.setObjectKey(src); + renameObjectRequest.setNewObjectKey(dst); + owner.getObsClient().renameFolder(renameObjectRequest); + owner.getSchemeStatistics().incrementWriteOps(1); + } catch (ObsException e) { + throw OBSCommonUtils.translateException( + "renameFile(" + src + ", " + dst + ")", src, e); + } + } + + static void innerFsRenameFile(final OBSFileSystem owner, + final String srcKey, + final String dstKey) throws IOException { + LOG.debug("RenameFile path {} to {}", srcKey, dstKey); + + try { + final RenameRequest renameObjectRequest = new RenameRequest(); + renameObjectRequest.setBucketName(owner.getBucket()); + renameObjectRequest.setObjectKey(srcKey); + renameObjectRequest.setNewObjectKey(dstKey); + owner.getObsClient().renameFile(renameObjectRequest); + owner.getSchemeStatistics().incrementWriteOps(1); + } catch (ObsException e) { + if (e.getResponseCode() == OBSCommonUtils.NOT_FOUND_CODE) { + throw new FileNotFoundException( + "No such file or directory: " + srcKey); + } + if (e.getResponseCode() == OBSCommonUtils.CONFLICT_CODE) { + throw new FileConflictException( + "File conflicts during rename, " + e.getResponseStatus()); + } + throw OBSCommonUtils.translateException( + "renameFile(" + srcKey + ", " + dstKey + ")", srcKey, e); + } + } + + /** + * Used to rename a source object to a destination object which is not existed + * before rename. + * + * @param owner OBS File System instance + * @param srcKey source object key + * @param dstKey destination object key + * @throws IOException io exception + */ + static void fsRenameToNewObject(final OBSFileSystem owner, + final String srcKey, + final String dstKey) throws IOException { + String newSrcKey = srcKey; + String newdstKey = dstKey; + newSrcKey = OBSCommonUtils.maybeDeleteBeginningSlash(newSrcKey); + newdstKey = OBSCommonUtils.maybeDeleteBeginningSlash(newdstKey); + if (newSrcKey.endsWith("/")) { + // Rename folder. + fsRenameToNewFolder(owner, newSrcKey, newdstKey); + } else { + // Rename file. + innerFsRenameFile(owner, newSrcKey, newdstKey); + } + } + + // Delete a file. + private static int fsRemoveFile(final OBSFileSystem owner, + final String sonObjectKey, + final List files) + throws IOException { + files.add(new KeyAndVersion(sonObjectKey)); + if (files.size() == owner.getMaxEntriesToDelete()) { + // batch delete files. + OBSCommonUtils.removeKeys(owner, files, true, false); + return owner.getMaxEntriesToDelete(); + } + return 0; + } + + // Recursively delete a folder that might be not empty. + static boolean fsDelete(final OBSFileSystem owner, final FileStatus status, + final boolean recursive) + throws IOException, ObsException { + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + Path f = status.getPath(); + String key = OBSCommonUtils.pathToKey(owner, f); + + if (!status.isDirectory()) { + LOG.debug("delete: Path is a file"); + trashObjectIfNeed(owner, key); + } else { + LOG.debug("delete: Path is a directory: {} - recursive {}", f, + recursive); + key = OBSCommonUtils.maybeAddTrailingSlash(key); + boolean isEmptyDir = OBSCommonUtils.isFolderEmpty(owner, key); + if (key.equals("")) { + return OBSCommonUtils.rejectRootDirectoryDelete( + owner.getBucket(), isEmptyDir, recursive); + } + if (!recursive && !isEmptyDir) { + LOG.warn("delete: Path is not empty: {} - recursive {}", f, + recursive); + throw new PathIsNotEmptyDirectoryException(f.toString()); + } + if (isEmptyDir) { + LOG.debug( + "delete: Deleting fake empty directory {} - recursive {}", + f, recursive); + OBSCommonUtils.deleteObject(owner, key); + } else { + LOG.debug( + "delete: Deleting objects for directory prefix {} to " + + "delete - recursive {}", f, recursive); + trashFolderIfNeed(owner, key, f); + } + } + + long endTime = System.currentTimeMillis(); + LOG.debug("delete Path:{} thread:{}, timeUsedInMilliSec:{}", f, + threadId, endTime - startTime); + return true; + } + + private static void trashObjectIfNeed(final OBSFileSystem owner, + final String key) + throws ObsException, IOException { + if (needToTrash(owner, key)) { + mkTrash(owner, key); + StringBuilder sb = new StringBuilder(owner.getTrashDir()); + sb.append(key); + if (owner.exists(new Path(sb.toString()))) { + SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmss"); + sb.append(df.format(new Date())); + } + fsRenameToNewObject(owner, key, sb.toString()); + LOG.debug("Moved: '" + key + "' to trash at: " + sb.toString()); + } else { + OBSCommonUtils.deleteObject(owner, key); + } + } + + private static void trashFolderIfNeed(final OBSFileSystem owner, + final String key, + final Path f) throws ObsException, IOException { + if (needToTrash(owner, key)) { + mkTrash(owner, key); + StringBuilder sb = new StringBuilder(owner.getTrashDir()); + String subKey = OBSCommonUtils.maybeAddTrailingSlash(key); + sb.append(subKey); + if (owner.exists(new Path(sb.toString()))) { + SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmss"); + sb.insert(sb.length() - 1, df.format(new Date())); + } + + String srcKey = OBSCommonUtils.maybeDeleteBeginningSlash(key); + String dstKey = OBSCommonUtils.maybeDeleteBeginningSlash( + sb.toString()); + fsRenameToNewFolder(owner, srcKey, dstKey); + LOG.debug("Moved: '" + key + "' to trash at: " + sb.toString()); + } else { + if (owner.isEnableMultiObjectDeleteRecursion()) { + long delNum = fsRecursivelyDeleteDir(owner, key, true); + LOG.debug("Recursively delete {} files/dirs when deleting {}", + delNum, key); + } else { + fsNonRecursivelyDelete(owner, f); + } + } + } + + static long fsRecursivelyDeleteDir(final OBSFileSystem owner, + final String parentKey, + final boolean deleteParent) throws IOException { + long delNum = 0; + List subdirList = new ArrayList<>( + owner.getMaxEntriesToDelete()); + List fileList = new ArrayList<>( + owner.getMaxEntriesToDelete()); + + ListObjectsRequest request = OBSCommonUtils.createListObjectsRequest( + owner, parentKey, "/", owner.getMaxKeys()); + ObjectListing objects = OBSCommonUtils.listObjects(owner, request); + while (true) { + for (String commonPrefix : objects.getCommonPrefixes()) { + if (commonPrefix.equals(parentKey)) { + // skip prefix itself + continue; + } + + delNum += fsRemoveSubdir(owner, commonPrefix, subdirList); + } + + for (ObsObject sonObject : objects.getObjects()) { + String sonObjectKey = sonObject.getObjectKey(); + + if (sonObjectKey.equals(parentKey)) { + // skip prefix itself + continue; + } + + if (!sonObjectKey.endsWith("/")) { + delNum += fsRemoveFile(owner, sonObjectKey, fileList); + } else { + delNum += fsRemoveSubdir(owner, sonObjectKey, subdirList); + } + } + + if (!objects.isTruncated()) { + break; + } + + objects = OBSCommonUtils.continueListObjects(owner, objects); + } + + delNum += fileList.size(); + OBSCommonUtils.removeKeys(owner, fileList, true, false); + + delNum += subdirList.size(); + OBSCommonUtils.removeKeys(owner, subdirList, true, false); + + if (deleteParent) { + OBSCommonUtils.deleteObject(owner, parentKey); + delNum++; + } + + return delNum; + } + + private static boolean needToTrash(final OBSFileSystem owner, + final String key) { + String newKey = key; + newKey = OBSCommonUtils.maybeDeleteBeginningSlash(newKey); + if (owner.isEnableTrash() && newKey.startsWith(owner.getTrashDir())) { + return false; + } + return owner.isEnableTrash(); + } + + // Delete a sub dir. + private static int fsRemoveSubdir(final OBSFileSystem owner, + final String subdirKey, + final List subdirList) + throws IOException { + fsRecursivelyDeleteDir(owner, subdirKey, false); + + subdirList.add(new KeyAndVersion(subdirKey)); + if (subdirList.size() == owner.getMaxEntriesToDelete()) { + // batch delete subdirs. + OBSCommonUtils.removeKeys(owner, subdirList, true, false); + return owner.getMaxEntriesToDelete(); + } + + return 0; + } + + private static void mkTrash(final OBSFileSystem owner, final String key) + throws ObsException, IOException { + String newKey = key; + StringBuilder sb = new StringBuilder(owner.getTrashDir()); + newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey); + sb.append(newKey); + sb.deleteCharAt(sb.length() - 1); + sb.delete(sb.lastIndexOf("/"), sb.length()); + Path fastDeleteRecycleDirPath = new Path(sb.toString()); + // keep the parent directory of the target path exists + if (!owner.exists(fastDeleteRecycleDirPath)) { + owner.mkdirs(fastDeleteRecycleDirPath); + } + } + + // List all sub objects at first, delete sub objects in batch secondly. + private static void fsNonRecursivelyDelete(final OBSFileSystem owner, + final Path parent) + throws IOException, ObsException { + // List sub objects sorted by path depth. + FileStatus[] arFileStatus = OBSCommonUtils.innerListStatus(owner, + parent, true); + // Remove sub objects one depth by one depth to avoid that parents and + // children in a same batch. + fsRemoveKeys(owner, arFileStatus); + // Delete parent folder that should has become empty. + OBSCommonUtils.deleteObject(owner, + OBSCommonUtils.pathToKey(owner, parent)); + } + + // Remove sub objects of each depth one by one to avoid that parents and + // children in a same batch. + private static void fsRemoveKeys(final OBSFileSystem owner, + final FileStatus[] arFileStatus) + throws ObsException, IOException { + if (arFileStatus.length <= 0) { + // exit fast if there are no keys to delete + return; + } + + String key; + for (FileStatus fileStatus : arFileStatus) { + key = OBSCommonUtils.pathToKey(owner, fileStatus.getPath()); + OBSCommonUtils.blockRootDelete(owner.getBucket(), key); + } + + fsRemoveKeysByDepth(owner, arFileStatus); + } + + // Batch delete sub objects one depth by one depth to avoid that parents and + // children in a same + // batch. + // A batch deletion might be split into some concurrent deletions to promote + // the performance, but + // it + // can't make sure that an object is deleted before it's children. + private static void fsRemoveKeysByDepth(final OBSFileSystem owner, + final FileStatus[] arFileStatus) + throws ObsException, IOException { + if (arFileStatus.length <= 0) { + // exit fast if there is no keys to delete + return; + } + + // Find all leaf keys in the list. + String key; + int depth = Integer.MAX_VALUE; + List leafKeys = new ArrayList<>( + owner.getMaxEntriesToDelete()); + for (int idx = arFileStatus.length - 1; idx >= 0; idx--) { + if (leafKeys.size() >= owner.getMaxEntriesToDelete()) { + OBSCommonUtils.removeKeys(owner, leafKeys, true, false); + } + + key = OBSCommonUtils.pathToKey(owner, arFileStatus[idx].getPath()); + + // Check file. + if (!arFileStatus[idx].isDirectory()) { + // A file must be a leaf. + leafKeys.add(new KeyAndVersion(key, null)); + continue; + } + + // Check leaf folder at current depth. + int keyDepth = fsGetObjectKeyDepth(key); + if (keyDepth == depth) { + // Any key at current depth must be a leaf. + leafKeys.add(new KeyAndVersion(key, null)); + continue; + } + if (keyDepth < depth) { + // The last batch delete at current depth. + OBSCommonUtils.removeKeys(owner, leafKeys, true, false); + // Go on at the upper depth. + depth = keyDepth; + leafKeys.add(new KeyAndVersion(key, null)); + continue; + } + LOG.warn( + "The objects list is invalid because it isn't sorted by" + + " path depth."); + throw new ObsException("System failure"); + } + + // The last batch delete at the minimum depth of all keys. + OBSCommonUtils.removeKeys(owner, leafKeys, true, false); + } + + // Used to create a folder + static void fsCreateFolder(final OBSFileSystem owner, + final String objectName) + throws ObsException { + for (int retryTime = 1; + retryTime < OBSCommonUtils.MAX_RETRY_TIME; retryTime++) { + try { + innerFsCreateFolder(owner, objectName); + return; + } catch (ObsException e) { + LOG.warn("Failed to create folder [{}], retry time [{}], " + + "exception [{}]", objectName, retryTime, e); + try { + Thread.sleep(OBSCommonUtils.DELAY_TIME); + } catch (InterruptedException ie) { + throw e; + } + } + } + + innerFsCreateFolder(owner, objectName); + } + + private static void innerFsCreateFolder(final OBSFileSystem owner, + final String objectName) + throws ObsException { + final NewFolderRequest newFolderRequest = new NewFolderRequest( + owner.getBucket(), objectName); + newFolderRequest.setAcl(owner.getCannedACL()); + long len = newFolderRequest.getObjectKey().length(); + owner.getObsClient().newFolder(newFolderRequest); + owner.getSchemeStatistics().incrementWriteOps(1); + owner.getSchemeStatistics().incrementBytesWritten(len); + } + + // Used to get the status of a file or folder in a file-gateway bucket. + static OBSFileStatus innerFsGetObjectStatus(final OBSFileSystem owner, + final Path f) throws IOException { + final Path path = OBSCommonUtils.qualify(owner, f); + String key = OBSCommonUtils.pathToKey(owner, path); + LOG.debug("Getting path status for {} ({})", path, key); + + if (key.isEmpty()) { + LOG.debug("Found root directory"); + return new OBSFileStatus(path, owner.getUsername()); + } + + try { + final GetAttributeRequest getAttrRequest = new GetAttributeRequest( + owner.getBucket(), key); + ObsFSAttribute meta = owner.getObsClient() + .getAttribute(getAttrRequest); + owner.getSchemeStatistics().incrementReadOps(1); + if (fsIsFolder(meta)) { + LOG.debug("Found file (with /): fake directory"); + return new OBSFileStatus(path, + OBSCommonUtils.dateToLong(meta.getLastModified()), + owner.getUsername()); + } else { + LOG.debug( + "Found file (with /): real file? should not happen: {}", + key); + return new OBSFileStatus( + meta.getContentLength(), + OBSCommonUtils.dateToLong(meta.getLastModified()), + path, + owner.getDefaultBlockSize(path), + owner.getUsername()); + } + } catch (ObsException e) { + if (e.getResponseCode() == OBSCommonUtils.NOT_FOUND_CODE) { + LOG.debug("Not Found: {}", path); + throw new FileNotFoundException( + "No such file or directory: " + path); + } + if (e.getResponseCode() == OBSCommonUtils.CONFLICT_CODE) { + throw new FileConflictException( + "file conflicts: " + e.getResponseStatus()); + } + throw OBSCommonUtils.translateException("getFileStatus", path, e); + } + } + + static ContentSummary fsGetDirectoryContentSummary( + final OBSFileSystem owner, + final String key) throws IOException { + String newKey = key; + newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey); + long[] summary = {0, 0, 1}; + LOG.debug("Summary key {}", newKey); + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(owner.getBucket()); + request.setPrefix(newKey); + request.setMaxKeys(owner.getMaxKeys()); + ObjectListing objects = OBSCommonUtils.listObjects(owner, request); + while (true) { + if (!objects.getCommonPrefixes().isEmpty() || !objects.getObjects() + .isEmpty()) { + if (LOG.isDebugEnabled()) { + LOG.debug("Found path as directory (with /): {}/{}", + objects.getCommonPrefixes().size(), + objects.getObjects().size()); + } + for (String prefix : objects.getCommonPrefixes()) { + if (!prefix.equals(newKey)) { + summary[2]++; + } + } + + for (ObsObject obj : objects.getObjects()) { + if (!obj.getObjectKey().endsWith("/")) { + summary[0] += obj.getMetadata().getContentLength(); + summary[1] += 1; + } else if (!obj.getObjectKey().equals(newKey)) { + summary[2]++; + } + } + } + if (!objects.isTruncated()) { + break; + } + objects = OBSCommonUtils.continueListObjects(owner, objects); + } + LOG.debug(String.format( + "file size [%d] - file count [%d] - directory count [%d] - " + + "file path [%s]", + summary[0], summary[1], summary[2], newKey)); + return new ContentSummary.Builder().length(summary[0]) + .fileCount(summary[1]).directoryCount(summary[2]) + .spaceConsumed(summary[0]).build(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java new file mode 100644 index 00000000000..5cc3008f1dc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java @@ -0,0 +1,310 @@ +/* + * 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.obs; + +import org.apache.hadoop.thirdparty.com.google.common.base.Preconditions; +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import com.obs.services.model.AbortMultipartUploadRequest; +import com.obs.services.model.CompleteMultipartUploadRequest; +import com.obs.services.model.CompleteMultipartUploadResult; +import com.obs.services.model.InitiateMultipartUploadRequest; +import com.obs.services.model.ObjectMetadata; +import com.obs.services.model.PartEtag; +import com.obs.services.model.PutObjectRequest; +import com.obs.services.model.PutObjectResult; +import com.obs.services.model.UploadPartRequest; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.io.InputStream; +import java.util.ArrayList; +import java.util.List; + +/** + * Helper for an ongoing write operation. + * + *

It hides direct access to the OBS API from the output stream, and is a + * location where the object upload process can be evolved/enhanced. + * + *

Features + * + *

    + *
  • Methods to create and submit requests to OBS, so avoiding all direct + * interaction with the OBS APIs. + *
  • Some extra preflight checks of arguments, so failing fast on errors. + *
  • Callbacks to let the FS know of events in the output stream upload + * process. + *
+ *

+ * Each instance of this state is unique to a single output stream. + */ +class OBSWriteOperationHelper { + /** + * Class logger. + */ + public static final Logger LOG = LoggerFactory.getLogger( + OBSWriteOperationHelper.class); + + /** + * Part number of the multipart task. + */ + static final int PART_NUMBER = 10000; + + /** + * Owning filesystem. + */ + private final OBSFileSystem owner; + + /** + * Bucket of the owner FS. + */ + private final String bucket; + + /** + * Define obs client. + */ + private final ObsClient obs; + + protected OBSWriteOperationHelper(final OBSFileSystem fs) { + this.owner = fs; + this.bucket = fs.getBucket(); + this.obs = fs.getObsClient(); + } + + /** + * Create a {@link PutObjectRequest} request. If {@code length} is set, the + * metadata is configured with the size of the upload. + * + * @param destKey key of object + * @param inputStream source data + * @param length size, if known. Use -1 for not known + * @return the request + */ + PutObjectRequest newPutRequest(final String destKey, + final InputStream inputStream, + final long length) { + return OBSCommonUtils.newPutObjectRequest(owner, destKey, + newObjectMetadata(length), inputStream); + } + + /** + * Create a {@link PutObjectRequest} request to upload a file. + * + * @param destKey object key for request + * @param sourceFile source file + * @return the request + */ + PutObjectRequest newPutRequest(final String destKey, + final File sourceFile) { + int length = (int) sourceFile.length(); + return OBSCommonUtils.newPutObjectRequest(owner, destKey, + newObjectMetadata(length), sourceFile); + } + + /** + * Callback on a successful write. + * + * @param destKey object key + */ + void writeSuccessful(final String destKey) { + LOG.debug("Finished write to {}", destKey); + } + + /** + * Create a new object metadata instance. Any standard metadata headers are + * added here, for example: encryption. + * + * @param length size, if known. Use -1 for not known + * @return a new metadata instance + */ + public ObjectMetadata newObjectMetadata(final long length) { + return OBSObjectBucketUtils.newObjectMetadata(length); + } + + /** + * Start the multipart upload process. + * + * @param destKey object key + * @return the upload result containing the ID + * @throws IOException IO problem + */ + String initiateMultiPartUpload(final String destKey) throws IOException { + LOG.debug("Initiating Multipart upload"); + final InitiateMultipartUploadRequest initiateMPURequest = + new InitiateMultipartUploadRequest(bucket, destKey); + initiateMPURequest.setAcl(owner.getCannedACL()); + initiateMPURequest.setMetadata(newObjectMetadata(-1)); + if (owner.getSse().isSseCEnable()) { + initiateMPURequest.setSseCHeader(owner.getSse().getSseCHeader()); + } else if (owner.getSse().isSseKmsEnable()) { + initiateMPURequest.setSseKmsHeader( + owner.getSse().getSseKmsHeader()); + } + try { + return obs.initiateMultipartUpload(initiateMPURequest) + .getUploadId(); + } catch (ObsException ace) { + throw OBSCommonUtils.translateException("Initiate MultiPartUpload", + destKey, ace); + } + } + + /** + * Complete a multipart upload operation. + * + * @param destKey Object key + * @param uploadId multipart operation Id + * @param partETags list of partial uploads + * @return the result + * @throws ObsException on problems. + */ + CompleteMultipartUploadResult completeMultipartUpload( + final String destKey, final String uploadId, + final List partETags) + throws ObsException { + Preconditions.checkNotNull(uploadId); + Preconditions.checkNotNull(partETags); + Preconditions.checkArgument(!partETags.isEmpty(), + "No partitions have been uploaded"); + LOG.debug("Completing multipart upload {} with {} parts", uploadId, + partETags.size()); + // a copy of the list is required, so that the OBS SDK doesn't + // attempt to sort an unmodifiable list. + return obs.completeMultipartUpload( + new CompleteMultipartUploadRequest(bucket, destKey, uploadId, + new ArrayList<>(partETags))); + } + + /** + * Abort a multipart upload operation. + * + * @param destKey object key + * @param uploadId multipart operation Id + * @throws ObsException on problems. Immediately execute + */ + void abortMultipartUpload(final String destKey, final String uploadId) + throws ObsException { + LOG.debug("Aborting multipart upload {}", uploadId); + obs.abortMultipartUpload( + new AbortMultipartUploadRequest(bucket, destKey, uploadId)); + } + + /** + * Create request for uploading one part of a multipart task. + * + * @param destKey destination object key + * @param uploadId upload id + * @param partNumber part number + * @param size data size + * @param sourceFile source file to be uploaded + * @return part upload request + */ + UploadPartRequest newUploadPartRequest( + final String destKey, + final String uploadId, + final int partNumber, + final int size, + final File sourceFile) { + Preconditions.checkNotNull(uploadId); + + Preconditions.checkArgument(sourceFile != null, "Data source"); + Preconditions.checkArgument(size > 0, "Invalid partition size %s", + size); + Preconditions.checkArgument( + partNumber > 0 && partNumber <= PART_NUMBER); + + LOG.debug("Creating part upload request for {} #{} size {}", uploadId, + partNumber, size); + UploadPartRequest request = new UploadPartRequest(); + request.setUploadId(uploadId); + request.setBucketName(bucket); + request.setObjectKey(destKey); + request.setPartSize((long) size); + request.setPartNumber(partNumber); + request.setFile(sourceFile); + if (owner.getSse().isSseCEnable()) { + request.setSseCHeader(owner.getSse().getSseCHeader()); + } + return request; + } + + /** + * Create request for uploading one part of a multipart task. + * + * @param destKey destination object key + * @param uploadId upload id + * @param partNumber part number + * @param size data size + * @param uploadStream upload stream for the part + * @return part upload request + */ + UploadPartRequest newUploadPartRequest( + final String destKey, + final String uploadId, + final int partNumber, + final int size, + final InputStream uploadStream) { + Preconditions.checkNotNull(uploadId); + + Preconditions.checkArgument(uploadStream != null, "Data source"); + Preconditions.checkArgument(size > 0, "Invalid partition size %s", + size); + Preconditions.checkArgument( + partNumber > 0 && partNumber <= PART_NUMBER); + + LOG.debug("Creating part upload request for {} #{} size {}", uploadId, + partNumber, size); + UploadPartRequest request = new UploadPartRequest(); + request.setUploadId(uploadId); + request.setBucketName(bucket); + request.setObjectKey(destKey); + request.setPartSize((long) size); + request.setPartNumber(partNumber); + request.setInput(uploadStream); + if (owner.getSse().isSseCEnable()) { + request.setSseCHeader(owner.getSse().getSseCHeader()); + } + return request; + } + + public String toString(final String destKey) { + return "{bucket=" + bucket + ", key='" + destKey + '\'' + '}'; + } + + /** + * PUT an object directly (i.e. not via the transfer manager). + * + * @param putObjectRequest the request + * @return the upload initiated + * @throws IOException on problems + */ + PutObjectResult putObject(final PutObjectRequest putObjectRequest) + throws IOException { + try { + return OBSCommonUtils.putObjectDirect(owner, putObjectRequest); + } catch (ObsException e) { + throw OBSCommonUtils.translateException("put", + putObjectRequest.getObjectKey(), e); + } + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java new file mode 100644 index 00000000000..b7f7965ebe2 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java @@ -0,0 +1,57 @@ +/* + * 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.obs; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathIOException; + +/** + * Exception to indicate a specific rename failure. The exit code defines the + * value returned by {@link OBSFileSystem#rename(Path, Path)}. + */ +class RenameFailedException extends PathIOException { + /** + * Exit code to be returned. + */ + private boolean exitCode = false; + + RenameFailedException(final Path src, final Path optionalDest, + final String error) { + super(src.toString(), error); + setOperation("rename"); + if (optionalDest != null) { + setTargetPath(optionalDest.toString()); + } + } + + public boolean getExitCode() { + return exitCode; + } + + /** + * Set the exit code. + * + * @param code exit code to raise + * @return the exception + */ + public RenameFailedException withExitCode(final boolean code) { + this.exitCode = code; + return this; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SseWrapper.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SseWrapper.java new file mode 100644 index 00000000000..d14479c2d85 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SseWrapper.java @@ -0,0 +1,87 @@ +/* + * 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.obs; + +import static org.apache.hadoop.fs.obs.OBSConstants.SSE_KEY; +import static org.apache.hadoop.fs.obs.OBSConstants.SSE_TYPE; + +import com.obs.services.model.SseCHeader; +import com.obs.services.model.SseKmsHeader; + +import org.apache.hadoop.conf.Configuration; + +/** + * Wrapper for Server-Side Encryption (SSE). + */ +class SseWrapper { + /** + * SSE-KMS: Server-Side Encryption with Key Management Service. + */ + private static final String SSE_KMS = "sse-kms"; + + /** + * SSE-C: Server-Side Encryption with Customer-Provided Encryption Keys. + */ + private static final String SSE_C = "sse-c"; + + /** + * SSE-C header. + */ + private SseCHeader sseCHeader; + + /** + * SSE-KMS header. + */ + private SseKmsHeader sseKmsHeader; + + @SuppressWarnings("deprecation") + SseWrapper(final Configuration conf) { + String sseType = conf.getTrimmed(SSE_TYPE); + if (null != sseType) { + String sseKey = conf.getTrimmed(SSE_KEY); + if (sseType.equalsIgnoreCase(SSE_C) && null != sseKey) { + sseCHeader = new SseCHeader(); + sseCHeader.setSseCKeyBase64(sseKey); + sseCHeader.setAlgorithm( + com.obs.services.model.ServerAlgorithm.AES256); + } else if (sseType.equalsIgnoreCase(SSE_KMS)) { + sseKmsHeader = new SseKmsHeader(); + sseKmsHeader.setEncryption( + com.obs.services.model.ServerEncryption.OBS_KMS); + sseKmsHeader.setKmsKeyId(sseKey); + } + } + } + + boolean isSseCEnable() { + return sseCHeader != null; + } + + boolean isSseKmsEnable() { + return sseKmsHeader != null; + } + + SseCHeader getSseCHeader() { + return sseCHeader; + } + + SseKmsHeader getSseKmsHeader() { + return sseKmsHeader; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/package-info.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/package-info.java new file mode 100644 index 00000000000..9e198d32057 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/package-info.java @@ -0,0 +1,29 @@ +/* + * 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 for supporting + * HuaweiCloud + * Object Storage Service (OBS) as a backend filesystem in Hadoop. + *

+ * OBS supports two kinds of buckets: object bucket and posix bucket. Posix + * bucket provides more POSIX-like semantics than object bucket, and is + * recommended for Hadoop. Object bucket is deprecated for Hadoop. + */ + +package org.apache.hadoop.fs.obs; diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem new file mode 100644 index 00000000000..e77425ab529 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.fs.FileSystem @@ -0,0 +1,16 @@ +# Licensed to the Apache Software Foundation (ASF) under one or more +# contributor license agreements. See the NOTICE file distributed with +# this work for additional information regarding copyright ownership. +# The ASF licenses this file to You under the Apache License, Version 2.0 +# (the "License"); you may not use this file except in compliance with +# the License. You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. + +org.apache.hadoop.fs.obs.OBSFileSystem diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/markdown/index.md b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/markdown/index.md new file mode 100644 index 00000000000..723da89e2be --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/markdown/index.md @@ -0,0 +1,370 @@ + + +# OBSA: HuaweiCloud OBS Adapter for Hadoop Support + + + +## Introduction + +The `hadoop-huaweicloud` module provides support for integration with the +[HuaweiCloud Object Storage Service (OBS)](https://www.huaweicloud.com/en-us/product/obs.html). +This support comes via the JAR file `hadoop-huaweicloud.jar`. + +## Features + +* Read and write data stored in a HuaweiCloud OBS account. +* Reference file system paths using URLs using the `obs` scheme. +* Present a hierarchical file system view by implementing the standard Hadoop `FileSystem` interface. +* Support multipart upload for a large file. +* Can act as a source of data in a MapReduce job, or a sink. +* Uses HuaweiCloud OBS’s Java SDK with support for latest OBS features and authentication schemes. +* Tested for scale. + +## Limitations + +Partial or no support for the following operations : + +* Symbolic link operations. +* Proxy users. +* File truncate. +* File concat. +* File checksum. +* File replication factor. +* Extended Attributes(XAttrs) operations. +* Snapshot operations. +* Storage policy. +* Quota. +* POSIX ACL. +* Delegation token operations. + +## Getting Started + +### Packages + +OBSA depends upon two JARs, alongside `hadoop-common` and its dependencies. + +* `hadoop-huaweicloud` JAR. +* `esdk-obs-java` JAR. + +The versions of `hadoop-common` and `hadoop-huaweicloud` must be identical. + +To import the libraries into a Maven build, add `hadoop-huaweicloud` JAR to the +build dependencies; it will pull in a compatible `esdk-obs-java` JAR. + +The `hadoop-huaweicloud` JAR *does not* declare any dependencies other than that +dependencies unique to it, the OBS SDK JAR. This is simplify excluding/tuning +Hadoop dependency JARs in downstream applications. The `hadoop-client` or +`hadoop-common` dependency must be declared. + + +```xml + + + 3.4.0 + + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + org.apache.hadoop + hadoop-huaweicloud + ${hadoop.version} + + +``` +### Accessing OBS URLs +Before access a URL, OBS implementation classes of Filesystem/AbstractFileSystem and +a region endpoint where a bucket is located shoud be configured as follows: +```xml + + fs.obs.impl + org.apache.hadoop.fs.obs.OBSFileSystem + The OBS implementation class of the Filesystem. + + + + fs.AbstractFileSystem.obs.impl + org.apache.hadoop.fs.obs.OBS + The OBS implementation class of the AbstractFileSystem. + + + + fs.obs.endpoint + obs.region.myhuaweicloud.com + OBS region endpoint where a bucket is located. + +``` + +OBS URLs can then be accessed as follows: + +``` +obs:///path +``` +The scheme `obs` identifies a URL on a Hadoop-compatible file system `OBSFileSystem` +backed by HuaweiCloud OBS. +For example, the following +[FileSystem Shell](../hadoop-project-dist/hadoop-common/FileSystemShell.html) +commands demonstrate access to a bucket named `mybucket`. +```bash +hadoop fs -mkdir obs://mybucket/testDir + +hadoop fs -put testFile obs://mybucket/testDir/testFile + +hadoop fs -cat obs://mybucket/testDir/testFile +test file content +``` + +For details on how to create a bucket, see +[**Help Center > Object Storage Service > Getting Started> Basic Operation Procedure**](https://support.huaweicloud.com/intl/en-us/qs-obs/obs_qs_0003.html) + +### Authenticating with OBS +Except when interacting with public OBS buckets, the OBSA client +needs the credentials needed to interact with buckets. +The client supports multiple authentication mechanisms. The simplest authentication mechanisms is +to provide OBS access key and secret key as follows. +```xml + + fs.obs.access.key + OBS access key. + Omit for provider-based authentication. + + + + fs.obs.secret.key + OBS secret key. + Omit for provider-based authentication. + +``` + +**Do not share access key, secret key, and session token. They must be kept secret.** + +Custom implementations +of `com.obs.services.IObsCredentialsProvider` (see [**Creating an Instance of ObsClient**](https://support.huaweicloud.com/intl/en-us/sdk-java-devg-obs/en-us_topic_0142815570.html)) or +`org.apache.hadoop.fs.obs.BasicSessionCredential` may also be used for authentication. + +```xml + + fs.obs.security.provider + + Class name of security provider class which implements + com.obs.services.IObsCredentialsProvider, which will + be used to construct an OBS client instance as an input parameter. + + + + + fs.obs.credentials.provider + + lass nameCof credential provider class which implements + org.apache.hadoop.fs.obs.BasicSessionCredential, + which must override three APIs: getOBSAccessKeyId(), + getOBSSecretKey(), and getSessionToken(). + + +``` + +## General OBSA Client Configuration + +All OBSA client options are configured with options with the prefix `fs.obs.`. + +```xml + + fs.obs.connection.ssl.enabled + false + Enable or disable SSL connections to OBS. + + + + fs.obs.connection.maximum + 1000 + Maximum number of simultaneous connections to OBS. + + + + fs.obs.connection.establish.timeout + 120000 + Socket connection setup timeout in milliseconds. + + + + fs.obs.connection.timeout + 120000 + Socket connection timeout in milliseconds. + + + + fs.obs.idle.connection.time + 30000 + Socket idle connection time. + + + + fs.obs.max.idle.connections + 1000 + Maximum number of socket idle connections. + + + + fs.obs.socket.send.buffer + 256 * 1024 + Socket send buffer to be used in OBS SDK. Represented in bytes. + + + + fs.obs.socket.recv.buffer + 256 * 1024 + Socket receive buffer to be used in OBS SDK. Represented in bytes. + + + + fs.obs.threads.keepalivetime + 60 + Number of seconds a thread can be idle before being + terminated in thread pool. + + + + fs.obs.threads.max + 20 + Maximum number of concurrent active (part)uploads, + which each use a thread from thread pool. + + + + fs.obs.max.total.tasks + 20 + Number of (part)uploads allowed to the queue before + blocking additional uploads. + + + + fs.obs.delete.threads.max + 20 + Max number of delete threads. + + + + fs.obs.multipart.size + 104857600 + Part size for multipart upload. + + + + + fs.obs.multiobjectdelete.maximum + 1000 + Max number of objects in one multi-object delete call. + + + + + fs.obs.fast.upload.buffer + disk + Which buffer to use. Default is `disk`, value may be + `disk` | `array` | `bytebuffer`. + + + + + fs.obs.buffer.dir + dir1,dir2,dir3 + Comma separated list of directories that will be used to buffer file + uploads to. This option takes effect only when the option 'fs.obs.fast.upload.buffer' + is set to 'disk'. + + + + + fs.obs.fast.upload.active.blocks + 4 + Maximum number of blocks a single output stream can have active + (uploading, or queued to the central FileSystem instance's pool of queued + operations). + + + + + fs.obs.readahead.range + 1024 * 1024 + Bytes to read ahead during a seek() before closing and + re-opening the OBS HTTP connection. + + + + fs.obs.read.transform.enable + true + Flag indicating if socket connections can be reused by + position read. Set `false` only for HBase. + + + + fs.obs.list.threads.core + 30 + Number of core list threads. + + + + fs.obs.list.threads.max + 60 + Maximum number of list threads. + + + + fs.obs.list.workqueue.capacity + 1024 + Capacity of list work queue. + + + + fs.obs.list.parallel.factor + 30 + List parallel factor. + + + + fs.obs.trash.enable + false + Switch for the fast delete. + + + + fs.obs.trash.dir + The fast delete recycle directory. + + + + fs.obs.block.size + 128 * 1024 * 1024 + Default block size for OBS FileSystem. + + +``` + +## Testing the hadoop-huaweicloud Module +The `hadoop-huaweicloud` module includes a full suite of unit tests. +Most of the tests will run against the HuaweiCloud OBS. To run these +tests, please create `src/test/resources/auth-keys.xml` with OBS account +information mentioned in the above sections and the following properties. + +```xml + + fs.contract.test.fs.obs + obs://obsfilesystem-bucket + +``` \ No newline at end of file diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/resources/css/site.css b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/resources/css/site.css new file mode 100644 index 00000000000..7315db31e53 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/site/resources/css/site.css @@ -0,0 +1,29 @@ +/* +* 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. +*/ +#banner { + height: 93px; + background: none; +} + +#bannerLeft img { + margin-left: 30px; + margin-top: 10px; +} + +#bannerRight img { + margin: 17px; +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSContract.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSContract.java new file mode 100644 index 00000000000..ab9d6dae4cc --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSContract.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractBondedFSContract; + +/** + * The contract of OBS: only enabled if the test bucket is provided. + */ +public class OBSContract extends AbstractBondedFSContract { + + public static final String CONTRACT_XML = "contract/obs.xml"; + + private static final String CONTRACT_ENABLE_KEY = + "fs.obs.test.contract.enable"; + + private static final boolean CONTRACT_ENABLE_DEFAULT = false; + + public OBSContract(Configuration conf) { + super(conf); + //insert the base features + addConfResource(CONTRACT_XML); + } + + @Override + public String getScheme() { + return "obs"; + } + + @Override + public Path getTestPath() { + return OBSTestUtils.createTestPath(super.getTestPath()); + } + + public synchronized static boolean isContractTestEnabled() { + Configuration conf = null; + boolean isContractTestEnabled = true; + + if (conf == null) { + conf = getConfiguration(); + } + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + isContractTestEnabled = false; + } + return isContractTestEnabled; + } + + public synchronized static Configuration getConfiguration() { + Configuration newConf = new Configuration(); + newConf.addResource(CONTRACT_XML); + return newConf; + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java new file mode 100644 index 00000000000..4fcff35b9c9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java @@ -0,0 +1,40 @@ +/* + * 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.obs; + +/** + * Constants for OBS Testing. + */ + +final class OBSTestConstants { + + private OBSTestConstants(){ + } + + /** + * Name of the test filesystem. + */ + static final String TEST_FS_OBS_NAME = "fs.contract.test.fs.obs"; + + /** + * Fork ID passed down from maven if the test is running in parallel. + */ + static final String TEST_UNIQUE_FORK_ID = "test.unique.fork.id"; + +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java new file mode 100644 index 00000000000..9496617256a --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java @@ -0,0 +1,119 @@ +/* + * 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.obs; + +import org.apache.commons.lang3.StringUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.junit.internal.AssumptionViolatedException; + +import java.io.IOException; +import java.net.URI; + +import static org.apache.hadoop.fs.obs.OBSTestConstants.*; +import static org.apache.hadoop.fs.obs.OBSConstants.*; + +/** + * Utilities for the OBS tests. + */ +public final class OBSTestUtils { + + /** + * Create the test filesystem. + *

+ * If the test.fs.obs.name property is not set, this will trigger a JUnit + * failure. + *

+ * Multipart purging is enabled. + * + * @param conf configuration + * @return the FS + * @throws IOException IO Problems + * @throws AssumptionViolatedException if the FS is not named + */ + public static OBSFileSystem createTestFileSystem(Configuration conf) + throws IOException { + return createTestFileSystem(conf, false); + } + + /** + * Create the test filesystem with or without multipart purging + *

+ * If the test.fs.obs.name property is not set, this will trigger a JUnit + * failure. + * + * @param conf configuration + * @param purge flag to enable Multipart purging + * @return the FS + * @throws IOException IO Problems + * @throws AssumptionViolatedException if the FS is not named + */ + @SuppressWarnings("deprecation") + public static OBSFileSystem createTestFileSystem(Configuration conf, + boolean purge) + throws IOException { + + String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, ""); + + boolean liveTest = !StringUtils.isEmpty(fsname); + URI testURI = null; + if (liveTest) { + testURI = URI.create(fsname); + liveTest = testURI.getScheme().equals(OBSConstants.OBS_SCHEME); + } + if (!liveTest) { + // This doesn't work with our JUnit 3 style test cases, so instead we'll + // make this whole class not run by default + throw new AssumptionViolatedException( + "No test filesystem in " + TEST_FS_OBS_NAME); + } + OBSFileSystem fs1 = new OBSFileSystem(); + //enable purging in tests + if (purge) { + conf.setBoolean(PURGE_EXISTING_MULTIPART, true); + // but a long delay so that parallel multipart tests don't + // suddenly start timing out + conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 30 * 60); + } + fs1.initialize(testURI, conf); + return fs1; + } + + /** + * Create a test path, using the value of + * {@link OBSTestConstants#TEST_UNIQUE_FORK_ID} + * if it is set. + * + * @param defVal default value + * @return a path + */ + public static Path createTestPath(Path defVal) { + String testUniqueForkId = System.getProperty( + OBSTestConstants.TEST_UNIQUE_FORK_ID); + return testUniqueForkId == null ? defVal : + new Path("/" + testUniqueForkId, "test"); + } + + /** + * This class should not be instantiated. + */ + private OBSTestUtils() { + } + +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractAppend.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractAppend.java new file mode 100644 index 00000000000..a4fb8153e7c --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractAppend.java @@ -0,0 +1,40 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractAppendTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.junit.Assume; + +/** + * Append test cases on obs file system. + */ +public class TestOBSContractAppend extends AbstractContractAppendTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @Override + public void testRenameFileBeingAppended() { + Assume.assumeTrue("unsupport.", false); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractCreate.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractCreate.java new file mode 100644 index 00000000000..d3966a13b95 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractCreate.java @@ -0,0 +1,45 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractCreateTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.junit.Assume; + +/** + * Create test cases on obs file system. + */ +public class TestOBSContractCreate extends AbstractContractCreateTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @Override + public void testCreatedFileIsImmediatelyVisible() { + Assume.assumeTrue("unsupport.", false); + } + + @Override + public void testCreatedFileIsVisibleOnFlush() { + Assume.assumeTrue("unsupport", false); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractDelete.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractDelete.java new file mode 100644 index 00000000000..9dd67ad779b --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractDelete.java @@ -0,0 +1,34 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Delete test cases on obs file system. + */ +public class TestOBSContractDelete extends AbstractContractDeleteTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractGetFileStatus.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractGetFileStatus.java new file mode 100644 index 00000000000..15ffd97e090 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractGetFileStatus.java @@ -0,0 +1,36 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Get file status test cases on obs file system. + */ +public class TestOBSContractGetFileStatus extends + AbstractContractGetFileStatusTest { + + @Override + protected AbstractFSContract createContract( + final Configuration conf) { + return new OBSContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractMkdir.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractMkdir.java new file mode 100644 index 00000000000..e06ad860e21 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractMkdir.java @@ -0,0 +1,34 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractMkdirTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Mkdir test cases on obs file system. + */ +public class TestOBSContractMkdir extends AbstractContractMkdirTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractOpen.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractOpen.java new file mode 100644 index 00000000000..c8641dfd627 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractOpen.java @@ -0,0 +1,34 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractOpenTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Open test cases on obs file system. + */ +public class TestOBSContractOpen extends AbstractContractOpenTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRename.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRename.java new file mode 100644 index 00000000000..25502a23f27 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRename.java @@ -0,0 +1,45 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractRenameTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.junit.Assume; + +/** + * Rename test cases on obs file system. + */ +public class TestOBSContractRename extends AbstractContractRenameTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @Override + public void testRenameFileUnderFileSubdir() { + Assume.assumeTrue("unsupport.", false); + } + + @Override + public void testRenameFileUnderFile() { + Assume.assumeTrue("unsupport.", false); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRootDir.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRootDir.java new file mode 100644 index 00000000000..ba961a300ef --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractRootDir.java @@ -0,0 +1,34 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Root directory test cases on obs file system. + */ +public class TestOBSContractRootDir extends AbstractContractRootDirectoryTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractSeek.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractSeek.java new file mode 100644 index 00000000000..48751ea6696 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSContractSeek.java @@ -0,0 +1,34 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractSeekTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; + +/** + * Seek test cases on obs file system. + */ +public class TestOBSContractSeek extends AbstractContractSeekTest { + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java new file mode 100644 index 00000000000..b62023b6424 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java @@ -0,0 +1,93 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.TestFSMainOperationsLocalFileSystem; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; + +/** + *

+ * A collection of tests for the {@link FileSystem}. This test should be used + * for testing an instance of FileSystem that has been initialized to a specific + * default FileSystem such a LocalFileSystem, HDFS,OBS, etc. + *

+ *

+ * To test a given {@link FileSystem} implementation create a subclass of this + * test and override {@link #setUp()} to initialize the fSys {@link + * FileSystem} instance variable. + *

+ * Since this a junit 4 you can also do a single setup before the start of any + * tests. E.g. + * + * + *

+ */ +public class TestOBSFSMainOperations extends + TestFSMainOperationsLocalFileSystem { + + @Override + @Before + public void setUp() throws Exception { + skipTestCheck(); + Configuration conf = new Configuration(); + conf.addResource(OBSContract.CONTRACT_XML); + fSys = OBSTestUtils.createTestFileSystem(conf); + } + + @Override + public void testWorkingDirectory() { + Assume.assumeTrue("unspport.", false); + } + + @Override + public void testListStatusThrowsExceptionForUnreadableDir() { + Assume.assumeTrue("unspport.", false); + } + + @Override + public void testRenameDirectoryToItself() { + Assume.assumeTrue("unspport.", false); + } + + @Override + public void testGlobStatusThrowsExceptionForUnreadableDir() { + Assume.assumeTrue("unspport.", false); + } + + @Override + public void testRenameFileToItself() { + Assume.assumeTrue("unspport.", false); + } + + @Override + @After + public void tearDown() throws Exception { + if(fSys != null) { + super.tearDown(); + } + } + + public void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java new file mode 100644 index 00000000000..7860f356aa3 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java @@ -0,0 +1,75 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest; +import org.apache.hadoop.fs.FileContextTestHelper; +import org.apache.hadoop.fs.FileSystem; +import org.junit.Assume; +import org.junit.BeforeClass; + +import java.net.URI; +import java.util.UUID; + +import edu.umd.cs.findbugs.annotations.SuppressFBWarnings; + +/** + * File context create mkdir test cases on obs file system. + */ +public class TestOBSFileContextCreateMkdir extends + FileContextCreateMkdirBaseTest { + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + + @SuppressFBWarnings("ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD") + @Override + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(); + conf.addResource(OBSContract.CONTRACT_XML); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + FileSystem fs = OBSTestUtils.createTestFileSystem(conf); + if (fc == null) { + this.fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + } + super.setUp(); + } + + @Override + protected FileContextTestHelper createFileContextHelper() { + // On Windows, root directory path is created from local running + // directory. + // obs does not support ':' as part of the path which results in + // failure. + return new FileContextTestHelper(UUID.randomUUID().toString()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java new file mode 100644 index 00000000000..ef6d31215f7 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java @@ -0,0 +1,77 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileContextMainOperationsBaseTest; +import org.apache.hadoop.fs.FileSystem; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.net.URI; + +/** + * Rename test cases on obs file system. + */ +public class TestOBSFileContextMainOperations extends + FileContextMainOperationsBaseTest { + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @edu.umd.cs.findbugs.annotations.SuppressFBWarnings( + "ST_WRITE_TO_STATIC_FROM_INSTANCE_METHOD") + @Override + public void setUp() throws Exception { + Configuration conf = new Configuration(); + conf.addResource(OBSContract.CONTRACT_XML); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + FileSystem fs = OBSTestUtils.createTestFileSystem(conf); + fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + super.setUp(); + } + + @Override + protected boolean listCorruptedBlocksSupported() { + return false; + } + + @Override + @Test + public void testSetVerifyChecksum() { + Assume.assumeTrue("unsupport.", false); + } + + @Override + public void testMkdirsFailsForSubdirectoryOfExistingFile() { + Assume.assumeTrue("unsupport.", false); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java new file mode 100644 index 00000000000..b3f523092a9 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileContextURIBase; +import org.apache.hadoop.fs.FileSystem; +import org.junit.Assume; +import org.junit.BeforeClass; + +import java.net.URI; + +/** + *

+ * A collection of tests for the {@link FileContext} to test path names passed + * as URIs. This test should be used for testing an instance of FileContext that + * has been initialized to a specific default FileSystem such a LocalFileSystem, + * HDFS,OBS, etc, and where path names are passed that are URIs in a different + * FileSystem. + *

+ * + *

+ * To test a given {@link FileSystem} implementation create a subclass of this + * test and override {@link #setUp()} to initialize the fc1 and + * fc2 + *

+ * The tests will do operations on fc1 that use a URI in fc2 + *

+ * {@link FileContext} instance variable. + *

+ */ +public class TestOBSFileContextURI extends FileContextURIBase { + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Override + public void setUp() throws Exception { + Configuration conf = new Configuration(); + conf.addResource(OBSContract.CONTRACT_XML); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + FileSystem fs = OBSTestUtils.createTestFileSystem(conf); + fc1 = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + + fc2 = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + super.setUp(); + } + + @Override + public void testMkdirsFailsForSubdirectoryOfExistingFile() { + Assume.assumeTrue("unsupport.", false); + } + + @Override + public void testFileStatus() { + Assume.assumeTrue("unsupport.", false); + } + +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextUtil.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextUtil.java new file mode 100644 index 00000000000..1404e06a452 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextUtil.java @@ -0,0 +1,68 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileContextUtilBase; +import org.apache.hadoop.fs.FileSystem; +import org.junit.Assume; +import org.junit.BeforeClass; + +import java.net.URI; + +/** + *

+ * A collection of Util tests for the {@link FileContext#util()}. This test + * should be used for testing an instance of {@link FileContext#util()} that has + * been initialized to a specific default FileSystem such a LocalFileSystem, + * HDFS,OBS, etc. + *

+ *

+ * To test a given {@link FileSystem} implementation create a subclass of this + * test and override {@link #setUp()} to initialize the fc {@link + * FileContext} instance variable. + * + *

+ */ +public class TestOBSFileContextUtil extends FileContextUtilBase { + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Override + public void setUp() throws Exception { + Configuration conf = new Configuration(); + conf.addResource(OBSContract.CONTRACT_XML); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + FileSystem fs = OBSTestUtils.createTestFileSystem(conf); + fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + super.setUp(); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java new file mode 100644 index 00000000000..defd3ba40f2 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java @@ -0,0 +1,59 @@ +/* + * 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.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystemContractBaseTest; +import org.junit.Assume; +import org.junit.Before; + + +/** + * Tests a live OBS system. If your keys and bucket aren't specified, all tests + * are marked as passed. + *

+ * This uses BlockJUnit4ClassRunner because FileSystemContractBaseTest from + * TestCase which uses the old Junit3 runner that doesn't ignore assumptions + * properly making it impossible to skip the tests if we don't have a valid + * bucket. + **/ +public class TestOBSFileSystemContract extends FileSystemContractBaseTest { + + @Before + public void setUp() throws Exception { + skipTestCheck(); + Configuration conf = new Configuration(); + conf.addResource(OBSContract.CONTRACT_XML); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @Override + public void testMkdirsWithUmask() { + Assume.assumeTrue("unspport.", false); + } + + @Override + public void testRenameRootDirForbidden() { + Assume.assumeTrue("unspport.", false); + } + + public void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } +} diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/contract/obs.xml b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/contract/obs.xml new file mode 100644 index 00000000000..30b2cf04234 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/contract/obs.xml @@ -0,0 +1,139 @@ + + + + + fs.contract.test.root-tests-enabled + true + + + + fs.contract.test.supports-concat + true + + + + fs.contract.rename-returns-false-if-source-missing + true + + + + fs.contract.test.random-seek-count + 10 + + + + fs.contract.is-case-sensitive + true + + + + fs.contract.rename-returns-true-if-dest-exists + false + + + + fs.contract.rename-returns-true-if-source-missing + false + + + + fs.contract.rename-creates-dest-dirs + false + + + + fs.contract.rename-remove-dest-if-empty-dir + false + + + + fs.contract.supports-settimes + true + + + + fs.contract.supports-append + true + + + + fs.contract.supports-atomic-directory-delete + true + + + + fs.contract.supports-atomic-rename + true + + + + fs.contract.supports-block-locality + true + + + + fs.contract.supports-concat + true + + + + fs.contract.supports-seek + true + + + + fs.contract.supports-seek-on-closed-file + true + + + + fs.contract.rejects-seek-past-eof + true + + + + fs.contract.supports-available-on-closed-file + true + + + + fs.contract.supports-strict-exceptions + false + + + + fs.contract.supports-unix-permissions + true + + + + fs.contract.rename-overwrites-dest + false + + + + fs.contract.supports-append + true + + + + fs.contract.supports-getfilestatus + true + + diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/core-site.xml b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/core-site.xml new file mode 100644 index 00000000000..2058293646e --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/core-site.xml @@ -0,0 +1,136 @@ + + + + + + + + + hadoop.tmp.dir + target/build/test + A base for other temporary directories. + true + + + + hadoop.security.authentication + simple + + + + fs.obs.impl + org.apache.hadoop.fs.obs.OBSFileSystem + The implementation class of the obs Filesystem + + + fs.obs.connection.establish.timeout + 60000 + + + fs.obs.connection.timeout + 60000 + + + fs.obs.idle.connection.time + 30000 + + + fs.obs.max.idle.connections + 10 + + + fs.obs.connection.maximum + 1000 + + + fs.obs.attempts.maximum + 5 + + + fs.obs.upload.stream.retry.buffer.size + 524288 + + + fs.obs.read.buffer.size + 8192 + + + fs.obs.write.buffer.size + 8192 + + + fs.obs.socket.recv.buffer + -1 + + + fs.obs.socket.send.buffer + -1 + + + fs.obs.keep.alive + true + + + fs.obs.validate.certificate + false + + + fs.obs.verify.response.content.type + true + + + fs.obs.strict.hostname.verification + false + + + fs.obs.cname + false + + + + fs.obs.test.local.path + /uplod_file + + + + fs.obs.fast.upload + true + + + fs.obs.multipart.size + 10485760 + + + fs.obs.experimental.input.fadvise + random + + + + + + + + diff --git a/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/log4j.properties b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/log4j.properties new file mode 100644 index 00000000000..6c0829f4ee6 --- /dev/null +++ b/hadoop-cloud-storage-project/hadoop-huaweicloud/src/test/resources/log4j.properties @@ -0,0 +1,23 @@ +# Licensed 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. +# log4j configuration used during build and unit tests + +log4j.rootLogger=error,stdout +log4j.threshold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n + +log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR + +# for debugging low level obs operations, uncomment this line +log4j.logger.org.apache.hadoop.fs.obs=ERROR diff --git a/hadoop-cloud-storage-project/pom.xml b/hadoop-cloud-storage-project/pom.xml index da0d88a8117..8df6bb41e90 100644 --- a/hadoop-cloud-storage-project/pom.xml +++ b/hadoop-cloud-storage-project/pom.xml @@ -32,6 +32,7 @@ hadoop-cloud-storage hadoop-cos + hadoop-huaweicloud diff --git a/hadoop-project/pom.xml b/hadoop-project/pom.xml index f3fa47505aa..ff19abce837 100644 --- a/hadoop-project/pom.xml +++ b/hadoop-project/pom.xml @@ -676,6 +676,12 @@ ${hadoop.version} + + org.apache.hadoop + hadoop-huaweicloud + ${hadoop.version} + + org.apache.hadoop hadoop-kms diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index 86949b0404c..50f75873bf8 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -179,6 +179,7 @@ href="hadoop-azure-datalake/index.html"/> +