HADOOP-16492. Support HuaweiCloud Object Storage as a Hadoop Backend File System. Contributed by zhongjun.
This commit is contained in:
parent
ec22850dbe
commit
1ec5c67b5e
@ -133,5 +133,10 @@
|
||||
<artifactId>hadoop-cos</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-huaweicloud</artifactId>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
||||
|
@ -0,0 +1,18 @@
|
||||
<!--
|
||||
Licensed to the Apache Software Foundation (ASF) under one or more
|
||||
contributor license agreements. See the NOTICE file distributed with
|
||||
this work for additional information regarding copyright ownership.
|
||||
The ASF licenses this file to You under the Apache License, Version 2.0
|
||||
(the "License"); you may not use this file except in compliance with
|
||||
the License. You may obtain a copy of the License at
|
||||
|
||||
http://www.apache.org/licenses/LICENSE-2.0
|
||||
|
||||
Unless required by applicable law or agreed to in writing, software
|
||||
distributed under the License is distributed on an "AS IS" BASIS,
|
||||
WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
|
||||
See the License for the specific language governing permissions and
|
||||
limitations under the License.
|
||||
-->
|
||||
<FindBugsFilter>
|
||||
</FindBugsFilter>
|
180
hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml
Executable file
180
hadoop-cloud-storage-project/hadoop-huaweicloud/pom.xml
Executable file
@ -0,0 +1,180 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<!--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. See accompanying LICENSE file.-->
|
||||
<project xmlns="http://maven.apache.org/POM/4.0.0"
|
||||
xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
|
||||
xsi:schemaLocation="http://maven.apache.org/POM/4.0.0 http://maven.apache.org/xsd/maven-4.0.0.xsd">
|
||||
<modelVersion>4.0.0</modelVersion>
|
||||
<parent>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-project</artifactId>
|
||||
<version>3.4.0-SNAPSHOT</version>
|
||||
<relativePath>../../hadoop-project</relativePath>
|
||||
</parent>
|
||||
<artifactId>hadoop-huaweicloud</artifactId>
|
||||
<version>3.4.0-SNAPSHOT</version>
|
||||
<name>Apache Hadoop OBS support</name>
|
||||
<description>
|
||||
This module contains code to support integration with OBS.
|
||||
It also declares the dependencies needed to work with OBS services.
|
||||
</description>
|
||||
<packaging>jar</packaging>
|
||||
<properties>
|
||||
<file.encoding>UTF-8</file.encoding>
|
||||
<downloadSources>true</downloadSources>
|
||||
<esdk.version>3.20.4.2</esdk.version>
|
||||
</properties>
|
||||
|
||||
<profiles>
|
||||
<profile>
|
||||
<id>tests-off</id>
|
||||
<activation>
|
||||
<file>
|
||||
<missing>src/test/resources/auth-keys.xml</missing>
|
||||
</file>
|
||||
</activation>
|
||||
<properties>
|
||||
<maven.test.skip>true</maven.test.skip>
|
||||
</properties>
|
||||
</profile>
|
||||
<profile>
|
||||
<id>tests-on</id>
|
||||
<activation>
|
||||
<file>
|
||||
<exists>src/test/resources/auth-keys.xml</exists>
|
||||
</file>
|
||||
</activation>
|
||||
<properties>
|
||||
<maven.test.skip>false</maven.test.skip>
|
||||
</properties>
|
||||
</profile>
|
||||
</profiles>
|
||||
|
||||
<build>
|
||||
<plugins>
|
||||
<plugin>
|
||||
<groupId>org.codehaus.mojo</groupId>
|
||||
<artifactId>findbugs-maven-plugin</artifactId>
|
||||
<configuration>
|
||||
<findbugsXmlOutput>true</findbugsXmlOutput>
|
||||
<xmlOutput>true</xmlOutput>
|
||||
<excludeFilterFile>${basedir}/dev-support/findbugs-exclude.xml
|
||||
</excludeFilterFile>
|
||||
<effort>Max</effort>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-surefire-plugin</artifactId>
|
||||
<configuration>
|
||||
<forkedProcessTimeoutInSeconds>3600</forkedProcessTimeoutInSeconds>
|
||||
</configuration>
|
||||
</plugin>
|
||||
<plugin>
|
||||
<groupId>org.apache.maven.plugins</groupId>
|
||||
<artifactId>maven-dependency-plugin</artifactId>
|
||||
<executions>
|
||||
<execution>
|
||||
<id>deplist</id>
|
||||
<phase>compile</phase>
|
||||
<goals>
|
||||
<goal>list</goal>
|
||||
</goals>
|
||||
<configuration>
|
||||
<outputFile>${project.basedir}/target/hadoop-cloud-storage-deps/${project.artifactId}.cloud-storage-optional.txt</outputFile>
|
||||
</configuration>
|
||||
</execution>
|
||||
</executions>
|
||||
</plugin>
|
||||
</plugins>
|
||||
</build>
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<scope>provided</scope>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<groupId>jdk.tools</groupId>
|
||||
<artifactId>jdk.tools</artifactId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<version>4.12</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.mockito</groupId>
|
||||
<artifactId>mockito-all</artifactId>
|
||||
<version>1.10.19</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-client-jobclient</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-yarn-server-tests</artifactId>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-mapreduce-examples</artifactId>
|
||||
<scope>test</scope>
|
||||
<type>jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-distcp</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-distcp</artifactId>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>com.huaweicloud</groupId>
|
||||
<artifactId>esdk-obs-java</artifactId>
|
||||
<version>${esdk.version}</version>
|
||||
<exclusions>
|
||||
<exclusion>
|
||||
<artifactId>okio</artifactId>
|
||||
<groupId>com.squareup.okio</groupId>
|
||||
</exclusion>
|
||||
</exclusions>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.powermock</groupId>
|
||||
<artifactId>powermock-api-mockito</artifactId>
|
||||
<version>1.7.4</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.powermock</groupId>
|
||||
<artifactId>powermock-module-junit4</artifactId>
|
||||
<version>1.7.4</version>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
</project>
|
@ -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();
|
||||
}
|
@ -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<Constructor> 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<Constructor> 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);
|
||||
}
|
||||
}
|
@ -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 <code>FileConflictException</code> with the specified detail
|
||||
* message. The string <code>s</code> can be retrieved later by the
|
||||
* <code>{@link Throwable#getMessage}</code>
|
||||
* method of class <code>java.lang.Throwable</code>.
|
||||
*
|
||||
* @param s the detail message.
|
||||
*/
|
||||
FileConflictException(final String s) {
|
||||
super(s);
|
||||
}
|
||||
}
|
@ -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 <code>theUri</code> 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;
|
||||
}
|
||||
}
|
@ -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.
|
||||
* <p>
|
||||
* Upload files/parts directly via different buffering mechanisms: including
|
||||
* memory and disk.
|
||||
*
|
||||
* <p>If the stream is closed and no update has started, then the upload is
|
||||
* instead done as a single PUT operation.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>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<PartEtag> 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<ListenableFuture<PartEtag>> 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<PartEtag> 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<PartEtag> 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<PartEtag> 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<PartEtag> 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<PartEtag> 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);
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -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}.
|
||||
*
|
||||
* <p>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}
|
||||
*
|
||||
* <p>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<? extends OBSClientFactory>
|
||||
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() {
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -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.
|
||||
*
|
||||
* <p>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);
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -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<LevelStats> 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<ListEntity> listStack,
|
||||
final Queue<ListEntity> resultQueue,
|
||||
final String marker,
|
||||
final int maxKeyNum,
|
||||
final List<ObsObject> objectSummaries,
|
||||
final List<LevelStats> 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<ListEntity> listStack,
|
||||
final Queue<ListEntity> resultQueue, final int maxKeyNum,
|
||||
final List<ObsObject> objectSummaries,
|
||||
final List<LevelStats> levelStatsList,
|
||||
final int resultNum) throws IOException {
|
||||
int newResultNum = resultNum;
|
||||
while (!listStack.empty() && (newResultNum < maxKeyNum
|
||||
|| resultQueue.isEmpty())) {
|
||||
List<ListObjectsRequest> oneLevelListRequests = new ArrayList<>();
|
||||
List<Future<ObjectListing>> oneLevelListFutures = new ArrayList<>();
|
||||
List<Integer> levels = new ArrayList<>();
|
||||
List<ObjectListing> 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<ListEntity> resultQueue,
|
||||
final int maxKeyNum,
|
||||
final List<ObsObject> objectSummaries,
|
||||
final List<LevelStats> levelStatsList,
|
||||
final int resultNum,
|
||||
final List<ListObjectsRequest> oneLevelListRequests,
|
||||
final List<Integer> levels,
|
||||
final List<ObjectListing> 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<ListObjectsRequest> oneLevelListRequests,
|
||||
final List<Future<ObjectListing>> oneLevelListFutures,
|
||||
final List<ObjectListing> 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<ObjectListing> 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<ListEntity> listStack, final int maxKeyNum,
|
||||
final List<ListObjectsRequest> oneLevelListRequests,
|
||||
final List<Future<ObjectListing>> oneLevelListFutures,
|
||||
final List<Integer> 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<ListEntity> listStack,
|
||||
final List<ListObjectsRequest> oneLevelListRequests,
|
||||
final List<Integer> levels,
|
||||
final List<ObjectListing> 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<String> 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<ObsObject> 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<ListEntity> resultQueue, final int maxKeyNum,
|
||||
final List<ObsObject> objectSummaries,
|
||||
final List<LevelStats> 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<ObsObject> 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<ObsObject> objectSummaries = new ArrayList<>();
|
||||
List<String> 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<ListEntity> listStack = new Stack<>();
|
||||
Queue<ListEntity> resultQueue = new LinkedList<>();
|
||||
List<LevelStats> 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<ObsObject> objectSummaries = new ArrayList<>();
|
||||
List<String> 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<ListEntity> listStack = obsFsDFSListing.getListStack();
|
||||
Queue<ListEntity> resultQueue = obsFsDFSListing.getResultQueue();
|
||||
List<LevelStats> 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<ListEntity> listStack;
|
||||
|
||||
/**
|
||||
* Queue of entity list.
|
||||
*/
|
||||
private Queue<ListEntity> resultQueue;
|
||||
|
||||
/**
|
||||
* List of levelStats.
|
||||
*/
|
||||
private List<LevelStats> levelStatsList;
|
||||
|
||||
OBSFsDFSListing(final ListObjectsRequest request,
|
||||
final List<ObsObject> objectSummaries,
|
||||
final List<String> commonPrefixes,
|
||||
final String nextMarker,
|
||||
final Stack<ListEntity> listEntityStack,
|
||||
final Queue<ListEntity> listEntityQueue,
|
||||
final List<LevelStats> 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<ObsObject> objectSummaries,
|
||||
final List<String> commonPrefixes,
|
||||
final String nextMarker,
|
||||
final Stack<ListEntity> listEntityStack,
|
||||
final Queue<ListEntity> listEntityQueue,
|
||||
final List<LevelStats> 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<ListEntity> getListStack() {
|
||||
return listStack;
|
||||
}
|
||||
|
||||
Queue<ListEntity> getResultQueue() {
|
||||
return resultQueue;
|
||||
}
|
||||
|
||||
List<LevelStats> getLevelStatsList() {
|
||||
return levelStatsList;
|
||||
}
|
||||
}
|
@ -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();
|
||||
}
|
||||
}
|
File diff suppressed because it is too large
Load Diff
@ -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<FileStatus> 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.
|
||||
*
|
||||
* <p>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<LocatedFileStatus> {
|
||||
|
||||
/**
|
||||
* 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.
|
||||
*
|
||||
* <p>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
|
||||
* <i>and those results filtered</i> before the iterator can declare that
|
||||
* there is more data available.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>Thread safety: None.
|
||||
*/
|
||||
class FileStatusListingIterator implements RemoteIterator<FileStatus> {
|
||||
|
||||
/**
|
||||
* 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<FileStatus> 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<FileStatus> 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.
|
||||
*
|
||||
* <p>That is:
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>2. Second and later invocations will continue the ongoing listing,
|
||||
* calling {@link OBSCommonUtils#continueListObjects} to request the next
|
||||
* batch of results.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>Thread safety: none.
|
||||
*/
|
||||
class ObjectListingIterator implements RemoteIterator<ObjectListing> {
|
||||
|
||||
/**
|
||||
* 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<LocatedFileStatus> {
|
||||
/**
|
||||
* File status.
|
||||
*/
|
||||
private final RemoteIterator<FileStatus> statusIterator;
|
||||
|
||||
/**
|
||||
* Constructor.
|
||||
*
|
||||
* @param statusRemoteIterator an iterator over the remote status entries
|
||||
*/
|
||||
LocatedFileStatusIterator(
|
||||
final RemoteIterator<FileStatus> 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());
|
||||
}
|
||||
}
|
||||
}
|
@ -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.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>Essentially
|
||||
*
|
||||
* <ol>
|
||||
* <li>The URI is canonicalized.
|
||||
* <li>If the schemas match, the hosts are compared.
|
||||
* <li>If there is a mismatch between null/non-null host,
|
||||
* the default FS values are used to patch in the host.
|
||||
* </ol>
|
||||
* <p>
|
||||
* 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;
|
||||
}
|
||||
}
|
||||
}
|
@ -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<KeyAndVersion> 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<Future<CopyObjectResult>> 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<Future<CopyObjectResult>> copyFutures)
|
||||
throws IOException {
|
||||
try {
|
||||
for (Future<CopyObjectResult> 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<CopyObjectResult> 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<KeyAndVersion> 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<PartEtag> 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<PartEtag> getCopyFilePartEtags(final OBSFileSystem owner,
|
||||
final String srcKey,
|
||||
final String dstKey,
|
||||
final long objectSize,
|
||||
final String uploadId,
|
||||
final long partCount)
|
||||
throws IOException {
|
||||
final List<PartEtag> partEtags = Collections.synchronizedList(
|
||||
new ArrayList<>());
|
||||
final List<Future<?>> 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<PartEtag> partEtags,
|
||||
final List<Future<?>> 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<String> 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<String> 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<CopyObjectResult> copyFileAsync(
|
||||
final OBSFileSystem owner,
|
||||
final String srcKey,
|
||||
final String dstKey, final long size) {
|
||||
return owner.getBoundedCopyThreadPool().submit(() -> {
|
||||
copyFile(owner, srcKey, dstKey, size);
|
||||
return null;
|
||||
});
|
||||
}
|
||||
}
|
@ -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<KeyAndVersion> 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<KeyAndVersion> subdirList = new ArrayList<>(
|
||||
owner.getMaxEntriesToDelete());
|
||||
List<KeyAndVersion> 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<KeyAndVersion> 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<KeyAndVersion> 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();
|
||||
}
|
||||
}
|
@ -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.
|
||||
*
|
||||
* <p>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.
|
||||
*
|
||||
* <p>Features
|
||||
*
|
||||
* <ul>
|
||||
* <li>Methods to create and submit requests to OBS, so avoiding all direct
|
||||
* interaction with the OBS APIs.
|
||||
* <li>Some extra preflight checks of arguments, so failing fast on errors.
|
||||
* <li>Callbacks to let the FS know of events in the output stream upload
|
||||
* process.
|
||||
* </ul>
|
||||
* <p>
|
||||
* 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<PartEtag> 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);
|
||||
}
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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
|
||||
* <a href="https://www.huaweicloud.com/en-us/product/obs.html">HuaweiCloud
|
||||
* Object Storage Service (OBS)</a> as a backend filesystem in Hadoop.
|
||||
* <p>
|
||||
* 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;
|
@ -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
|
@ -0,0 +1,370 @@
|
||||
<!---
|
||||
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. See accompanying LICENSE file.
|
||||
-->
|
||||
|
||||
# OBSA: HuaweiCloud OBS Adapter for Hadoop Support
|
||||
|
||||
<!-- MACRO{toc|fromDepth=1|toDepth=3} -->
|
||||
|
||||
## 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
|
||||
<properties>
|
||||
<!-- Your exact Hadoop version here-->
|
||||
<hadoop.version>3.4.0</hadoop.version>
|
||||
</properties>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-client</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-huaweicloud</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
</dependencies>
|
||||
```
|
||||
### 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
|
||||
<property>
|
||||
<name>fs.obs.impl</name>
|
||||
<value>org.apache.hadoop.fs.obs.OBSFileSystem</value>
|
||||
<description>The OBS implementation class of the Filesystem.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.AbstractFileSystem.obs.impl</name>
|
||||
<value>org.apache.hadoop.fs.obs.OBS</value>
|
||||
<description>The OBS implementation class of the AbstractFileSystem.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.endpoint</name>
|
||||
<value>obs.region.myhuaweicloud.com</value>
|
||||
<description>OBS region endpoint where a bucket is located.</description>
|
||||
</property>
|
||||
```
|
||||
|
||||
OBS URLs can then be accessed as follows:
|
||||
|
||||
```
|
||||
obs://<bucket_name>/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
|
||||
<property>
|
||||
<name>fs.obs.access.key</name>
|
||||
<description>OBS access key.
|
||||
Omit for provider-based authentication.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.secret.key</name>
|
||||
<description>OBS secret key.
|
||||
Omit for provider-based authentication.</description>
|
||||
</property>
|
||||
```
|
||||
|
||||
**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
|
||||
<property>
|
||||
<name>fs.obs.security.provider</name>
|
||||
<description>
|
||||
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.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.credentials.provider</name>
|
||||
<description>
|
||||
lass nameCof credential provider class which implements
|
||||
org.apache.hadoop.fs.obs.BasicSessionCredential,
|
||||
which must override three APIs: getOBSAccessKeyId(),
|
||||
getOBSSecretKey(), and getSessionToken().
|
||||
</description>
|
||||
</property>
|
||||
```
|
||||
|
||||
## General OBSA Client Configuration
|
||||
|
||||
All OBSA client options are configured with options with the prefix `fs.obs.`.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.obs.connection.ssl.enabled</name>
|
||||
<value>false</value>
|
||||
<description>Enable or disable SSL connections to OBS.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.connection.maximum</name>
|
||||
<value>1000</value>
|
||||
<description>Maximum number of simultaneous connections to OBS.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.connection.establish.timeout</name>
|
||||
<value>120000</value>
|
||||
<description>Socket connection setup timeout in milliseconds.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.connection.timeout</name>
|
||||
<value>120000</value>
|
||||
<description>Socket connection timeout in milliseconds.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.idle.connection.time</name>
|
||||
<value>30000</value>
|
||||
<description>Socket idle connection time.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.max.idle.connections</name>
|
||||
<value>1000</value>
|
||||
<description>Maximum number of socket idle connections.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.socket.send.buffer</name>
|
||||
<value>256 * 1024</value>
|
||||
<description>Socket send buffer to be used in OBS SDK. Represented in bytes.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.socket.recv.buffer</name>
|
||||
<value>256 * 1024</value>
|
||||
<description>Socket receive buffer to be used in OBS SDK. Represented in bytes.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.threads.keepalivetime</name>
|
||||
<value>60</value>
|
||||
<description>Number of seconds a thread can be idle before being
|
||||
terminated in thread pool.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.threads.max</name>
|
||||
<value>20</value>
|
||||
<description> Maximum number of concurrent active (part)uploads,
|
||||
which each use a thread from thread pool.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.max.total.tasks</name>
|
||||
<value>20</value>
|
||||
<description>Number of (part)uploads allowed to the queue before
|
||||
blocking additional uploads.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.delete.threads.max</name>
|
||||
<value>20</value>
|
||||
<description>Max number of delete threads.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.multipart.size</name>
|
||||
<value>104857600</value>
|
||||
<description>Part size for multipart upload.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.multiobjectdelete.maximum</name>
|
||||
<value>1000</value>
|
||||
<description>Max number of objects in one multi-object delete call.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.fast.upload.buffer</name>
|
||||
<value>disk</value>
|
||||
<description>Which buffer to use. Default is `disk`, value may be
|
||||
`disk` | `array` | `bytebuffer`.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.buffer.dir</name>
|
||||
<value>dir1,dir2,dir3</value>
|
||||
<description>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'.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.fast.upload.active.blocks</name>
|
||||
<value>4</value>
|
||||
<description>Maximum number of blocks a single output stream can have active
|
||||
(uploading, or queued to the central FileSystem instance's pool of queued
|
||||
operations).
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.readahead.range</name>
|
||||
<value>1024 * 1024</value>
|
||||
<description>Bytes to read ahead during a seek() before closing and
|
||||
re-opening the OBS HTTP connection. </description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.read.transform.enable</name>
|
||||
<value>true</value>
|
||||
<description>Flag indicating if socket connections can be reused by
|
||||
position read. Set `false` only for HBase.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.list.threads.core</name>
|
||||
<value>30</value>
|
||||
<description>Number of core list threads.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.list.threads.max</name>
|
||||
<value>60</value>
|
||||
<description>Maximum number of list threads.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.list.workqueue.capacity</name>
|
||||
<value>1024</value>
|
||||
<value>Capacity of list work queue.</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.list.parallel.factor</name>
|
||||
<value>30</value>
|
||||
<description>List parallel factor.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.trash.enable</name>
|
||||
<value>false</value>
|
||||
<description>Switch for the fast delete.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.trash.dir</name>
|
||||
<description>The fast delete recycle directory.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.block.size</name>
|
||||
<value>128 * 1024 * 1024</value>
|
||||
<description>Default block size for OBS FileSystem.
|
||||
</description>
|
||||
</property>
|
||||
```
|
||||
|
||||
## 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
|
||||
<property>
|
||||
<name>fs.contract.test.fs.obs</name>
|
||||
<value>obs://obsfilesystem-bucket</value>
|
||||
</property>
|
||||
```
|
@ -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;
|
||||
}
|
@ -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;
|
||||
}
|
||||
}
|
@ -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";
|
||||
|
||||
}
|
@ -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.
|
||||
* <p>
|
||||
* If the test.fs.obs.name property is not set, this will trigger a JUnit
|
||||
* failure.
|
||||
* <p>
|
||||
* 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
|
||||
* <p>
|
||||
* 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() {
|
||||
}
|
||||
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
* <p>
|
||||
* To test a given {@link FileSystem} implementation create a subclass of this
|
||||
* test and override {@link #setUp()} to initialize the <code>fSys</code> {@link
|
||||
* FileSystem} instance variable.
|
||||
* <p>
|
||||
* Since this a junit 4 you can also do a single setup before the start of any
|
||||
* tests. E.g.
|
||||
*
|
||||
*
|
||||
* </p>
|
||||
*/
|
||||
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());
|
||||
}
|
||||
}
|
@ -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());
|
||||
}
|
||||
}
|
@ -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);
|
||||
}
|
||||
}
|
@ -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;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
*
|
||||
* <p>
|
||||
* To test a given {@link FileSystem} implementation create a subclass of this
|
||||
* test and override {@link #setUp()} to initialize the <code>fc1</code> and
|
||||
* <code>fc2</code>
|
||||
* <p>
|
||||
* The tests will do operations on fc1 that use a URI in fc2
|
||||
* <p>
|
||||
* {@link FileContext} instance variable.
|
||||
* </p>
|
||||
*/
|
||||
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);
|
||||
}
|
||||
|
||||
}
|
@ -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;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* 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.
|
||||
* </p>
|
||||
* <p>
|
||||
* To test a given {@link FileSystem} implementation create a subclass of this
|
||||
* test and override {@link #setUp()} to initialize the <code>fc</code> {@link
|
||||
* FileContext} instance variable.
|
||||
*
|
||||
* </p>
|
||||
*/
|
||||
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();
|
||||
}
|
||||
}
|
@ -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.
|
||||
* <p>
|
||||
* 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());
|
||||
}
|
||||
}
|
@ -0,0 +1,139 @@
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
<configuration>
|
||||
<property>
|
||||
<name>fs.contract.test.root-tests-enabled</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.test.supports-concat</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-returns-false-if-source-missing</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.test.random-seek-count</name>
|
||||
<value>10</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.is-case-sensitive</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-returns-true-if-dest-exists</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-returns-true-if-source-missing</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-creates-dest-dirs</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-remove-dest-if-empty-dir</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-settimes</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-append</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-atomic-directory-delete</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-atomic-rename</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-block-locality</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-concat</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-seek</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-seek-on-closed-file</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rejects-seek-past-eof</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-available-on-closed-file</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-strict-exceptions</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-unix-permissions</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-overwrites-dest</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-append</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-getfilestatus</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
</configuration>
|
@ -0,0 +1,136 @@
|
||||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
~ 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.
|
||||
-->
|
||||
|
||||
<!-- Values used when running unit tests. Specify any values in here that
|
||||
should override the default values. -->
|
||||
|
||||
<configuration>
|
||||
<property>
|
||||
<name>hadoop.tmp.dir</name>
|
||||
<value>target/build/test</value>
|
||||
<description>A base for other temporary directories.</description>
|
||||
<final>true</final>
|
||||
</property>
|
||||
<!-- Turn security off for tests by default -->
|
||||
<property>
|
||||
<name>hadoop.security.authentication</name>
|
||||
<value>simple</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.impl</name>
|
||||
<value>org.apache.hadoop.fs.obs.OBSFileSystem</value>
|
||||
<description>The implementation class of the obs Filesystem</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.connection.establish.timeout</name>
|
||||
<value>60000</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.connection.timeout</name>
|
||||
<value>60000</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.idle.connection.time</name>
|
||||
<value>30000</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.max.idle.connections</name>
|
||||
<value>10</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.connection.maximum</name>
|
||||
<value>1000</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.attempts.maximum</name>
|
||||
<value>5</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.upload.stream.retry.buffer.size</name>
|
||||
<value>524288</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.read.buffer.size</name>
|
||||
<value>8192</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.write.buffer.size</name>
|
||||
<value>8192</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.socket.recv.buffer</name>
|
||||
<value>-1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.socket.send.buffer</name>
|
||||
<value>-1</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.keep.alive</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.validate.certificate</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.verify.response.content.type</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.strict.hostname.verification</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.cname</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.test.local.path</name>
|
||||
<value>/uplod_file</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.obs.fast.upload</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.multipart.size</name>
|
||||
<value>10485760</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.obs.experimental.input.fadvise</name>
|
||||
<value>random</value>
|
||||
</property>
|
||||
|
||||
<!--
|
||||
To run these tests.
|
||||
|
||||
# Create a file auth-keys.xml - DO NOT ADD TO REVISION CONTROL
|
||||
# add the property test.fs.obs.name to point to an OBS filesystem URL
|
||||
# Add the credentials for the service you are testing against
|
||||
-->
|
||||
<include xmlns="http://www.w3.org/2001/XInclude" href="auth-keys.xml">
|
||||
<fallback/>
|
||||
</include>
|
||||
|
||||
</configuration>
|
@ -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
|
@ -32,6 +32,7 @@
|
||||
<modules>
|
||||
<module>hadoop-cloud-storage</module>
|
||||
<module>hadoop-cos</module>
|
||||
<module>hadoop-huaweicloud</module>
|
||||
</modules>
|
||||
|
||||
<build>
|
||||
|
@ -676,6 +676,12 @@
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-huaweicloud</artifactId>
|
||||
<version>${hadoop.version}</version>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-kms</artifactId>
|
||||
|
@ -179,6 +179,7 @@
|
||||
href="hadoop-azure-datalake/index.html"/>
|
||||
<item name="OpenStack Swift" href="hadoop-openstack/index.html"/>
|
||||
<item name="Tencent COS" href="hadoop-cos/cloud-storage/index.html"/>
|
||||
<item name="Huaweicloud OBS" href="hadoop-huaweicloud/cloud-storage/index.html"/>
|
||||
</menu>
|
||||
|
||||
<menu name="Auth" inherit="top">
|
||||
|
Loading…
Reference in New Issue
Block a user