HADOOP-15616. Incorporate Tencent Cloud COS File System Implementation. Contributed by Yang Yu.
This commit is contained in:
parent
2870668cfa
commit
8a9ede5cff
@ -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>
|
140
hadoop-cloud-storage-project/hadoop-cos/pom.xml
Normal file
140
hadoop-cloud-storage-project/hadoop-cos/pom.xml
Normal file
@ -0,0 +1,140 @@
|
||||
<?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.3.0-SNAPSHOT</version>
|
||||
<relativePath>../../hadoop-project</relativePath>
|
||||
</parent>
|
||||
<artifactId>hadoop-cos</artifactId>
|
||||
<name>Apache Hadoop Tencent COS Support</name>
|
||||
<description>
|
||||
This module contains code to support integration with Tencent COS.
|
||||
It also declares the dependencies needed to work with COS.
|
||||
</description>
|
||||
<packaging>jar</packaging>
|
||||
|
||||
<properties>
|
||||
<file.encoding>UTF-8</file.encoding>
|
||||
<downloadSources>true</downloadSources>
|
||||
</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>
|
||||
</plugins>
|
||||
</build>
|
||||
|
||||
<dependencies>
|
||||
<dependency>
|
||||
<groupId>junit</groupId>
|
||||
<artifactId>junit</artifactId>
|
||||
<scope>test</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>com.qcloud</groupId>
|
||||
<artifactId>cos_api</artifactId>
|
||||
<version>5.4.9</version>
|
||||
<scope>compile</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<scope>provided</scope>
|
||||
</dependency>
|
||||
|
||||
<dependency>
|
||||
<groupId>org.apache.hadoop</groupId>
|
||||
<artifactId>hadoop-common</artifactId>
|
||||
<scope>test</scope>
|
||||
<type>test-jar</type>
|
||||
</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-client-hs</artifactId>
|
||||
<scope>test</scope>
|
||||
</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>
|
||||
|
||||
</dependencies>
|
||||
</project>
|
@ -0,0 +1,367 @@
|
||||
<!---
|
||||
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.
|
||||
-->
|
||||
|
||||
# Integeration of Tencent COS in Hadoop
|
||||
|
||||
## Introduction
|
||||
|
||||
[Tencent COS](https://intl.cloud.tencent.com/product/cos) is a famous object storage system provided by Tencent Corp. Hadoop-COS is a client that makes the upper computing systems based on HDFS be able to use the COS as its underlying storage system. The big data-processing systems that have been identified for support are: Hadoop MR, Spark, Alluxio and etc. In addition, Druid also can use COS as its deep storage by configuring HDFS-Load-Plugin integerating with HADOOP-COS.
|
||||
|
||||
|
||||
## Features
|
||||
|
||||
- Support Hadoop MapReduce and Spark write data into COS and read from it directly.
|
||||
|
||||
- Implements the interfaces of the Hadoop file system and provides the pseudo-hierarchical directory structure same as HDFS.
|
||||
|
||||
- Supports multipart uploads for a large file. Single file supports up to 19TB
|
||||
|
||||
- High performance and high availability. The performance difference between Hadoop-COS and HDFS is not more than 30%.
|
||||
|
||||
|
||||
> Notes:
|
||||
|
||||
> Object Storage is not a file system and it has some limitations:
|
||||
|
||||
> 1. Object storage is a key-value storage and it does not support hierarchical directory naturally. Usually, using the directory separatory in object key to simulate the hierarchical directory, such as "/hadoop/data/words.dat".
|
||||
|
||||
> 2. COS Object storage can not support the object's append operation currently. It means that you can not append content to the end of an existing object(file).
|
||||
|
||||
> 3. Both `delete` and `rename` operations are non-atomic, which means that the operations are interrupted, the operation result may be inconsistent state.
|
||||
|
||||
> 4. Object storages have different authorization models:
|
||||
|
||||
> - Directory permissions are reported as 777.
|
||||
|
||||
> - File permissions are reported as 666.
|
||||
|
||||
> - File owner is reported as the local current user.
|
||||
|
||||
> - File group is also reported as the local current user.
|
||||
|
||||
> 5. Supports multipart uploads for a large file(up to 40TB), but the number of part is limited as 10000.
|
||||
|
||||
> 6. The number of files listed each time is limited to 1000.
|
||||
|
||||
|
||||
## Quick Start
|
||||
|
||||
### Concepts
|
||||
|
||||
- **Bucket**: A container for storing data in COS. Its name is made up of user-defined bucketname and user appid.
|
||||
|
||||
- **Appid**: Unique resource identifier for the user dimension.
|
||||
|
||||
- **SecretId**: ID used to authenticate the user
|
||||
|
||||
- **SecretKey**: Key used to authenticate the user
|
||||
|
||||
- **Region**: The region where a bucket locates.
|
||||
|
||||
- **CosN**: Hadoop-COS uses `cosn` as its URI scheme, so CosN is often used to refer to Hadoop-COS.
|
||||
|
||||
|
||||
### Usage
|
||||
|
||||
#### System Requirements
|
||||
|
||||
Linux kernel 2.6+
|
||||
|
||||
|
||||
#### Dependencies
|
||||
|
||||
- cos_api (version 5.4.10 or later )
|
||||
- cos-java-sdk (version 2.0.6 recommended)
|
||||
- joda-time (version 2.9.9 recommended)
|
||||
- httpClient (version 4.5.1 or later recommended)
|
||||
- Jackson: jackson-core, jackson-databind, jackson-annotations (version 2.9.8 or later)
|
||||
- bcprov-jdk15on (version 1.59 recommended)
|
||||
|
||||
|
||||
#### Configure Properties
|
||||
|
||||
##### URI and Region Properties
|
||||
|
||||
If you plan to use COS as the default file system for Hadoop or other big data systems, you need to configure `fs.defaultFS` as the URI of Hadoop-COS in core-site.xml. Hadoop-COS uses `cosn` as its URI scheme, and the bucket as its URI host. At the same time, you need to explicitly set `fs.cosn.userinfo.region` to indicate the region your bucket locates.
|
||||
|
||||
**NOTE**:
|
||||
|
||||
- For Hadoop-COS, `fs.defaultFS` is an option. If you are only temporarily using the COS as a data source for Hadoop, you do not need to set the property, just specify the full URI when you use it. For example: `hadoop fs -ls cosn://testBucket-125236746/testDir/test.txt`.
|
||||
|
||||
- `fs.cosn.userinfo.region` is an required property for Hadoop-COS. The reason is that Hadoop-COS must know the region of the using bucket in order to accurately construct a URL to access it.
|
||||
|
||||
- COS supports multi-region storage, and different regions have different access domains by default. It is recommended to choose the nearest storage region according to your own business scenarios, so as to improve the object upload and download speed. You can find the available region from [https://intl.cloud.tencent.com/document/product/436/6224](https://intl.cloud.tencent.com/document/product/436/6224)
|
||||
|
||||
The following is an example for the configuration format:
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.defaultFS</name>
|
||||
<value>cosn://<bucket-appid></value>
|
||||
<description>
|
||||
Optional: If you don't want to use CosN as the default file system, you don't need to configure it.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.bucket.region</name>
|
||||
<value>ap-xxx</value>
|
||||
<description>The region where the bucket is located</description>
|
||||
</property>
|
||||
|
||||
```
|
||||
|
||||
|
||||
##### User Authentication Properties
|
||||
|
||||
Each user needs to properly configure the credentials ( User's secreteId and secretKey ) properly to access the object stored in COS. These credentials can be obtained from the official console provided by Tencent Cloud.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.cosn.credentials.provider</name>
|
||||
<value>org.apache.hadoop.fs.auth.SimpleCredentialProvider</value>
|
||||
<description>
|
||||
|
||||
This option allows the user to specify how to get the credentials.
|
||||
Comma-separated class names of credential provider classes which implement
|
||||
com.qcloud.cos.auth.COSCredentialsProvider:
|
||||
|
||||
1.org.apache.hadoop.fs.auth.SimpleCredentialProvider: Obtain the secret id and secret key
|
||||
from fs.cosn.userinfo.secretId and fs.cosn.userinfo.secretKey in core-site.xml
|
||||
2.org.apache.hadoop.fs.auth.EnvironmentVariableCredentialProvider: Obtain the secret id and secret key from system environment variables named COS_SECRET_ID and COS_SECRET_KEY
|
||||
|
||||
If unspecified, the default order of credential providers is:
|
||||
1. org.apache.hadoop.fs.auth.SimpleCredentialProvider
|
||||
2. org.apache.hadoop.fs.auth.EnvironmentVariableCredentialProvider
|
||||
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.userinfo.secretId</name>
|
||||
<value>xxxxxxxxxxxxxxxxxxxxxxxxx</value>
|
||||
<description>Tencent Cloud Secret Id </description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.userinfo.secretKey</name>
|
||||
<value>xxxxxxxxxxxxxxxxxxxxxxxx</value>
|
||||
<description>Tencent Cloud Secret Key</description>
|
||||
</property>
|
||||
|
||||
```
|
||||
|
||||
|
||||
##### Integration Properties
|
||||
|
||||
You need to explicitly specify the A and B options in order for Hadoop to properly integrate the COS as the underlying file system
|
||||
|
||||
Only correctly set `fs.cosn.impl` and `fs.AbstractFileSystem.cosn.impl` to enable Hadoop to integrate COS as its underlying file system. `fs.cosn.impl` must be set as `org.apache.hadoop.fs.cos.CosFileSystem` and `fs.AbstractFileSystem.cosn.impl` must be set as `org.apache.hadoop.fs.cos.CosN`.
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.cosn.impl</name>
|
||||
<value>org.apache.hadoop.fs.cosn.CosNFileSystem</value>
|
||||
<description>The implementation class of the CosN Filesystem</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.AbstractFileSystem.cosn.impl</name>
|
||||
<value>org.apache.hadoop.fs.cos.CosN</value>
|
||||
<description>The implementation class of the CosN AbstractFileSystem.</description>
|
||||
</property>
|
||||
|
||||
```
|
||||
|
||||
##### Other Runtime Properties
|
||||
|
||||
Hadoop-COS provides rich runtime properties to set, and most of these do not require custom values because a well-run default value provided for them.
|
||||
|
||||
**It is important to note that**:
|
||||
|
||||
- Hadoop-COS will generate some temporary files and consumes some disk space. All temporary files would be placed in the directory specified by option `fs.cosn.tmp.dir` (Default: /tmp/hadoop_cos);
|
||||
|
||||
- The default block size is 8MB and it means that you can only upload a single file up to 78GB into the COS blob storage system. That is mainly due to the fact that the multipart-upload can only support up to 10,000 blocks. For this reason, if needing to support larger single files, you must increase the block size accordingly by setting the property `fs.cosn.block.size`. For example, the size of the largest single file is 1TB, the block size is at least greater than or equal to (1 \* 1024 \* 1024 \* 1024 \* 1024)/10000 = 109951163. Currently, the maximum support file is 19TB (block size: 2147483648)
|
||||
|
||||
```xml
|
||||
<property>
|
||||
<name>fs.cosn.tmp.dir</name>
|
||||
<value>/tmp/hadoop_cos</value>
|
||||
<description>Temporary files would be placed here.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.buffer.size</name>
|
||||
<value>33554432</value>
|
||||
<description>The total size of the buffer pool.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.block.size</name>
|
||||
<value>8388608</value>
|
||||
<description>
|
||||
Block size to use cosn filesysten, which is the part size for MultipartUpload. Considering the COS supports up to 10000 blocks, user should estimate the maximum size of a single file. For example, 8MB part size can allow writing a 78GB single file.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.maxRetries</name>
|
||||
<value>3</value>
|
||||
<description>
|
||||
The maximum number of retries for reading or writing files to COS, before throwing a failure to the application.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.retry.interval.seconds</name>
|
||||
<value>3</value>
|
||||
<description>The number of seconds to sleep between each COS retry.</description>
|
||||
</property>
|
||||
|
||||
```
|
||||
|
||||
|
||||
##### Properties Summary
|
||||
|
||||
| properties | description | default value | required |
|
||||
|:----------:|:-----------|:-------------:|:--------:|
|
||||
| fs.defaultFS | Configure the default file system used by Hadoop.| None | NO |
|
||||
| fs.cosn.credentials.provider | This option allows the user to specify how to get the credentials. Comma-separated class names of credential provider classes which implement com.qcloud.cos.auth.COSCredentialsProvider: <br> 1. org.apache.hadoop.fs.cos.auth.SimpleCredentialProvider: Obtain the secret id and secret key from `fs.cosn.userinfo.secretId` and `fs.cosn.userinfo.secretKey` in core-site.xml; <br> 2. org.apache.hadoop.fs.auth.EnvironmentVariableCredentialProvider: Obtain the secret id and secret key from system environment variables named `COSN_SECRET_ID` and `COSN_SECRET_KEY`. <br> <br> If unspecified, the default order of credential providers is: <br> 1. org.apache.hadoop.fs.auth.SimpleCredentialProvider; <br> 2. org.apache.hadoop.fs.auth.EnvironmentVariableCredentialProvider. | None | NO |
|
||||
| fs.cosn.userinfo.secretId/secretKey | The API key information of your account | None | YES |
|
||||
| fs.cosn.bucket.region | The region where the bucket is located. | None | YES |
|
||||
| fs.cosn.impl | The implementation class of the CosN filesystem. | None | YES |
|
||||
| fs.AbstractFileSystem.cosn.impl | The implementation class of the CosN AbstractFileSystem. | None | YES |
|
||||
| fs.cosn.tmp.dir | Temporary files generated by cosn would be stored here during the program running. | /tmp/hadoop_cos | NO |
|
||||
| fs.cosn.buffer.size | The total size of the buffer pool. Require greater than or equal to block size. | 33554432 | NO |
|
||||
| fs.cosn.block.size | The size of file block. Considering the limitation that each file can be divided into a maximum of 10,000 to upload, the option must be set according to the maximum size of used single file. For example, 8MB part size can allow writing a 78GB single file. | 8388608 | NO |
|
||||
| fs.cosn.upload_thread_pool | Number of threads used for concurrent uploads when files are streamed to COS. | CPU core number * 3 | NO |
|
||||
| fs.cosn.read.ahead.block.size | The size of each read-ahead block. | 524288 (512KB) | NO |
|
||||
| fs.cosn.read.ahead.queue.size | The length of readahead queue. | 10 | NO |
|
||||
| fs.cosn.maxRetries | The maxium number of retries for reading or writing files to COS, before throwing a failure to the application. | 3 | NO |
|
||||
| fs.cosn.retry.interval.seconds | The number of seconds to sleep between each retry | 3 | NO |
|
||||
|
||||
|
||||
#### Command Usage
|
||||
|
||||
Command format: `hadoop fs -ls -R cosn://bucket-appid/<path>` or `hadoop fs -ls -R /<path>`, the latter requires the defaultFs option to be set as `cosn`.
|
||||
|
||||
|
||||
#### Example
|
||||
|
||||
Use CosN as the underlying file system to run the WordCount routine:
|
||||
|
||||
```shell
|
||||
bin/hadoop jar share/hadoop/mapreduce/hadoop-mapreduce-examples-x.x.x.jar wordcount cosn://example/mr/input.txt cosn://example/mr/output
|
||||
```
|
||||
|
||||
If setting CosN as the default file system for Hadoop, you can run it as follows:
|
||||
|
||||
```shell
|
||||
bin/hadoop jar share/hadoop/mapreduce/hadoop-mapreduce-examples-x.x.x.jar wordcount /mr/input.txt /mr/output
|
||||
```
|
||||
|
||||
## Testing the hadoop-cos Module
|
||||
|
||||
To test CosN filesystem, the following two files which pass in authentication details to the test runner are needed.
|
||||
|
||||
1. auth-keys.xml
|
||||
2. core-site.xml
|
||||
|
||||
These two files need to be created under the `hadoop-cloud-storage-project/hadoop-cos/src/test/resource` directory.
|
||||
|
||||
|
||||
### `auth-key.xml`
|
||||
|
||||
COS credentials can specified in `auth-key.xml`. At the same time, it is also a trigger for the CosN filesystem tests.
|
||||
COS bucket URL should be provided by specifying the option: `test.fs.cosn.name`.
|
||||
|
||||
An example of the `auth-keys.xml` is as follow:
|
||||
|
||||
```xml
|
||||
<configuration>
|
||||
<property>
|
||||
<name>test.fs.cosn.name</name>
|
||||
<value>cosn://testbucket-12xxxxxx</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.cosn.bucket.region</name>
|
||||
<value>ap-xxx</value>
|
||||
<description>The region where the bucket is located</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.cosn.userinfo.secretId</name>
|
||||
<value>AKIDXXXXXXXXXXXXXXXXXXXX</value>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.cosn.userinfo.secretKey</name>
|
||||
<value>xxxxxxxxxxxxxxxxxxxxxxxxx</value>
|
||||
</property>
|
||||
</configuration>
|
||||
|
||||
|
||||
```
|
||||
|
||||
Without this file, all tests in this module will be skipped.
|
||||
|
||||
### `core-site.xml`
|
||||
|
||||
This file pre-exists and sources the configurations created in auth-keys.xml.
|
||||
For most cases, no modification is needed, unless a specific, non-default property needs to be set during the testing.
|
||||
|
||||
### `contract-test-options.xml`
|
||||
|
||||
All configurations related to support contract tests need to be specified in `contract-test-options.xml`. Here is an example of `contract-test-options.xml`.
|
||||
|
||||
```xml
|
||||
<?xml version="1.0"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<configuration>
|
||||
<include xmlns="http://www.w3.org/2001/XInclude"
|
||||
href="auth-keys.xml"/>
|
||||
<property>
|
||||
<name>fs.contract.test.fs.cosn</name>
|
||||
<value>cosn://testbucket-12xxxxxx</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.bucket.region</name>
|
||||
<value>ap-xxx</value>
|
||||
<description>The region where the bucket is located</description>
|
||||
</property>
|
||||
|
||||
</configuration>
|
||||
|
||||
```
|
||||
|
||||
If the option `fs.contract.test.fs.cosn` not definded in the file, all contract tests will be skipped.
|
||||
|
||||
## Other issues
|
||||
|
||||
### Performance Loss
|
||||
|
||||
The IO performance of COS is lower than HDFS in principle, even on virtual clusters running on Tencent CVM.
|
||||
|
||||
The main reason can be attributed to the following points:
|
||||
|
||||
- HDFS replicates data for faster query.
|
||||
|
||||
- HDFS is significantly faster for many “metadata” operations: listing the contents of a directory, calling getFileStatus() on path, creating or deleting directories.
|
||||
|
||||
- HDFS stores the data on the local hard disks, avoiding network traffic if the code can be executed on that host. But access to the object storing in COS requires access to network almost each time. It is a critical point in damaging IO performance. Hadoop-COS also do a lot of optimization work for it, such as the pre-read queue, the upload buffer pool, the concurrent upload thread pool, etc.
|
||||
|
||||
- File IO performing many seek calls/positioned read calls will also encounter performance problems due to the size of the HTTP requests made. Despite the pre-read cache optimizations, a large number of random reads can still cause frequent network requests.
|
||||
|
||||
- On HDFS, both the `rename` and `mv` for a directory or a file are an atomic and O(1)-level operation, but in COS, the operation need to combine `copy` and `delete` sequentially. Therefore, performing rename and move operations on a COS object is not only low performance, but also difficult to guarantee data consistency.
|
||||
|
||||
At present, using the COS blob storage system through Hadoop-COS occurs about 20% ~ 25% performance loss compared to HDFS. But, the cost of using COS is lower than HDFS, which includes both storage and maintenance costs.
|
@ -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,245 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.MappedByteBuffer;
|
||||
import java.nio.channels.FileChannel;
|
||||
import java.util.concurrent.BlockingQueue;
|
||||
import java.util.concurrent.LinkedBlockingQueue;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* BufferPool class is used to manage the buffers during program execution.
|
||||
* It is provided in a thread-safe singleton mode,and
|
||||
* keeps the program's memory and disk consumption at a stable value.
|
||||
*/
|
||||
public final class BufferPool {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(BufferPool.class);
|
||||
|
||||
private static BufferPool ourInstance = new BufferPool();
|
||||
|
||||
/**
|
||||
* Use this method to get the instance of BufferPool.
|
||||
*
|
||||
* @return the instance of BufferPool
|
||||
*/
|
||||
public static BufferPool getInstance() {
|
||||
return ourInstance;
|
||||
}
|
||||
|
||||
private BlockingQueue<ByteBuffer> bufferPool = null;
|
||||
private long singleBufferSize = 0;
|
||||
private File diskBufferDir = null;
|
||||
|
||||
private AtomicBoolean isInitialize = new AtomicBoolean(false);
|
||||
|
||||
private BufferPool() {
|
||||
}
|
||||
|
||||
private File createDir(String dirPath) throws IOException {
|
||||
File dir = new File(dirPath);
|
||||
if (null != dir) {
|
||||
if (!dir.exists()) {
|
||||
LOG.debug("Buffer dir: [{}] does not exists. create it first.",
|
||||
dirPath);
|
||||
if (dir.mkdirs()) {
|
||||
if (!dir.setWritable(true) || !dir.setReadable(true)
|
||||
|| !dir.setExecutable(true)) {
|
||||
LOG.warn("Set the buffer dir: [{}]'s permission [writable,"
|
||||
+ "readable, executable] failed.", dir.getAbsolutePath());
|
||||
}
|
||||
LOG.debug("Buffer dir: [{}] is created successfully.",
|
||||
dir.getAbsolutePath());
|
||||
} else {
|
||||
// Once again, check if it has been created successfully.
|
||||
// Prevent problems created by multiple processes at the same time.
|
||||
if (!dir.exists()) {
|
||||
throw new IOException("buffer dir:" + dir.getAbsolutePath()
|
||||
+ " is created unsuccessfully");
|
||||
}
|
||||
}
|
||||
} else {
|
||||
LOG.debug("buffer dir: {} already exists.", dirPath);
|
||||
}
|
||||
} else {
|
||||
throw new IOException("creating buffer dir: " + dir.getAbsolutePath()
|
||||
+ "unsuccessfully.");
|
||||
}
|
||||
|
||||
return dir;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create buffers correctly by reading the buffer file directory,
|
||||
* buffer pool size,and file block size in the configuration.
|
||||
*
|
||||
* @param conf Provides configurations for the Hadoop runtime
|
||||
* @throws IOException Configuration errors,
|
||||
* insufficient or no access for memory or
|
||||
* disk space may cause this exception
|
||||
*/
|
||||
public synchronized void initialize(Configuration conf)
|
||||
throws IOException {
|
||||
if (this.isInitialize.get()) {
|
||||
return;
|
||||
}
|
||||
this.singleBufferSize = conf.getLong(CosNConfigKeys.COSN_BLOCK_SIZE_KEY,
|
||||
CosNConfigKeys.DEFAULT_BLOCK_SIZE);
|
||||
|
||||
// The block size of CosN can only support up to 2GB.
|
||||
if (this.singleBufferSize < Constants.MIN_PART_SIZE
|
||||
|| this.singleBufferSize > Constants.MAX_PART_SIZE) {
|
||||
String exceptionMsg = String.format(
|
||||
"The block size of CosN is limited to %d to %d",
|
||||
Constants.MIN_PART_SIZE, Constants.MAX_PART_SIZE);
|
||||
throw new IOException(exceptionMsg);
|
||||
}
|
||||
|
||||
long memoryBufferLimit = conf.getLong(
|
||||
CosNConfigKeys.COSN_UPLOAD_BUFFER_SIZE_KEY,
|
||||
CosNConfigKeys.DEFAULT_UPLOAD_BUFFER_SIZE);
|
||||
|
||||
this.diskBufferDir = this.createDir(conf.get(
|
||||
CosNConfigKeys.COSN_BUFFER_DIR_KEY,
|
||||
CosNConfigKeys.DEFAULT_BUFFER_DIR));
|
||||
|
||||
int bufferPoolSize = (int) (memoryBufferLimit / this.singleBufferSize);
|
||||
if (0 == bufferPoolSize) {
|
||||
throw new IOException(
|
||||
String.format("The total size of the buffer [%d] is " +
|
||||
"smaller than a single block [%d]."
|
||||
+ "please consider increase the buffer size " +
|
||||
"or decrease the block size",
|
||||
memoryBufferLimit, this.singleBufferSize));
|
||||
}
|
||||
this.bufferPool = new LinkedBlockingQueue<>(bufferPoolSize);
|
||||
for (int i = 0; i < bufferPoolSize; i++) {
|
||||
this.bufferPool.add(ByteBuffer.allocateDirect(
|
||||
(int) this.singleBufferSize));
|
||||
}
|
||||
|
||||
this.isInitialize.set(true);
|
||||
}
|
||||
|
||||
/**
|
||||
* Check if the buffer pool has been initialized.
|
||||
*
|
||||
* @throws IOException if the buffer pool is not initialized
|
||||
*/
|
||||
private void checkInitialize() throws IOException {
|
||||
if (!this.isInitialize.get()) {
|
||||
throw new IOException(
|
||||
"The buffer pool has not been initialized yet");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Obtain a buffer from this buffer pool through the method.
|
||||
*
|
||||
* @param bufferSize expected buffer size to get
|
||||
* @return a buffer wrapper that satisfies the bufferSize.
|
||||
* @throws IOException if the buffer pool not initialized,
|
||||
* or the bufferSize parameter is not within
|
||||
* the range[1MB to the single buffer size]
|
||||
*/
|
||||
public ByteBufferWrapper getBuffer(int bufferSize) throws IOException {
|
||||
this.checkInitialize();
|
||||
if (bufferSize > 0 && bufferSize <= this.singleBufferSize) {
|
||||
ByteBufferWrapper byteBufferWrapper = this.getByteBuffer();
|
||||
if (null == byteBufferWrapper) {
|
||||
// Use a disk buffer when the memory buffer is not enough
|
||||
byteBufferWrapper = this.getMappedBuffer();
|
||||
}
|
||||
return byteBufferWrapper;
|
||||
} else {
|
||||
String exceptionMsg = String.format(
|
||||
"Parameter buffer size out of range: 1048576 to %d",
|
||||
this.singleBufferSize
|
||||
);
|
||||
throw new IOException(exceptionMsg);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a ByteBufferWrapper from the buffer pool.
|
||||
*
|
||||
* @return a new byte buffer wrapper
|
||||
* @throws IOException if the buffer pool is not initialized
|
||||
*/
|
||||
private ByteBufferWrapper getByteBuffer() throws IOException {
|
||||
this.checkInitialize();
|
||||
ByteBuffer buffer = this.bufferPool.poll();
|
||||
return buffer == null ? null : new ByteBufferWrapper(buffer);
|
||||
}
|
||||
|
||||
/**
|
||||
* Get a mapped buffer from the buffer pool.
|
||||
*
|
||||
* @return a new mapped buffer
|
||||
* @throws IOException If the buffer pool is not initialized.
|
||||
* or some I/O error occurs
|
||||
*/
|
||||
private ByteBufferWrapper getMappedBuffer() throws IOException {
|
||||
this.checkInitialize();
|
||||
File tmpFile = File.createTempFile(Constants.BLOCK_TMP_FILE_PREFIX,
|
||||
Constants.BLOCK_TMP_FILE_SUFFIX, this.diskBufferDir);
|
||||
tmpFile.deleteOnExit();
|
||||
RandomAccessFile raf = new RandomAccessFile(tmpFile, "rw");
|
||||
raf.setLength(this.singleBufferSize);
|
||||
MappedByteBuffer buf = raf.getChannel().map(
|
||||
FileChannel.MapMode.READ_WRITE, 0, this.singleBufferSize);
|
||||
return new ByteBufferWrapper(buf, raf, tmpFile);
|
||||
}
|
||||
|
||||
/**
|
||||
* return the byte buffer wrapper to the buffer pool.
|
||||
*
|
||||
* @param byteBufferWrapper the byte buffer wrapper getting from the pool
|
||||
* @throws InterruptedException if interrupted while waiting
|
||||
* @throws IOException some io error occurs
|
||||
*/
|
||||
public void returnBuffer(ByteBufferWrapper byteBufferWrapper)
|
||||
throws InterruptedException, IOException {
|
||||
if (null == this.bufferPool || null == byteBufferWrapper) {
|
||||
return;
|
||||
}
|
||||
|
||||
if (byteBufferWrapper.isDiskBuffer()) {
|
||||
byteBufferWrapper.close();
|
||||
} else {
|
||||
ByteBuffer byteBuffer = byteBufferWrapper.getByteBuffer();
|
||||
if (null != byteBuffer) {
|
||||
byteBuffer.clear();
|
||||
LOG.debug("Return the buffer to the buffer pool.");
|
||||
if (!this.bufferPool.offer(byteBuffer)) {
|
||||
LOG.error("Return the buffer to buffer pool failed.");
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,89 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.InvalidMarkException;
|
||||
|
||||
/**
|
||||
* The input stream class is used for buffered files.
|
||||
* The purpose of providing this class is to optimize buffer read performance.
|
||||
*/
|
||||
public class ByteBufferInputStream extends InputStream {
|
||||
private ByteBuffer byteBuffer;
|
||||
private boolean isClosed;
|
||||
|
||||
public ByteBufferInputStream(ByteBuffer byteBuffer) throws IOException {
|
||||
if (null == byteBuffer) {
|
||||
throw new IOException("byte buffer is null");
|
||||
}
|
||||
this.byteBuffer = byteBuffer;
|
||||
this.isClosed = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
if (null == this.byteBuffer) {
|
||||
throw new IOException("this byte buffer for InputStream is null");
|
||||
}
|
||||
if (!this.byteBuffer.hasRemaining()) {
|
||||
return -1;
|
||||
}
|
||||
return this.byteBuffer.get() & 0xFF;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void mark(int readLimit) {
|
||||
if (!this.markSupported()) {
|
||||
return;
|
||||
}
|
||||
this.byteBuffer.mark();
|
||||
// Parameter readLimit is ignored
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean markSupported() {
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void reset() throws IOException {
|
||||
if (this.isClosed) {
|
||||
throw new IOException("Closed in InputStream");
|
||||
}
|
||||
try {
|
||||
this.byteBuffer.reset();
|
||||
} catch (InvalidMarkException e) {
|
||||
throw new IOException("Invalid mark");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public int available() {
|
||||
return this.byteBuffer.remaining();
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
this.byteBuffer.rewind();
|
||||
this.byteBuffer = null;
|
||||
this.isClosed = true;
|
||||
}
|
||||
}
|
@ -0,0 +1,74 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.nio.ByteBuffer;
|
||||
|
||||
/**
|
||||
* The input stream class is used for buffered files.
|
||||
* The purpose of providing this class is to optimize buffer write performance.
|
||||
*/
|
||||
public class ByteBufferOutputStream extends OutputStream {
|
||||
private ByteBuffer byteBuffer;
|
||||
private boolean isFlush;
|
||||
private boolean isClosed;
|
||||
|
||||
public ByteBufferOutputStream(ByteBuffer byteBuffer) throws IOException {
|
||||
if (null == byteBuffer) {
|
||||
throw new IOException("byte buffer is null");
|
||||
}
|
||||
this.byteBuffer = byteBuffer;
|
||||
this.byteBuffer.clear();
|
||||
this.isFlush = false;
|
||||
this.isClosed = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(int b) {
|
||||
byte[] singleBytes = new byte[1];
|
||||
singleBytes[0] = (byte) b;
|
||||
this.byteBuffer.put(singleBytes, 0, 1);
|
||||
this.isFlush = false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush() {
|
||||
if (this.isFlush) {
|
||||
return;
|
||||
}
|
||||
this.isFlush = true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
if (this.isClosed) {
|
||||
return;
|
||||
}
|
||||
if (null == this.byteBuffer) {
|
||||
throw new IOException("Can not close a null object");
|
||||
}
|
||||
|
||||
this.flush();
|
||||
this.byteBuffer.flip();
|
||||
this.byteBuffer = null;
|
||||
this.isFlush = false;
|
||||
this.isClosed = true;
|
||||
}
|
||||
}
|
@ -0,0 +1,103 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.RandomAccessFile;
|
||||
import java.nio.ByteBuffer;
|
||||
import java.nio.MappedByteBuffer;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.util.CleanerUtil;
|
||||
|
||||
/**
|
||||
* The wrapper for memory buffers and disk buffers.
|
||||
*/
|
||||
public class ByteBufferWrapper {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(ByteBufferWrapper.class);
|
||||
private ByteBuffer byteBuffer;
|
||||
private File file;
|
||||
private RandomAccessFile randomAccessFile;
|
||||
|
||||
ByteBufferWrapper(ByteBuffer byteBuffer) {
|
||||
this(byteBuffer, null, null);
|
||||
}
|
||||
|
||||
ByteBufferWrapper(ByteBuffer byteBuffer, RandomAccessFile randomAccessFile,
|
||||
File file) {
|
||||
this.byteBuffer = byteBuffer;
|
||||
this.file = file;
|
||||
this.randomAccessFile = randomAccessFile;
|
||||
}
|
||||
|
||||
public ByteBuffer getByteBuffer() {
|
||||
return this.byteBuffer;
|
||||
}
|
||||
|
||||
boolean isDiskBuffer() {
|
||||
return this.file != null && this.randomAccessFile != null;
|
||||
}
|
||||
|
||||
private void munmap(MappedByteBuffer buffer) {
|
||||
if (CleanerUtil.UNMAP_SUPPORTED) {
|
||||
try {
|
||||
CleanerUtil.getCleaner().freeBuffer(buffer);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Failed to unmap the buffer", e);
|
||||
}
|
||||
} else {
|
||||
LOG.trace(CleanerUtil.UNMAP_NOT_SUPPORTED_REASON);
|
||||
}
|
||||
}
|
||||
|
||||
void close() throws IOException {
|
||||
if (null != this.byteBuffer) {
|
||||
this.byteBuffer.clear();
|
||||
}
|
||||
|
||||
IOException exception = null;
|
||||
// catch all exceptions, and try to free up resources that can be freed.
|
||||
try {
|
||||
if (null != randomAccessFile) {
|
||||
this.randomAccessFile.close();
|
||||
}
|
||||
} catch (IOException e) {
|
||||
LOG.error("Close the random access file occurs an exception.", e);
|
||||
exception = e;
|
||||
}
|
||||
|
||||
if (this.byteBuffer instanceof MappedByteBuffer) {
|
||||
munmap((MappedByteBuffer) this.byteBuffer);
|
||||
}
|
||||
|
||||
if (null != this.file && this.file.exists()) {
|
||||
if (!this.file.delete()) {
|
||||
LOG.warn("Delete the tmp file: [{}] failed.",
|
||||
this.file.getAbsolutePath());
|
||||
}
|
||||
}
|
||||
|
||||
if (null != exception) {
|
||||
throw exception;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,43 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
/**
|
||||
* constant definition.
|
||||
*/
|
||||
public final class Constants {
|
||||
private Constants() {
|
||||
}
|
||||
|
||||
public static final String BLOCK_TMP_FILE_PREFIX = "cos_";
|
||||
public static final String BLOCK_TMP_FILE_SUFFIX = "_local_block";
|
||||
|
||||
// The maximum number of files listed in a single COS list request.
|
||||
public static final int COS_MAX_LISTING_LENGTH = 999;
|
||||
|
||||
// The maximum number of parts supported by a multipart uploading.
|
||||
public static final int MAX_PART_NUM = 10000;
|
||||
|
||||
// The maximum size of a part
|
||||
public static final long MAX_PART_SIZE = (long) 2 * Unit.GB;
|
||||
// The minimum size of a part
|
||||
public static final long MIN_PART_SIZE = (long) Unit.MB;
|
||||
|
||||
public static final String COSN_SECRET_ID_ENV = "COSN_SECRET_ID";
|
||||
public static final String COSN_SECRET_KEY_ENV = "COSN_SECRET_KEY";
|
||||
}
|
@ -0,0 +1,41 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
import java.net.URISyntaxException;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.DelegateToFileSystem;
|
||||
|
||||
/**
|
||||
* CosN implementation for the Hadoop's AbstractFileSystem.
|
||||
* This implementation delegates to the CosNFileSystem {@link CosNFileSystem}.
|
||||
*/
|
||||
public class CosN extends DelegateToFileSystem {
|
||||
public CosN(URI theUri, Configuration conf)
|
||||
throws IOException, URISyntaxException {
|
||||
super(theUri, new CosNFileSystem(), conf, CosNFileSystem.SCHEME, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public int getUriDefaultPort() {
|
||||
return -1;
|
||||
}
|
||||
}
|
@ -0,0 +1,86 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.fs.CommonConfigurationKeys;
|
||||
|
||||
/**
|
||||
* This class contains constants for configuration keys used in COS.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
public class CosNConfigKeys extends CommonConfigurationKeys {
|
||||
public static final String USER_AGENT = "fs.cosn.user.agent";
|
||||
public static final String DEFAULT_USER_AGENT = "cos-hadoop-plugin-v5.3";
|
||||
|
||||
public static final String COSN_CREDENTIALS_PROVIDER =
|
||||
"fs.cosn.credentials.provider";
|
||||
public static final String COSN_SECRET_ID_KEY = "fs.cosn.userinfo.secretId";
|
||||
public static final String COSN_SECRET_KEY_KEY = "fs.cosn.userinfo.secretKey";
|
||||
public static final String COSN_REGION_KEY = "fs.cosn.bucket.region";
|
||||
public static final String COSN_ENDPOINT_SUFFIX_KEY =
|
||||
"fs.cosn.bucket.endpoint_suffix";
|
||||
|
||||
public static final String COSN_USE_HTTPS_KEY = "fs.cosn.useHttps";
|
||||
public static final boolean DEFAULT_USE_HTTPS = false;
|
||||
|
||||
public static final String COSN_BUFFER_DIR_KEY = "fs.cosn.tmp.dir";
|
||||
public static final String DEFAULT_BUFFER_DIR = "/tmp/hadoop_cos";
|
||||
|
||||
public static final String COSN_UPLOAD_BUFFER_SIZE_KEY =
|
||||
"fs.cosn.buffer.size";
|
||||
public static final long DEFAULT_UPLOAD_BUFFER_SIZE = 32 * Unit.MB;
|
||||
|
||||
public static final String COSN_BLOCK_SIZE_KEY = "fs.cosn.block.size";
|
||||
public static final long DEFAULT_BLOCK_SIZE = 8 * Unit.MB;
|
||||
|
||||
public static final String COSN_MAX_RETRIES_KEY = "fs.cosn.maxRetries";
|
||||
public static final int DEFAULT_MAX_RETRIES = 3;
|
||||
public static final String COSN_RETRY_INTERVAL_KEY =
|
||||
"fs.cosn.retry.interval.seconds";
|
||||
public static final long DEFAULT_RETRY_INTERVAL = 3;
|
||||
|
||||
public static final String UPLOAD_THREAD_POOL_SIZE_KEY =
|
||||
"fs.cosn.upload_thread_pool";
|
||||
public static final int DEFAULT_UPLOAD_THREAD_POOL_SIZE = 1;
|
||||
|
||||
public static final String COPY_THREAD_POOL_SIZE_KEY =
|
||||
"fs.cosn.copy_thread_pool";
|
||||
public static final int DEFAULT_COPY_THREAD_POOL_SIZE = 1;
|
||||
|
||||
/**
|
||||
* This is the maximum time that excess idle threads will wait for new tasks
|
||||
* before terminating. The time unit for it is second.
|
||||
*/
|
||||
public static final String THREAD_KEEP_ALIVE_TIME_KEY =
|
||||
"fs.cosn.threads.keep_alive_time";
|
||||
// The default keep_alive_time is 60 seconds.
|
||||
public static final long DEFAULT_THREAD_KEEP_ALIVE_TIME = 60L;
|
||||
|
||||
public static final String READ_AHEAD_BLOCK_SIZE_KEY =
|
||||
"fs.cosn.read.ahead.block.size";
|
||||
public static final long DEFAULT_READ_AHEAD_BLOCK_SIZE = 512 * Unit.KB;
|
||||
public static final String READ_AHEAD_QUEUE_SIZE =
|
||||
"fs.cosn.read.ahead.queue.size";
|
||||
public static final int DEFAULT_READ_AHEAD_QUEUE_SIZE = 5;
|
||||
|
||||
public static final String MAX_CONNECTION_NUM = "fs.cosn.max.connection.num";
|
||||
public static final int DEFAULT_MAX_CONNECTION_NUM = 2048;
|
||||
}
|
@ -0,0 +1,66 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
/**
|
||||
* The context of the copy task, including concurrency control,
|
||||
* asynchronous acquisition of copy results and etc.
|
||||
*/
|
||||
public class CosNCopyFileContext {
|
||||
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
private Condition readyCondition = lock.newCondition();
|
||||
|
||||
private AtomicBoolean copySuccess = new AtomicBoolean(true);
|
||||
private AtomicInteger copiesFinish = new AtomicInteger(0);
|
||||
|
||||
public void lock() {
|
||||
this.lock.lock();
|
||||
}
|
||||
|
||||
public void unlock() {
|
||||
this.lock.unlock();
|
||||
}
|
||||
|
||||
public void awaitAllFinish(int waitCopiesFinish) throws InterruptedException {
|
||||
while (this.copiesFinish.get() != waitCopiesFinish) {
|
||||
this.readyCondition.await();
|
||||
}
|
||||
}
|
||||
|
||||
public void signalAll() {
|
||||
this.readyCondition.signalAll();
|
||||
}
|
||||
|
||||
public boolean isCopySuccess() {
|
||||
return this.copySuccess.get();
|
||||
}
|
||||
|
||||
public void setCopySuccess(boolean copySuccess) {
|
||||
this.copySuccess.set(copySuccess);
|
||||
}
|
||||
|
||||
public void incCopiesFinish() {
|
||||
this.copiesFinish.addAndGet(1);
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* Used by {@link CosNFileSystem} as an task that submitted
|
||||
* to the thread pool to accelerate the copy progress.
|
||||
* Each task is responsible for copying the source key to the destination.
|
||||
*/
|
||||
public class CosNCopyFileTask implements Runnable {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(CosNCopyFileTask.class);
|
||||
|
||||
private NativeFileSystemStore store;
|
||||
private String srcKey;
|
||||
private String dstKey;
|
||||
private CosNCopyFileContext cosCopyFileContext;
|
||||
|
||||
public CosNCopyFileTask(NativeFileSystemStore store, String srcKey,
|
||||
String dstKey, CosNCopyFileContext cosCopyFileContext) {
|
||||
this.store = store;
|
||||
this.srcKey = srcKey;
|
||||
this.dstKey = dstKey;
|
||||
this.cosCopyFileContext = cosCopyFileContext;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
boolean fail = false;
|
||||
LOG.info(Thread.currentThread().getName() + "copying...");
|
||||
try {
|
||||
this.store.copy(srcKey, dstKey);
|
||||
} catch (IOException e) {
|
||||
LOG.warn("Exception thrown when copy from {} to {}, exception:{}",
|
||||
this.srcKey, this.dstKey, e);
|
||||
fail = true;
|
||||
} finally {
|
||||
this.cosCopyFileContext.lock();
|
||||
if (fail) {
|
||||
cosCopyFileContext.setCopySuccess(false);
|
||||
}
|
||||
cosCopyFileContext.incCopiesFinish();
|
||||
cosCopyFileContext.signalAll();
|
||||
this.cosCopyFileContext.unlock();
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,125 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.util.HashMap;
|
||||
import java.util.Map;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.io.retry.RetryPolicies;
|
||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||
|
||||
/**
|
||||
* Used by {@link CosNInputStream} as an asynchronous task
|
||||
* submitted to the thread pool.
|
||||
* Each task is responsible for reading a part of a large file.
|
||||
* It is used to pre-read the data from COS to accelerate file reading process.
|
||||
*/
|
||||
public class CosNFileReadTask implements Runnable {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(CosNFileReadTask.class);
|
||||
|
||||
private final String key;
|
||||
private final NativeFileSystemStore store;
|
||||
private final CosNInputStream.ReadBuffer readBuffer;
|
||||
|
||||
private RetryPolicy retryPolicy;
|
||||
|
||||
public CosNFileReadTask(
|
||||
Configuration conf,
|
||||
String key, NativeFileSystemStore store,
|
||||
CosNInputStream.ReadBuffer readBuffer) {
|
||||
this.key = key;
|
||||
this.store = store;
|
||||
this.readBuffer = readBuffer;
|
||||
|
||||
RetryPolicy defaultPolicy =
|
||||
RetryPolicies.retryUpToMaximumCountWithFixedSleep(
|
||||
conf.getInt(
|
||||
CosNConfigKeys.COSN_MAX_RETRIES_KEY,
|
||||
CosNConfigKeys.DEFAULT_MAX_RETRIES),
|
||||
conf.getLong(
|
||||
CosNConfigKeys.COSN_RETRY_INTERVAL_KEY,
|
||||
CosNConfigKeys.DEFAULT_RETRY_INTERVAL),
|
||||
TimeUnit.SECONDS);
|
||||
Map<Class<? extends Exception>, RetryPolicy> retryPolicyMap =
|
||||
new HashMap<>();
|
||||
retryPolicyMap.put(IOException.class, defaultPolicy);
|
||||
retryPolicyMap.put(
|
||||
IndexOutOfBoundsException.class, RetryPolicies.TRY_ONCE_THEN_FAIL);
|
||||
retryPolicyMap.put(
|
||||
NullPointerException.class, RetryPolicies.TRY_ONCE_THEN_FAIL);
|
||||
|
||||
this.retryPolicy = RetryPolicies.retryByException(
|
||||
defaultPolicy, retryPolicyMap);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void run() {
|
||||
int retries = 0;
|
||||
RetryPolicy.RetryAction retryAction;
|
||||
LOG.info(Thread.currentThread().getName() + "read ...");
|
||||
try {
|
||||
this.readBuffer.lock();
|
||||
do {
|
||||
try {
|
||||
InputStream inputStream = this.store.retrieveBlock(this.key,
|
||||
this.readBuffer.getStart(), this.readBuffer.getEnd());
|
||||
IOUtils.readFully(inputStream, this.readBuffer.getBuffer(), 0,
|
||||
readBuffer.getBuffer().length);
|
||||
inputStream.close();
|
||||
this.readBuffer.setStatus(CosNInputStream.ReadBuffer.SUCCESS);
|
||||
break;
|
||||
} catch (IOException e) {
|
||||
this.readBuffer.setStatus(CosNInputStream.ReadBuffer.ERROR);
|
||||
LOG.warn(
|
||||
"Exception occurs when retrieve the block range start: "
|
||||
+ String.valueOf(this.readBuffer.getStart()) + " end: "
|
||||
+ this.readBuffer.getEnd());
|
||||
try {
|
||||
retryAction = this.retryPolicy.shouldRetry(
|
||||
e, retries++, 0, true);
|
||||
if (retryAction.action
|
||||
== RetryPolicy.RetryAction.RetryDecision.RETRY) {
|
||||
Thread.sleep(retryAction.delayMillis);
|
||||
}
|
||||
} catch (Exception e1) {
|
||||
String errMsg = String.format("Exception occurs when retry[%s] "
|
||||
+ "to retrieve the block range start: %s, end:%s",
|
||||
this.retryPolicy.toString(),
|
||||
String.valueOf(this.readBuffer.getStart()),
|
||||
String.valueOf(this.readBuffer.getEnd()));
|
||||
LOG.error(errMsg, e1);
|
||||
break;
|
||||
}
|
||||
}
|
||||
} while (retryAction.action ==
|
||||
RetryPolicy.RetryAction.RetryDecision.RETRY);
|
||||
this.readBuffer.signalAll();
|
||||
} finally {
|
||||
this.readBuffer.unLock();
|
||||
}
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.FileNotFoundException;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.EnumSet;
|
||||
import java.util.List;
|
||||
import java.util.Map;
|
||||
import java.util.HashMap;
|
||||
import java.util.Set;
|
||||
import java.util.TreeSet;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.BufferedFSInputStream;
|
||||
import org.apache.hadoop.fs.CreateFlag;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileAlreadyExistsException;
|
||||
import org.apache.hadoop.fs.FileStatus;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.PathIOException;
|
||||
import org.apache.hadoop.fs.permission.FsPermission;
|
||||
import org.apache.hadoop.io.retry.RetryPolicies;
|
||||
import org.apache.hadoop.io.retry.RetryPolicy;
|
||||
import org.apache.hadoop.io.retry.RetryProxy;
|
||||
import org.apache.hadoop.util.BlockingThreadPoolExecutorService;
|
||||
import org.apache.hadoop.util.Progressable;
|
||||
|
||||
/**
|
||||
* The core CosN Filesystem implementation.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Stable
|
||||
public class CosNFileSystem extends FileSystem {
|
||||
static final Logger LOG = LoggerFactory.getLogger(CosNFileSystem.class);
|
||||
|
||||
public static final String SCHEME = "cosn";
|
||||
public static final String PATH_DELIMITER = Path.SEPARATOR;
|
||||
|
||||
private URI uri;
|
||||
private String bucket;
|
||||
private NativeFileSystemStore store;
|
||||
private Path workingDir;
|
||||
private String owner = "Unknown";
|
||||
private String group = "Unknown";
|
||||
|
||||
private ListeningExecutorService boundedIOThreadPool;
|
||||
private ListeningExecutorService boundedCopyThreadPool;
|
||||
|
||||
public CosNFileSystem() {
|
||||
}
|
||||
|
||||
public CosNFileSystem(NativeFileSystemStore store) {
|
||||
this.store = store;
|
||||
}
|
||||
|
||||
/**
|
||||
* Return the protocol scheme for the FileSystem.
|
||||
*
|
||||
* @return <code>cosn</code>
|
||||
*/
|
||||
@Override
|
||||
public String getScheme() {
|
||||
return CosNFileSystem.SCHEME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void initialize(URI name, Configuration conf) throws IOException {
|
||||
super.initialize(name, conf);
|
||||
this.bucket = name.getHost();
|
||||
if (this.store == null) {
|
||||
this.store = createDefaultStore(conf);
|
||||
}
|
||||
this.store.initialize(name, conf);
|
||||
setConf(conf);
|
||||
this.uri = URI.create(name.getScheme() + "://" + name.getAuthority());
|
||||
this.workingDir = new Path("/user",
|
||||
System.getProperty("user.name")).makeQualified(
|
||||
this.uri,
|
||||
this.getWorkingDirectory());
|
||||
this.owner = getOwnerId();
|
||||
this.group = getGroupId();
|
||||
LOG.debug("owner:" + owner + ", group:" + group);
|
||||
|
||||
BufferPool.getInstance().initialize(this.getConf());
|
||||
|
||||
// initialize the thread pool
|
||||
int uploadThreadPoolSize = this.getConf().getInt(
|
||||
CosNConfigKeys.UPLOAD_THREAD_POOL_SIZE_KEY,
|
||||
CosNConfigKeys.DEFAULT_UPLOAD_THREAD_POOL_SIZE
|
||||
);
|
||||
int readAheadPoolSize = this.getConf().getInt(
|
||||
CosNConfigKeys.READ_AHEAD_QUEUE_SIZE,
|
||||
CosNConfigKeys.DEFAULT_READ_AHEAD_QUEUE_SIZE
|
||||
);
|
||||
int ioThreadPoolSize = uploadThreadPoolSize + readAheadPoolSize / 3;
|
||||
long threadKeepAlive = this.getConf().getLong(
|
||||
CosNConfigKeys.THREAD_KEEP_ALIVE_TIME_KEY,
|
||||
CosNConfigKeys.DEFAULT_THREAD_KEEP_ALIVE_TIME
|
||||
);
|
||||
this.boundedIOThreadPool = BlockingThreadPoolExecutorService.newInstance(
|
||||
ioThreadPoolSize / 2, ioThreadPoolSize,
|
||||
threadKeepAlive, TimeUnit.SECONDS,
|
||||
"cos-transfer-thread-pool");
|
||||
int copyThreadPoolSize = this.getConf().getInt(
|
||||
CosNConfigKeys.COPY_THREAD_POOL_SIZE_KEY,
|
||||
CosNConfigKeys.DEFAULT_COPY_THREAD_POOL_SIZE
|
||||
);
|
||||
this.boundedCopyThreadPool = BlockingThreadPoolExecutorService.newInstance(
|
||||
CosNConfigKeys.DEFAULT_COPY_THREAD_POOL_SIZE, copyThreadPoolSize,
|
||||
60L, TimeUnit.SECONDS,
|
||||
"cos-copy-thread-pool");
|
||||
}
|
||||
|
||||
private static NativeFileSystemStore createDefaultStore(Configuration conf) {
|
||||
NativeFileSystemStore store = new CosNativeFileSystemStore();
|
||||
RetryPolicy basePolicy = RetryPolicies.retryUpToMaximumCountWithFixedSleep(
|
||||
conf.getInt(CosNConfigKeys.COSN_MAX_RETRIES_KEY,
|
||||
CosNConfigKeys.DEFAULT_MAX_RETRIES),
|
||||
conf.getLong(CosNConfigKeys.COSN_RETRY_INTERVAL_KEY,
|
||||
CosNConfigKeys.DEFAULT_RETRY_INTERVAL),
|
||||
TimeUnit.SECONDS);
|
||||
Map<Class<? extends Exception>, RetryPolicy> exceptionToPolicyMap =
|
||||
new HashMap<>();
|
||||
|
||||
exceptionToPolicyMap.put(IOException.class, basePolicy);
|
||||
RetryPolicy methodPolicy = RetryPolicies.retryByException(
|
||||
RetryPolicies.TRY_ONCE_THEN_FAIL,
|
||||
exceptionToPolicyMap);
|
||||
Map<String, RetryPolicy> methodNameToPolicyMap = new HashMap<>();
|
||||
methodNameToPolicyMap.put("storeFile", methodPolicy);
|
||||
methodNameToPolicyMap.put("rename", methodPolicy);
|
||||
|
||||
return (NativeFileSystemStore) RetryProxy.create(
|
||||
NativeFileSystemStore.class, store, methodNameToPolicyMap);
|
||||
}
|
||||
|
||||
private String getOwnerId() {
|
||||
return System.getProperty("user.name");
|
||||
}
|
||||
|
||||
private String getGroupId() {
|
||||
return System.getProperty("user.name");
|
||||
}
|
||||
|
||||
private String getOwnerInfo(boolean getOwnerId) {
|
||||
String ownerInfoId = "";
|
||||
try {
|
||||
String userName = System.getProperty("user.name");
|
||||
String command = "id -u " + userName;
|
||||
if (!getOwnerId) {
|
||||
command = "id -g " + userName;
|
||||
}
|
||||
Process child = Runtime.getRuntime().exec(command);
|
||||
child.waitFor();
|
||||
|
||||
// Get the input stream and read from it
|
||||
InputStream in = child.getInputStream();
|
||||
StringBuilder strBuffer = new StringBuilder();
|
||||
int c;
|
||||
while ((c = in.read()) != -1) {
|
||||
strBuffer.append((char) c);
|
||||
}
|
||||
in.close();
|
||||
ownerInfoId = strBuffer.toString();
|
||||
} catch (IOException | InterruptedException e) {
|
||||
LOG.error("Getting owner info occurs a exception", e);
|
||||
}
|
||||
return ownerInfoId;
|
||||
}
|
||||
|
||||
private static String pathToKey(Path path) {
|
||||
if (path.toUri().getScheme() != null && path.toUri().getPath().isEmpty()) {
|
||||
// allow uris without trailing slash after bucket to refer to root,
|
||||
// like cosn://mybucket
|
||||
return "";
|
||||
}
|
||||
if (!path.isAbsolute()) {
|
||||
throw new IllegalArgumentException("Path must be absolute: " + path);
|
||||
}
|
||||
String ret = path.toUri().getPath();
|
||||
if (ret.endsWith("/") && (ret.indexOf("/") != ret.length() - 1)) {
|
||||
ret = ret.substring(0, ret.length() - 1);
|
||||
}
|
||||
return ret;
|
||||
}
|
||||
|
||||
private static Path keyToPath(String key) {
|
||||
if (!key.startsWith(PATH_DELIMITER)) {
|
||||
return new Path("/" + key);
|
||||
} else {
|
||||
return new Path(key);
|
||||
}
|
||||
}
|
||||
|
||||
private Path makeAbsolute(Path path) {
|
||||
if (path.isAbsolute()) {
|
||||
return path;
|
||||
}
|
||||
return new Path(workingDir, path);
|
||||
}
|
||||
|
||||
/**
|
||||
* This optional operation is not yet supported.
|
||||
*/
|
||||
@Override
|
||||
public FSDataOutputStream append(Path f, int bufferSize,
|
||||
Progressable progress) throws IOException {
|
||||
throw new IOException("Not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream create(Path f, FsPermission permission,
|
||||
boolean overwrite, int bufferSize, short replication, long blockSize,
|
||||
Progressable progress) throws IOException {
|
||||
FileStatus fileStatus;
|
||||
|
||||
try {
|
||||
fileStatus = getFileStatus(f);
|
||||
if (fileStatus.isDirectory()) {
|
||||
throw new FileAlreadyExistsException(f + " is a directory");
|
||||
}
|
||||
if (!overwrite) {
|
||||
// path references a file and overwrite is disabled
|
||||
throw new FileAlreadyExistsException(f + " already exists");
|
||||
}
|
||||
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.debug("Creating a new file: [{}] in COS.", f);
|
||||
}
|
||||
|
||||
Path absolutePath = makeAbsolute(f);
|
||||
String key = pathToKey(absolutePath);
|
||||
return new FSDataOutputStream(
|
||||
new CosNOutputStream(getConf(), store, key, blockSize,
|
||||
this.boundedIOThreadPool), statistics);
|
||||
}
|
||||
|
||||
private boolean rejectRootDirectoryDelete(boolean isEmptyDir,
|
||||
boolean recursive) throws PathIOException {
|
||||
if (isEmptyDir) {
|
||||
return true;
|
||||
}
|
||||
if (recursive) {
|
||||
return false;
|
||||
} else {
|
||||
throw new PathIOException(this.bucket, "Can not delete root path");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataOutputStream createNonRecursive(Path f, FsPermission permission,
|
||||
EnumSet<CreateFlag> flags, int bufferSize, short replication,
|
||||
long blockSize, Progressable progress) throws IOException {
|
||||
Path parent = f.getParent();
|
||||
if (null != parent) {
|
||||
if (!getFileStatus(parent).isDirectory()) {
|
||||
throw new FileAlreadyExistsException("Not a directory: " + parent);
|
||||
}
|
||||
}
|
||||
|
||||
return create(f, permission, flags.contains(CreateFlag.OVERWRITE),
|
||||
bufferSize, replication, blockSize, progress);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean delete(Path f, boolean recursive) throws IOException {
|
||||
LOG.debug("Ready to delete path: [{}]. recursive: [{}].", f, recursive);
|
||||
FileStatus status;
|
||||
try {
|
||||
status = getFileStatus(f);
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.debug("Ready to delete the file: [{}], but it does not exist.", f);
|
||||
return false;
|
||||
}
|
||||
Path absolutePath = makeAbsolute(f);
|
||||
String key = pathToKey(absolutePath);
|
||||
if (key.compareToIgnoreCase("/") == 0) {
|
||||
FileStatus[] fileStatuses = listStatus(f);
|
||||
return this.rejectRootDirectoryDelete(
|
||||
fileStatuses.length == 0, recursive);
|
||||
}
|
||||
|
||||
if (status.isDirectory()) {
|
||||
if (!key.endsWith(PATH_DELIMITER)) {
|
||||
key += PATH_DELIMITER;
|
||||
}
|
||||
if (!recursive && listStatus(f).length > 0) {
|
||||
String errMsg = String.format("Can not delete the directory: [%s], as"
|
||||
+ " it is not empty and option recursive is false.", f);
|
||||
throw new IOException(errMsg);
|
||||
}
|
||||
|
||||
createParent(f);
|
||||
|
||||
String priorLastKey = null;
|
||||
do {
|
||||
PartialListing listing = store.list(
|
||||
key,
|
||||
Constants.COS_MAX_LISTING_LENGTH,
|
||||
priorLastKey,
|
||||
true);
|
||||
for (FileMetadata file : listing.getFiles()) {
|
||||
store.delete(file.getKey());
|
||||
}
|
||||
for (FileMetadata commonPrefix : listing.getCommonPrefixes()) {
|
||||
store.delete(commonPrefix.getKey());
|
||||
}
|
||||
priorLastKey = listing.getPriorLastKey();
|
||||
} while (priorLastKey != null);
|
||||
try {
|
||||
store.delete(key);
|
||||
} catch (Exception e) {
|
||||
LOG.error("Deleting the COS key: [{}] occurs an exception.", key, e);
|
||||
}
|
||||
|
||||
} else {
|
||||
LOG.debug("Delete the file: {}", f);
|
||||
createParent(f);
|
||||
store.delete(key);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileStatus getFileStatus(Path f) throws IOException {
|
||||
Path absolutePath = makeAbsolute(f);
|
||||
String key = pathToKey(absolutePath);
|
||||
|
||||
if (key.length() == 0) {
|
||||
// root always exists
|
||||
return newDirectory(absolutePath);
|
||||
}
|
||||
|
||||
LOG.debug("Call the getFileStatus to obtain the metadata for "
|
||||
+ "the file: [{}].", f);
|
||||
|
||||
FileMetadata meta = store.retrieveMetadata(key);
|
||||
if (meta != null) {
|
||||
if (meta.isFile()) {
|
||||
LOG.debug("Path: [{}] is a file. COS key: [{}]", f, key);
|
||||
return newFile(meta, absolutePath);
|
||||
} else {
|
||||
LOG.debug("Path: [{}] is a dir. COS key: [{}]", f, key);
|
||||
return newDirectory(meta, absolutePath);
|
||||
}
|
||||
}
|
||||
|
||||
if (!key.endsWith(PATH_DELIMITER)) {
|
||||
key += PATH_DELIMITER;
|
||||
}
|
||||
|
||||
// Considering that the object store's directory is a common prefix in
|
||||
// the object key, it needs to check the existence of the path by listing
|
||||
// the COS key.
|
||||
LOG.debug("List COS key: [{}] to check the existence of the path.", key);
|
||||
PartialListing listing = store.list(key, 1);
|
||||
if (listing.getFiles().length > 0
|
||||
|| listing.getCommonPrefixes().length > 0) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Path: [{}] is a directory. COS key: [{}]", f, key);
|
||||
}
|
||||
return newDirectory(absolutePath);
|
||||
}
|
||||
|
||||
throw new FileNotFoundException(
|
||||
"No such file or directory '" + absolutePath + "'");
|
||||
}
|
||||
|
||||
@Override
|
||||
public URI getUri() {
|
||||
return uri;
|
||||
}
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* If <code>f</code> is a file, this method will make a single call to COS.
|
||||
* If <code>f</code> is a directory,
|
||||
* this method will make a maximum of ( <i>n</i> / 199) + 2 calls to cos,
|
||||
* where <i>n</i> is the total number of files
|
||||
* and directories contained directly in <code>f</code>.
|
||||
* </p>
|
||||
*/
|
||||
@Override
|
||||
public FileStatus[] listStatus(Path f) throws IOException {
|
||||
Path absolutePath = makeAbsolute(f);
|
||||
String key = pathToKey(absolutePath);
|
||||
|
||||
if (key.length() > 0) {
|
||||
FileStatus fileStatus = this.getFileStatus(f);
|
||||
if (fileStatus.isFile()) {
|
||||
return new FileStatus[]{fileStatus};
|
||||
}
|
||||
}
|
||||
|
||||
if (!key.endsWith(PATH_DELIMITER)) {
|
||||
key += PATH_DELIMITER;
|
||||
}
|
||||
|
||||
URI pathUri = absolutePath.toUri();
|
||||
Set<FileStatus> status = new TreeSet<>();
|
||||
String priorLastKey = null;
|
||||
do {
|
||||
PartialListing listing = store.list(
|
||||
key, Constants.COS_MAX_LISTING_LENGTH, priorLastKey, false);
|
||||
for (FileMetadata fileMetadata : listing.getFiles()) {
|
||||
Path subPath = keyToPath(fileMetadata.getKey());
|
||||
if (fileMetadata.getKey().equals(key)) {
|
||||
// this is just the directory we have been asked to list.
|
||||
LOG.debug("The file list contains the COS key [{}] to be listed.",
|
||||
key);
|
||||
} else {
|
||||
status.add(newFile(fileMetadata, subPath));
|
||||
}
|
||||
}
|
||||
|
||||
for (FileMetadata commonPrefix : listing.getCommonPrefixes()) {
|
||||
Path subPath = keyToPath(commonPrefix.getKey());
|
||||
String relativePath = pathUri.relativize(subPath.toUri()).getPath();
|
||||
status.add(
|
||||
newDirectory(commonPrefix, new Path(absolutePath, relativePath)));
|
||||
}
|
||||
priorLastKey = listing.getPriorLastKey();
|
||||
} while (priorLastKey != null);
|
||||
|
||||
return status.toArray(new FileStatus[status.size()]);
|
||||
}
|
||||
|
||||
private FileStatus newFile(FileMetadata meta, Path path) {
|
||||
return new FileStatus(meta.getLength(), false, 1, getDefaultBlockSize(),
|
||||
meta.getLastModified(), 0, null, this.owner, this.group,
|
||||
path.makeQualified(this.getUri(), this.getWorkingDirectory()));
|
||||
}
|
||||
|
||||
private FileStatus newDirectory(Path path) {
|
||||
return new FileStatus(0, true, 1, 0, 0, 0, null, this.owner, this.group,
|
||||
path.makeQualified(this.getUri(), this.getWorkingDirectory()));
|
||||
}
|
||||
|
||||
private FileStatus newDirectory(FileMetadata meta, Path path) {
|
||||
if (meta == null) {
|
||||
return newDirectory(path);
|
||||
}
|
||||
return new FileStatus(0, true, 1, 0, meta.getLastModified(),
|
||||
0, null, this.owner, this.group,
|
||||
path.makeQualified(this.getUri(), this.getWorkingDirectory()));
|
||||
}
|
||||
|
||||
/**
|
||||
* Validate the path from the bottom up.
|
||||
*
|
||||
* @param path The path to be validated
|
||||
* @throws FileAlreadyExistsException The specified path is an existing file
|
||||
* @throws IOException Getting the file status of the
|
||||
* specified path occurs
|
||||
* an IOException.
|
||||
*/
|
||||
private void validatePath(Path path) throws IOException {
|
||||
Path parent = path.getParent();
|
||||
do {
|
||||
try {
|
||||
FileStatus fileStatus = getFileStatus(parent);
|
||||
if (fileStatus.isDirectory()) {
|
||||
break;
|
||||
} else {
|
||||
throw new FileAlreadyExistsException(String.format(
|
||||
"Can't make directory for path '%s', it is a file.", parent));
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.debug("The Path: [{}] does not exist.", path);
|
||||
}
|
||||
parent = parent.getParent();
|
||||
} while (parent != null);
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean mkdirs(Path f, FsPermission permission) throws IOException {
|
||||
try {
|
||||
FileStatus fileStatus = getFileStatus(f);
|
||||
if (fileStatus.isDirectory()) {
|
||||
return true;
|
||||
} else {
|
||||
throw new FileAlreadyExistsException("Path is a file: " + f);
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
validatePath(f);
|
||||
}
|
||||
|
||||
return mkDirRecursively(f, permission);
|
||||
}
|
||||
|
||||
/**
|
||||
* Recursively create a directory.
|
||||
*
|
||||
* @param f Absolute path to the directory.
|
||||
* @param permission Directory permissions. Permission does not work for
|
||||
* the CosN filesystem currently.
|
||||
* @return Return true if the creation was successful, throw a IOException.
|
||||
* @throws IOException The specified path already exists or an error
|
||||
* creating the path.
|
||||
*/
|
||||
public boolean mkDirRecursively(Path f, FsPermission permission)
|
||||
throws IOException {
|
||||
Path absolutePath = makeAbsolute(f);
|
||||
List<Path> paths = new ArrayList<>();
|
||||
do {
|
||||
paths.add(absolutePath);
|
||||
absolutePath = absolutePath.getParent();
|
||||
} while (absolutePath != null);
|
||||
|
||||
for (Path path : paths) {
|
||||
if (path.equals(new Path(CosNFileSystem.PATH_DELIMITER))) {
|
||||
break;
|
||||
}
|
||||
try {
|
||||
FileStatus fileStatus = getFileStatus(path);
|
||||
if (fileStatus.isFile()) {
|
||||
throw new FileAlreadyExistsException(
|
||||
String.format("Can't make directory for path: %s, "
|
||||
+ "since it is a file.", f));
|
||||
}
|
||||
if (fileStatus.isDirectory()) {
|
||||
break;
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
LOG.debug("Making dir: [{}] in COS", f);
|
||||
|
||||
String folderPath = pathToKey(makeAbsolute(f));
|
||||
if (!folderPath.endsWith(PATH_DELIMITER)) {
|
||||
folderPath += PATH_DELIMITER;
|
||||
}
|
||||
store.storeEmptyFile(folderPath);
|
||||
}
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean mkdir(Path f) throws IOException {
|
||||
try {
|
||||
FileStatus fileStatus = getFileStatus(f);
|
||||
if (fileStatus.isFile()) {
|
||||
throw new FileAlreadyExistsException(
|
||||
String.format(
|
||||
"Can't make directory for path '%s' since it is a file.", f));
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("Make directory: [{}] in COS.", f);
|
||||
}
|
||||
|
||||
String folderPath = pathToKey(makeAbsolute(f));
|
||||
if (!folderPath.endsWith(PATH_DELIMITER)) {
|
||||
folderPath += PATH_DELIMITER;
|
||||
}
|
||||
store.storeEmptyFile(folderPath);
|
||||
}
|
||||
return true;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FSDataInputStream open(Path f, int bufferSize) throws IOException {
|
||||
FileStatus fs = getFileStatus(f); // will throw if the file doesn't
|
||||
// exist
|
||||
if (fs.isDirectory()) {
|
||||
throw new FileNotFoundException("'" + f + "' is a directory");
|
||||
}
|
||||
LOG.info("Open the file: [{}] for reading.", f);
|
||||
Path absolutePath = makeAbsolute(f);
|
||||
String key = pathToKey(absolutePath);
|
||||
long fileSize = store.getFileLength(key);
|
||||
return new FSDataInputStream(new BufferedFSInputStream(
|
||||
new CosNInputStream(this.getConf(), store, statistics, key, fileSize,
|
||||
this.boundedIOThreadPool), bufferSize));
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean rename(Path src, Path dst) throws IOException {
|
||||
LOG.debug("Rename source path: [{}] to dest path: [{}].", src, dst);
|
||||
|
||||
// Renaming the root directory is not allowed
|
||||
if (src.isRoot()) {
|
||||
LOG.debug("Cannot rename the root directory of a filesystem.");
|
||||
return false;
|
||||
}
|
||||
|
||||
// check the source path whether exists or not
|
||||
FileStatus srcFileStatus = this.getFileStatus(src);
|
||||
|
||||
// Source path and destination path are not allowed to be the same
|
||||
if (src.equals(dst)) {
|
||||
LOG.debug("Source path and dest path refer to "
|
||||
+ "the same file or directory: [{}].", dst);
|
||||
throw new IOException("Source path and dest path refer "
|
||||
+ "the same file or directory");
|
||||
}
|
||||
|
||||
// It is not allowed to rename a parent directory to its subdirectory
|
||||
Path dstParentPath;
|
||||
for (dstParentPath = dst.getParent();
|
||||
null != dstParentPath && !src.equals(dstParentPath);
|
||||
dstParentPath = dstParentPath.getParent()) {
|
||||
// Recursively find the common parent path of the source and
|
||||
// destination paths.
|
||||
LOG.debug("Recursively find the common parent directory of the source "
|
||||
+ "and destination paths. The currently found parent path: {}",
|
||||
dstParentPath);
|
||||
}
|
||||
|
||||
if (null != dstParentPath) {
|
||||
LOG.debug("It is not allowed to rename a parent directory:[{}] "
|
||||
+ "to its subdirectory:[{}].", src, dst);
|
||||
throw new IOException(String.format(
|
||||
"It is not allowed to rename a parent directory: %s "
|
||||
+ "to its subdirectory: %s", src, dst));
|
||||
}
|
||||
|
||||
FileStatus dstFileStatus;
|
||||
try {
|
||||
dstFileStatus = this.getFileStatus(dst);
|
||||
|
||||
// The destination path exists and is a file,
|
||||
// and the rename operation is not allowed.
|
||||
if (dstFileStatus.isFile()) {
|
||||
throw new FileAlreadyExistsException(String.format(
|
||||
"File: %s already exists", dstFileStatus.getPath()));
|
||||
} else {
|
||||
// The destination path is an existing directory,
|
||||
// and it is checked whether there is a file or directory
|
||||
// with the same name as the source path under the destination path
|
||||
dst = new Path(dst, src.getName());
|
||||
FileStatus[] statuses;
|
||||
try {
|
||||
statuses = this.listStatus(dst);
|
||||
} catch (FileNotFoundException e) {
|
||||
statuses = null;
|
||||
}
|
||||
if (null != statuses && statuses.length > 0) {
|
||||
LOG.debug("Cannot rename source file: [{}] to dest file: [{}], "
|
||||
+ "because the file already exists.", src, dst);
|
||||
throw new FileAlreadyExistsException(
|
||||
String.format(
|
||||
"File: %s already exists", dst
|
||||
)
|
||||
);
|
||||
}
|
||||
}
|
||||
} catch (FileNotFoundException e) {
|
||||
// destination path not exists
|
||||
Path tempDstParentPath = dst.getParent();
|
||||
FileStatus dstParentStatus = this.getFileStatus(tempDstParentPath);
|
||||
if (!dstParentStatus.isDirectory()) {
|
||||
throw new IOException(String.format(
|
||||
"Cannot rename %s to %s, %s is a file", src, dst, dst.getParent()
|
||||
));
|
||||
}
|
||||
// The default root directory is definitely there.
|
||||
}
|
||||
|
||||
boolean result;
|
||||
if (srcFileStatus.isDirectory()) {
|
||||
result = this.copyDirectory(src, dst);
|
||||
} else {
|
||||
result = this.copyFile(src, dst);
|
||||
}
|
||||
|
||||
if (!result) {
|
||||
//Since rename is a non-atomic operation, after copy fails,
|
||||
// it is not allowed to delete the data of the original path.
|
||||
return false;
|
||||
} else {
|
||||
return this.delete(src, true);
|
||||
}
|
||||
}
|
||||
|
||||
private boolean copyFile(Path srcPath, Path dstPath) throws IOException {
|
||||
String srcKey = pathToKey(srcPath);
|
||||
String dstKey = pathToKey(dstPath);
|
||||
this.store.copy(srcKey, dstKey);
|
||||
return true;
|
||||
}
|
||||
|
||||
private boolean copyDirectory(Path srcPath, Path dstPath) throws IOException {
|
||||
String srcKey = pathToKey(srcPath);
|
||||
if (!srcKey.endsWith(PATH_DELIMITER)) {
|
||||
srcKey += PATH_DELIMITER;
|
||||
}
|
||||
String dstKey = pathToKey(dstPath);
|
||||
if (!dstKey.endsWith(PATH_DELIMITER)) {
|
||||
dstKey += PATH_DELIMITER;
|
||||
}
|
||||
|
||||
if (dstKey.startsWith(srcKey)) {
|
||||
throw new IOException(
|
||||
"can not copy a directory to a subdirectory of self");
|
||||
}
|
||||
|
||||
this.store.storeEmptyFile(dstKey);
|
||||
CosNCopyFileContext copyFileContext = new CosNCopyFileContext();
|
||||
|
||||
int copiesToFinishes = 0;
|
||||
String priorLastKey = null;
|
||||
do {
|
||||
PartialListing objectList = this.store.list(
|
||||
srcKey, Constants.COS_MAX_LISTING_LENGTH, priorLastKey, true);
|
||||
for (FileMetadata file : objectList.getFiles()) {
|
||||
this.boundedCopyThreadPool.execute(new CosNCopyFileTask(
|
||||
this.store,
|
||||
file.getKey(),
|
||||
dstKey.concat(file.getKey().substring(srcKey.length())),
|
||||
copyFileContext));
|
||||
copiesToFinishes++;
|
||||
if (!copyFileContext.isCopySuccess()) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
priorLastKey = objectList.getPriorLastKey();
|
||||
} while (null != priorLastKey);
|
||||
|
||||
copyFileContext.lock();
|
||||
try {
|
||||
copyFileContext.awaitAllFinish(copiesToFinishes);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("interrupted when wait copies to finish");
|
||||
} finally {
|
||||
copyFileContext.lock();
|
||||
}
|
||||
|
||||
return copyFileContext.isCopySuccess();
|
||||
}
|
||||
|
||||
private void createParent(Path path) throws IOException {
|
||||
Path parent = path.getParent();
|
||||
if (parent != null) {
|
||||
String parentKey = pathToKey(parent);
|
||||
LOG.debug("Create parent key: {}", parentKey);
|
||||
if (!parentKey.equals(PATH_DELIMITER)) {
|
||||
String key = pathToKey(makeAbsolute(parent));
|
||||
if (key.length() > 0) {
|
||||
try {
|
||||
store.storeEmptyFile(key + PATH_DELIMITER);
|
||||
} catch (IOException e) {
|
||||
LOG.debug("Store a empty file in COS failed.", e);
|
||||
throw e;
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
@SuppressWarnings("deprecation")
|
||||
public long getDefaultBlockSize() {
|
||||
return getConf().getLong(
|
||||
CosNConfigKeys.COSN_BLOCK_SIZE_KEY,
|
||||
CosNConfigKeys.DEFAULT_BLOCK_SIZE);
|
||||
}
|
||||
|
||||
/**
|
||||
* Set the working directory to the given directory.
|
||||
*/
|
||||
@Override
|
||||
public void setWorkingDirectory(Path newDir) {
|
||||
workingDir = newDir;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getWorkingDirectory() {
|
||||
return workingDir;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getCanonicalServiceName() {
|
||||
// Does not support Token
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
try {
|
||||
this.store.close();
|
||||
this.boundedIOThreadPool.shutdown();
|
||||
this.boundedCopyThreadPool.shutdown();
|
||||
} finally {
|
||||
super.close();
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,365 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.EOFException;
|
||||
import java.io.IOException;
|
||||
import java.util.ArrayDeque;
|
||||
import java.util.Queue;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.util.concurrent.locks.Condition;
|
||||
import java.util.concurrent.locks.ReentrantLock;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSExceptionMessages;
|
||||
import org.apache.hadoop.fs.FSInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
|
||||
/**
|
||||
* The input stream for the COS blob store.
|
||||
* Optimized sequential read flow based on a forward read-ahead queue
|
||||
*/
|
||||
public class CosNInputStream extends FSInputStream {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(CosNInputStream.class);
|
||||
|
||||
/**
|
||||
* This class is used by {@link CosNInputStream}
|
||||
* and {@link CosNFileReadTask} to buffer data that read from COS blob store.
|
||||
*/
|
||||
public static class ReadBuffer {
|
||||
public static final int INIT = 1;
|
||||
public static final int SUCCESS = 0;
|
||||
public static final int ERROR = -1;
|
||||
|
||||
private final ReentrantLock lock = new ReentrantLock();
|
||||
private Condition readyCondition = lock.newCondition();
|
||||
|
||||
private byte[] buffer;
|
||||
private int status;
|
||||
private long start;
|
||||
private long end;
|
||||
|
||||
public ReadBuffer(long start, long end) {
|
||||
this.start = start;
|
||||
this.end = end;
|
||||
this.buffer = new byte[(int) (this.end - this.start) + 1];
|
||||
this.status = INIT;
|
||||
}
|
||||
|
||||
public void lock() {
|
||||
this.lock.lock();
|
||||
}
|
||||
|
||||
public void unLock() {
|
||||
this.lock.unlock();
|
||||
}
|
||||
|
||||
public void await(int waitStatus) throws InterruptedException {
|
||||
while (this.status == waitStatus) {
|
||||
readyCondition.await();
|
||||
}
|
||||
}
|
||||
|
||||
public void signalAll() {
|
||||
readyCondition.signalAll();
|
||||
}
|
||||
|
||||
public byte[] getBuffer() {
|
||||
return this.buffer;
|
||||
}
|
||||
|
||||
public int getStatus() {
|
||||
return this.status;
|
||||
}
|
||||
|
||||
public void setStatus(int status) {
|
||||
this.status = status;
|
||||
}
|
||||
|
||||
public long getStart() {
|
||||
return start;
|
||||
}
|
||||
|
||||
public long getEnd() {
|
||||
return end;
|
||||
}
|
||||
}
|
||||
|
||||
private FileSystem.Statistics statistics;
|
||||
private final Configuration conf;
|
||||
private final NativeFileSystemStore store;
|
||||
private final String key;
|
||||
private long position = 0;
|
||||
private long nextPos = 0;
|
||||
private long fileSize;
|
||||
private long partRemaining;
|
||||
private final long preReadPartSize;
|
||||
private final int maxReadPartNumber;
|
||||
private byte[] buffer;
|
||||
private boolean closed;
|
||||
|
||||
private final ExecutorService readAheadExecutorService;
|
||||
private final Queue<ReadBuffer> readBufferQueue;
|
||||
|
||||
public CosNInputStream(Configuration conf, NativeFileSystemStore store,
|
||||
FileSystem.Statistics statistics, String key, long fileSize,
|
||||
ExecutorService readAheadExecutorService) {
|
||||
super();
|
||||
this.conf = conf;
|
||||
this.store = store;
|
||||
this.statistics = statistics;
|
||||
this.key = key;
|
||||
this.fileSize = fileSize;
|
||||
this.preReadPartSize = conf.getLong(
|
||||
CosNConfigKeys.READ_AHEAD_BLOCK_SIZE_KEY,
|
||||
CosNConfigKeys.DEFAULT_READ_AHEAD_BLOCK_SIZE);
|
||||
this.maxReadPartNumber = conf.getInt(
|
||||
CosNConfigKeys.READ_AHEAD_QUEUE_SIZE,
|
||||
CosNConfigKeys.DEFAULT_READ_AHEAD_QUEUE_SIZE);
|
||||
|
||||
this.readAheadExecutorService = readAheadExecutorService;
|
||||
this.readBufferQueue = new ArrayDeque<>(this.maxReadPartNumber);
|
||||
this.closed = false;
|
||||
}
|
||||
|
||||
private synchronized void reopen(long pos) throws IOException {
|
||||
long partSize;
|
||||
|
||||
if (pos < 0) {
|
||||
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
|
||||
} else if (pos > this.fileSize) {
|
||||
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
|
||||
} else {
|
||||
if (pos + this.preReadPartSize > this.fileSize) {
|
||||
partSize = this.fileSize - pos;
|
||||
} else {
|
||||
partSize = this.preReadPartSize;
|
||||
}
|
||||
}
|
||||
|
||||
this.buffer = null;
|
||||
|
||||
boolean isRandomIO = true;
|
||||
if (pos == this.nextPos) {
|
||||
isRandomIO = false;
|
||||
} else {
|
||||
while (this.readBufferQueue.size() != 0) {
|
||||
if (this.readBufferQueue.element().getStart() != pos) {
|
||||
this.readBufferQueue.poll();
|
||||
} else {
|
||||
break;
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
this.nextPos = pos + partSize;
|
||||
|
||||
int currentBufferQueueSize = this.readBufferQueue.size();
|
||||
long lastByteStart;
|
||||
if (currentBufferQueueSize == 0) {
|
||||
lastByteStart = pos - partSize;
|
||||
} else {
|
||||
ReadBuffer[] readBuffers =
|
||||
this.readBufferQueue.toArray(
|
||||
new ReadBuffer[currentBufferQueueSize]);
|
||||
lastByteStart = readBuffers[currentBufferQueueSize - 1].getStart();
|
||||
}
|
||||
|
||||
int maxLen = this.maxReadPartNumber - currentBufferQueueSize;
|
||||
for (int i = 0; i < maxLen && i < (currentBufferQueueSize + 1) * 2; i++) {
|
||||
if (lastByteStart + partSize * (i + 1) > this.fileSize) {
|
||||
break;
|
||||
}
|
||||
|
||||
long byteStart = lastByteStart + partSize * (i + 1);
|
||||
long byteEnd = byteStart + partSize - 1;
|
||||
if (byteEnd >= this.fileSize) {
|
||||
byteEnd = this.fileSize - 1;
|
||||
}
|
||||
|
||||
ReadBuffer readBuffer = new ReadBuffer(byteStart, byteEnd);
|
||||
if (readBuffer.getBuffer().length == 0) {
|
||||
readBuffer.setStatus(ReadBuffer.SUCCESS);
|
||||
} else {
|
||||
this.readAheadExecutorService.execute(
|
||||
new CosNFileReadTask(
|
||||
this.conf, this.key, this.store, readBuffer));
|
||||
}
|
||||
|
||||
this.readBufferQueue.add(readBuffer);
|
||||
if (isRandomIO) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
ReadBuffer readBuffer = this.readBufferQueue.poll();
|
||||
if (null != readBuffer) {
|
||||
readBuffer.lock();
|
||||
try {
|
||||
readBuffer.await(ReadBuffer.INIT);
|
||||
if (readBuffer.getStatus() == ReadBuffer.ERROR) {
|
||||
this.buffer = null;
|
||||
} else {
|
||||
this.buffer = readBuffer.getBuffer();
|
||||
}
|
||||
} catch (InterruptedException e) {
|
||||
LOG.warn("An interrupted exception occurred "
|
||||
+ "when waiting a read buffer.");
|
||||
} finally {
|
||||
readBuffer.unLock();
|
||||
}
|
||||
}
|
||||
|
||||
if (null == this.buffer) {
|
||||
throw new IOException("Null IO stream");
|
||||
}
|
||||
|
||||
this.position = pos;
|
||||
this.partRemaining = partSize;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void seek(long pos) throws IOException {
|
||||
if (pos < 0) {
|
||||
throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK);
|
||||
}
|
||||
if (pos > this.fileSize) {
|
||||
throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF);
|
||||
}
|
||||
|
||||
if (this.position == pos) {
|
||||
return;
|
||||
}
|
||||
if (pos > position && pos < this.position + partRemaining) {
|
||||
long len = pos - this.position;
|
||||
this.position = pos;
|
||||
this.partRemaining -= len;
|
||||
} else {
|
||||
this.reopen(pos);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getPos() {
|
||||
return this.position;
|
||||
}
|
||||
|
||||
@Override
|
||||
public boolean seekToNewSource(long targetPos) {
|
||||
// Currently does not support to seek the offset of a new source
|
||||
return false;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read() throws IOException {
|
||||
if (this.closed) {
|
||||
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
|
||||
}
|
||||
|
||||
if (this.partRemaining <= 0 && this.position < this.fileSize) {
|
||||
this.reopen(this.position);
|
||||
}
|
||||
|
||||
int byteRead = -1;
|
||||
if (this.partRemaining != 0) {
|
||||
byteRead = this.buffer[
|
||||
(int) (this.buffer.length - this.partRemaining)] & 0xff;
|
||||
}
|
||||
if (byteRead >= 0) {
|
||||
this.position++;
|
||||
this.partRemaining--;
|
||||
if (null != this.statistics) {
|
||||
this.statistics.incrementBytesRead(byteRead);
|
||||
}
|
||||
}
|
||||
|
||||
return byteRead;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int read(byte[] b, int off, int len) throws IOException {
|
||||
if (this.closed) {
|
||||
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
|
||||
}
|
||||
|
||||
if (len == 0) {
|
||||
return 0;
|
||||
}
|
||||
|
||||
if (off < 0 || len < 0 || len > b.length) {
|
||||
throw new IndexOutOfBoundsException();
|
||||
}
|
||||
|
||||
int bytesRead = 0;
|
||||
while (position < fileSize && bytesRead < len) {
|
||||
if (partRemaining <= 0) {
|
||||
reopen(position);
|
||||
}
|
||||
|
||||
int bytes = 0;
|
||||
for (int i = this.buffer.length - (int) partRemaining;
|
||||
i < this.buffer.length; i++) {
|
||||
b[off + bytesRead] = this.buffer[i];
|
||||
bytes++;
|
||||
bytesRead++;
|
||||
if (off + bytesRead >= len) {
|
||||
break;
|
||||
}
|
||||
}
|
||||
|
||||
if (bytes > 0) {
|
||||
this.position += bytes;
|
||||
this.partRemaining -= bytes;
|
||||
} else if (this.partRemaining != 0) {
|
||||
throw new IOException(
|
||||
"Failed to read from stream. Remaining: " + this.partRemaining);
|
||||
}
|
||||
}
|
||||
if (null != this.statistics && bytesRead > 0) {
|
||||
this.statistics.incrementBytesRead(bytesRead);
|
||||
}
|
||||
|
||||
return bytesRead == 0 ? -1 : bytesRead;
|
||||
}
|
||||
|
||||
@Override
|
||||
public int available() throws IOException {
|
||||
if (this.closed) {
|
||||
throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED);
|
||||
}
|
||||
|
||||
long remaining = this.fileSize - this.position;
|
||||
if (remaining > Integer.MAX_VALUE) {
|
||||
return Integer.MAX_VALUE;
|
||||
}
|
||||
return (int)remaining;
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (this.closed) {
|
||||
return;
|
||||
}
|
||||
this.closed = true;
|
||||
this.buffer = null;
|
||||
}
|
||||
}
|
@ -0,0 +1,284 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.util.HashSet;
|
||||
import java.util.LinkedList;
|
||||
import java.util.List;
|
||||
import java.util.Set;
|
||||
import java.util.concurrent.Callable;
|
||||
import java.util.concurrent.ExecutionException;
|
||||
import java.util.concurrent.ExecutorService;
|
||||
import java.io.IOException;
|
||||
import java.io.OutputStream;
|
||||
import java.security.DigestOutputStream;
|
||||
import java.security.MessageDigest;
|
||||
import java.security.NoSuchAlgorithmException;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import com.google.common.util.concurrent.Futures;
|
||||
import com.google.common.util.concurrent.ListenableFuture;
|
||||
import com.google.common.util.concurrent.ListeningExecutorService;
|
||||
import com.google.common.util.concurrent.MoreExecutors;
|
||||
import com.qcloud.cos.model.PartETag;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* The output stream for the COS blob store.
|
||||
* Implement streaming upload to COS based on the multipart upload function.
|
||||
* ( the maximum size of each part is 5GB)
|
||||
* Support up to 40TB single file by multipart upload (each part is 5GB).
|
||||
* Improve the upload performance of writing large files by using byte buffers
|
||||
* and a fixed thread pool.
|
||||
*/
|
||||
public class CosNOutputStream extends OutputStream {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(CosNOutputStream.class);
|
||||
|
||||
private final Configuration conf;
|
||||
private final NativeFileSystemStore store;
|
||||
private MessageDigest digest;
|
||||
private long blockSize;
|
||||
private String key;
|
||||
private int currentBlockId = 0;
|
||||
private Set<ByteBufferWrapper> blockCacheBuffers = new HashSet<>();
|
||||
private ByteBufferWrapper currentBlockBuffer;
|
||||
private OutputStream currentBlockOutputStream;
|
||||
private String uploadId = null;
|
||||
private ListeningExecutorService executorService;
|
||||
private List<ListenableFuture<PartETag>> etagList = new LinkedList<>();
|
||||
private int blockWritten = 0;
|
||||
private boolean closed = false;
|
||||
|
||||
public CosNOutputStream(Configuration conf, NativeFileSystemStore store,
|
||||
String key, long blockSize, ExecutorService executorService)
|
||||
throws IOException {
|
||||
this.conf = conf;
|
||||
this.store = store;
|
||||
this.key = key;
|
||||
this.blockSize = blockSize;
|
||||
if (this.blockSize < Constants.MIN_PART_SIZE) {
|
||||
LOG.warn(
|
||||
String.format(
|
||||
"The minimum size of a single block is limited to %d.",
|
||||
Constants.MIN_PART_SIZE));
|
||||
this.blockSize = Constants.MIN_PART_SIZE;
|
||||
}
|
||||
if (this.blockSize > Constants.MAX_PART_SIZE) {
|
||||
LOG.warn(
|
||||
String.format(
|
||||
"The maximum size of a single block is limited to %d.",
|
||||
Constants.MAX_PART_SIZE));
|
||||
this.blockSize = Constants.MAX_PART_SIZE;
|
||||
}
|
||||
|
||||
// Use a blocking thread pool with fair scheduling
|
||||
this.executorService = MoreExecutors.listeningDecorator(executorService);
|
||||
|
||||
try {
|
||||
this.currentBlockBuffer =
|
||||
BufferPool.getInstance().getBuffer((int) this.blockSize);
|
||||
} catch (IOException e) {
|
||||
throw new IOException("Getting a buffer size: "
|
||||
+ String.valueOf(this.blockSize)
|
||||
+ " from buffer pool occurs an exception: ", e);
|
||||
}
|
||||
|
||||
try {
|
||||
this.digest = MessageDigest.getInstance("MD5");
|
||||
this.currentBlockOutputStream = new DigestOutputStream(
|
||||
new ByteBufferOutputStream(this.currentBlockBuffer.getByteBuffer()),
|
||||
this.digest);
|
||||
} catch (NoSuchAlgorithmException e) {
|
||||
this.digest = null;
|
||||
this.currentBlockOutputStream =
|
||||
new ByteBufferOutputStream(this.currentBlockBuffer.getByteBuffer());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void flush() throws IOException {
|
||||
this.currentBlockOutputStream.flush();
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized void close() throws IOException {
|
||||
if (this.closed) {
|
||||
return;
|
||||
}
|
||||
this.currentBlockOutputStream.flush();
|
||||
this.currentBlockOutputStream.close();
|
||||
LOG.info("The output stream has been close, and "
|
||||
+ "begin to upload the last block: [{}].", this.currentBlockId);
|
||||
this.blockCacheBuffers.add(this.currentBlockBuffer);
|
||||
if (this.blockCacheBuffers.size() == 1) {
|
||||
byte[] md5Hash = this.digest == null ? null : this.digest.digest();
|
||||
store.storeFile(this.key,
|
||||
new ByteBufferInputStream(this.currentBlockBuffer.getByteBuffer()),
|
||||
md5Hash, this.currentBlockBuffer.getByteBuffer().remaining());
|
||||
} else {
|
||||
PartETag partETag = null;
|
||||
if (this.blockWritten > 0) {
|
||||
LOG.info("Upload the last part..., blockId: [{}], written bytes: [{}]",
|
||||
this.currentBlockId, this.blockWritten);
|
||||
partETag = store.uploadPart(
|
||||
new ByteBufferInputStream(currentBlockBuffer.getByteBuffer()),
|
||||
key, uploadId, currentBlockId + 1,
|
||||
currentBlockBuffer.getByteBuffer().remaining());
|
||||
}
|
||||
final List<PartETag> futurePartETagList = this.waitForFinishPartUploads();
|
||||
if (null == futurePartETagList) {
|
||||
throw new IOException("Failed to multipart upload to cos, abort it.");
|
||||
}
|
||||
List<PartETag> tmpPartEtagList = new LinkedList<>(futurePartETagList);
|
||||
if (null != partETag) {
|
||||
tmpPartEtagList.add(partETag);
|
||||
}
|
||||
store.completeMultipartUpload(this.key, this.uploadId, tmpPartEtagList);
|
||||
}
|
||||
try {
|
||||
BufferPool.getInstance().returnBuffer(this.currentBlockBuffer);
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("An exception occurred "
|
||||
+ "while returning the buffer to the buffer pool.", e);
|
||||
}
|
||||
LOG.info("The outputStream for key: [{}] has been uploaded.", key);
|
||||
this.blockWritten = 0;
|
||||
this.closed = true;
|
||||
}
|
||||
|
||||
private List<PartETag> waitForFinishPartUploads() throws IOException {
|
||||
try {
|
||||
LOG.info("Wait for all parts to finish their uploading.");
|
||||
return Futures.allAsList(this.etagList).get();
|
||||
} catch (InterruptedException e) {
|
||||
LOG.error("Interrupt the part upload.", e);
|
||||
return null;
|
||||
} catch (ExecutionException e) {
|
||||
LOG.error("Cancelling futures.");
|
||||
for (ListenableFuture<PartETag> future : this.etagList) {
|
||||
future.cancel(true);
|
||||
}
|
||||
(store).abortMultipartUpload(this.key, this.uploadId);
|
||||
LOG.error("Multipart upload with id: [{}] to COS key: [{}]",
|
||||
this.uploadId, this.key, e);
|
||||
throw new IOException("Multipart upload with id: "
|
||||
+ this.uploadId + " to " + this.key, e);
|
||||
}
|
||||
}
|
||||
|
||||
private void uploadPart() throws IOException {
|
||||
this.currentBlockOutputStream.flush();
|
||||
this.currentBlockOutputStream.close();
|
||||
this.blockCacheBuffers.add(this.currentBlockBuffer);
|
||||
|
||||
if (this.currentBlockId == 0) {
|
||||
uploadId = (store).getUploadId(key);
|
||||
}
|
||||
|
||||
ListenableFuture<PartETag> partETagListenableFuture =
|
||||
this.executorService.submit(
|
||||
new Callable<PartETag>() {
|
||||
private final ByteBufferWrapper buf = currentBlockBuffer;
|
||||
private final String localKey = key;
|
||||
private final String localUploadId = uploadId;
|
||||
private final int blockId = currentBlockId;
|
||||
|
||||
@Override
|
||||
public PartETag call() throws Exception {
|
||||
if (LOG.isDebugEnabled()) {
|
||||
LOG.debug("{} is uploading a part.",
|
||||
Thread.currentThread().getName());
|
||||
}
|
||||
PartETag partETag = (store).uploadPart(
|
||||
new ByteBufferInputStream(this.buf.getByteBuffer()),
|
||||
this.localKey, this.localUploadId,
|
||||
this.blockId + 1, this.buf.getByteBuffer().remaining());
|
||||
BufferPool.getInstance().returnBuffer(this.buf);
|
||||
return partETag;
|
||||
}
|
||||
});
|
||||
this.etagList.add(partETagListenableFuture);
|
||||
try {
|
||||
this.currentBlockBuffer =
|
||||
BufferPool.getInstance().getBuffer((int) this.blockSize);
|
||||
} catch (IOException e) {
|
||||
String errMsg = String.format("Getting a buffer [size:%d] from "
|
||||
+ "the buffer pool failed.", this.blockSize);
|
||||
throw new IOException(errMsg, e);
|
||||
}
|
||||
this.currentBlockId++;
|
||||
if (null != this.digest) {
|
||||
this.digest.reset();
|
||||
this.currentBlockOutputStream = new DigestOutputStream(
|
||||
new ByteBufferOutputStream(this.currentBlockBuffer.getByteBuffer()),
|
||||
this.digest);
|
||||
} else {
|
||||
this.currentBlockOutputStream =
|
||||
new ByteBufferOutputStream(this.currentBlockBuffer.getByteBuffer());
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] b, int off, int len) throws IOException {
|
||||
if (this.closed) {
|
||||
throw new IOException("block stream has been closed.");
|
||||
}
|
||||
|
||||
while (len > 0) {
|
||||
long writeBytes;
|
||||
if (this.blockWritten + len > this.blockSize) {
|
||||
writeBytes = this.blockSize - this.blockWritten;
|
||||
} else {
|
||||
writeBytes = len;
|
||||
}
|
||||
|
||||
this.currentBlockOutputStream.write(b, off, (int) writeBytes);
|
||||
this.blockWritten += writeBytes;
|
||||
if (this.blockWritten >= this.blockSize) {
|
||||
this.uploadPart();
|
||||
this.blockWritten = 0;
|
||||
}
|
||||
len -= writeBytes;
|
||||
off += writeBytes;
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(byte[] b) throws IOException {
|
||||
this.write(b, 0, b.length);
|
||||
}
|
||||
|
||||
@Override
|
||||
public void write(int b) throws IOException {
|
||||
if (this.closed) {
|
||||
throw new IOException("block stream has been closed.");
|
||||
}
|
||||
|
||||
byte[] singleBytes = new byte[1];
|
||||
singleBytes[0] = (byte) b;
|
||||
this.currentBlockOutputStream.write(singleBytes, 0, 1);
|
||||
this.blockWritten += 1;
|
||||
if (this.blockWritten >= this.blockSize) {
|
||||
this.uploadPart();
|
||||
this.blockWritten = 0;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,167 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.lang.reflect.Constructor;
|
||||
import java.lang.reflect.InvocationTargetException;
|
||||
import java.lang.reflect.Method;
|
||||
import java.lang.reflect.Modifier;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
import com.qcloud.cos.auth.COSCredentialsProvider;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.cosn.auth.COSCredentialProviderList;
|
||||
import org.apache.hadoop.fs.cosn.auth.EnvironmentVariableCredentialProvider;
|
||||
import org.apache.hadoop.fs.cosn.auth.SimpleCredentialProvider;
|
||||
|
||||
/**
|
||||
* Utility methods for CosN code.
|
||||
*/
|
||||
public final class CosNUtils {
|
||||
private static final Logger LOG = LoggerFactory.getLogger(CosNUtils.class);
|
||||
|
||||
static final String INSTANTIATION_EXCEPTION
|
||||
= "instantiation exception";
|
||||
static final String NOT_COS_CREDENTIAL_PROVIDER
|
||||
= "is not cos credential provider";
|
||||
static final String ABSTRACT_CREDENTIAL_PROVIDER
|
||||
= "is abstract and therefore cannot be created";
|
||||
|
||||
private CosNUtils() {
|
||||
}
|
||||
|
||||
public static COSCredentialProviderList createCosCredentialsProviderSet(
|
||||
Configuration conf) throws IOException {
|
||||
COSCredentialProviderList credentialProviderList =
|
||||
new COSCredentialProviderList();
|
||||
|
||||
Class<?>[] cosClasses = CosNUtils.loadCosProviderClasses(
|
||||
conf,
|
||||
CosNConfigKeys.COSN_CREDENTIALS_PROVIDER);
|
||||
if (0 == cosClasses.length) {
|
||||
credentialProviderList.add(new SimpleCredentialProvider(conf));
|
||||
credentialProviderList.add(new EnvironmentVariableCredentialProvider());
|
||||
} else {
|
||||
for (Class<?> credClass : cosClasses) {
|
||||
credentialProviderList.add(createCOSCredentialProvider(
|
||||
conf,
|
||||
credClass));
|
||||
}
|
||||
}
|
||||
|
||||
return credentialProviderList;
|
||||
}
|
||||
|
||||
public static Class<?>[] loadCosProviderClasses(
|
||||
Configuration conf,
|
||||
String key,
|
||||
Class<?>... defaultValue) throws IOException {
|
||||
try {
|
||||
return conf.getClasses(key, defaultValue);
|
||||
} catch (RuntimeException e) {
|
||||
Throwable c = e.getCause() != null ? e.getCause() : e;
|
||||
throw new IOException("From option " + key + ' ' + c, c);
|
||||
}
|
||||
}
|
||||
|
||||
public static COSCredentialsProvider createCOSCredentialProvider(
|
||||
Configuration conf,
|
||||
Class<?> credClass) throws IOException {
|
||||
COSCredentialsProvider credentialsProvider;
|
||||
if (!COSCredentialsProvider.class.isAssignableFrom(credClass)) {
|
||||
throw new IllegalArgumentException(
|
||||
"class " + credClass + " " + NOT_COS_CREDENTIAL_PROVIDER);
|
||||
}
|
||||
if (Modifier.isAbstract(credClass.getModifiers())) {
|
||||
throw new IllegalArgumentException(
|
||||
"class " + credClass + " " + ABSTRACT_CREDENTIAL_PROVIDER);
|
||||
}
|
||||
LOG.debug("Credential Provider class: " + credClass.getName());
|
||||
|
||||
try {
|
||||
// new credClass()
|
||||
Constructor constructor = getConstructor(credClass);
|
||||
if (constructor != null) {
|
||||
credentialsProvider =
|
||||
(COSCredentialsProvider) constructor.newInstance();
|
||||
return credentialsProvider;
|
||||
}
|
||||
// new credClass(conf)
|
||||
constructor = getConstructor(credClass, Configuration.class);
|
||||
if (null != constructor) {
|
||||
credentialsProvider =
|
||||
(COSCredentialsProvider) constructor.newInstance(conf);
|
||||
return credentialsProvider;
|
||||
}
|
||||
|
||||
Method factory = getFactoryMethod(
|
||||
credClass, COSCredentialsProvider.class, "getInstance");
|
||||
if (null != factory) {
|
||||
credentialsProvider = (COSCredentialsProvider) factory.invoke(null);
|
||||
return credentialsProvider;
|
||||
}
|
||||
|
||||
throw new IllegalArgumentException(
|
||||
"Not supported constructor or factory method found"
|
||||
);
|
||||
|
||||
} catch (IllegalAccessException e) {
|
||||
throw new IOException(
|
||||
credClass.getName() + " " + INSTANTIATION_EXCEPTION + ": " + e, e);
|
||||
} catch (InstantiationException e) {
|
||||
throw new IOException(
|
||||
credClass.getName() + " " + INSTANTIATION_EXCEPTION + ": " + e, e);
|
||||
} catch (InvocationTargetException e) {
|
||||
Throwable targetException = e.getTargetException();
|
||||
if (targetException == null) {
|
||||
targetException = e;
|
||||
}
|
||||
throw new IOException(
|
||||
credClass.getName() + " " + INSTANTIATION_EXCEPTION + ": "
|
||||
+ targetException, targetException);
|
||||
}
|
||||
}
|
||||
|
||||
private static Constructor<?> getConstructor(Class<?> cl, Class<?>... args) {
|
||||
try {
|
||||
Constructor constructor = cl.getDeclaredConstructor(args);
|
||||
return Modifier.isPublic(constructor.getModifiers()) ? constructor : null;
|
||||
} catch (NoSuchMethodException e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
|
||||
private static Method getFactoryMethod(
|
||||
Class<?> cl, Class<?> returnType, String methodName) {
|
||||
try {
|
||||
Method m = cl.getDeclaredMethod(methodName);
|
||||
if (Modifier.isPublic(m.getModifiers())
|
||||
&& Modifier.isStatic(m.getModifiers())
|
||||
&& returnType.isAssignableFrom(m.getReturnType())) {
|
||||
return m;
|
||||
} else {
|
||||
return null;
|
||||
}
|
||||
} catch (NoSuchMethodException e) {
|
||||
return null;
|
||||
}
|
||||
}
|
||||
}
|
@ -0,0 +1,768 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.BufferedInputStream;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.File;
|
||||
import java.io.FileInputStream;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collections;
|
||||
import java.util.Comparator;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.ThreadLocalRandom;
|
||||
|
||||
import com.qcloud.cos.COSClient;
|
||||
import com.qcloud.cos.ClientConfig;
|
||||
import com.qcloud.cos.auth.BasicCOSCredentials;
|
||||
import com.qcloud.cos.auth.COSCredentials;
|
||||
import com.qcloud.cos.exception.CosClientException;
|
||||
import com.qcloud.cos.exception.CosServiceException;
|
||||
import com.qcloud.cos.http.HttpProtocol;
|
||||
import com.qcloud.cos.model.AbortMultipartUploadRequest;
|
||||
import com.qcloud.cos.model.COSObject;
|
||||
import com.qcloud.cos.model.COSObjectSummary;
|
||||
import com.qcloud.cos.model.CompleteMultipartUploadRequest;
|
||||
import com.qcloud.cos.model.CompleteMultipartUploadResult;
|
||||
import com.qcloud.cos.model.CopyObjectRequest;
|
||||
import com.qcloud.cos.model.DeleteObjectRequest;
|
||||
import com.qcloud.cos.model.GetObjectMetadataRequest;
|
||||
import com.qcloud.cos.model.GetObjectRequest;
|
||||
import com.qcloud.cos.model.InitiateMultipartUploadRequest;
|
||||
import com.qcloud.cos.model.InitiateMultipartUploadResult;
|
||||
import com.qcloud.cos.model.ListObjectsRequest;
|
||||
import com.qcloud.cos.model.ObjectListing;
|
||||
import com.qcloud.cos.model.ObjectMetadata;
|
||||
import com.qcloud.cos.model.PartETag;
|
||||
import com.qcloud.cos.model.PutObjectRequest;
|
||||
import com.qcloud.cos.model.PutObjectResult;
|
||||
import com.qcloud.cos.model.UploadPartRequest;
|
||||
import com.qcloud.cos.model.UploadPartResult;
|
||||
import com.qcloud.cos.region.Region;
|
||||
import com.qcloud.cos.utils.Base64;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.cosn.auth.COSCredentialProviderList;
|
||||
import org.apache.hadoop.util.VersionInfo;
|
||||
import org.apache.http.HttpStatus;
|
||||
|
||||
/**
|
||||
* The class actually performs access operation to the COS blob store.
|
||||
* It provides the bridging logic for the Hadoop's abstract filesystem and COS.
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
class CosNativeFileSystemStore implements NativeFileSystemStore {
|
||||
private COSClient cosClient;
|
||||
private String bucketName;
|
||||
private int maxRetryTimes;
|
||||
|
||||
public static final Logger LOG =
|
||||
LoggerFactory.getLogger(CosNativeFileSystemStore.class);
|
||||
|
||||
/**
|
||||
* Initialize the client to access COS blob storage.
|
||||
*
|
||||
* @param conf Hadoop configuration with COS configuration options.
|
||||
* @throws IOException Initialize the COS client failed,
|
||||
* caused by incorrect options.
|
||||
*/
|
||||
private void initCOSClient(Configuration conf) throws IOException {
|
||||
COSCredentialProviderList credentialProviderList =
|
||||
CosNUtils.createCosCredentialsProviderSet(conf);
|
||||
String region = conf.get(CosNConfigKeys.COSN_REGION_KEY);
|
||||
String endpointSuffix = conf.get(
|
||||
CosNConfigKeys.COSN_ENDPOINT_SUFFIX_KEY);
|
||||
if (null == region && null == endpointSuffix) {
|
||||
String exceptionMsg = String.format("config %s and %s at least one",
|
||||
CosNConfigKeys.COSN_REGION_KEY,
|
||||
CosNConfigKeys.COSN_ENDPOINT_SUFFIX_KEY);
|
||||
throw new IOException(exceptionMsg);
|
||||
}
|
||||
|
||||
COSCredentials cosCred;
|
||||
cosCred = new BasicCOSCredentials(
|
||||
credentialProviderList.getCredentials().getCOSAccessKeyId(),
|
||||
credentialProviderList.getCredentials().getCOSSecretKey());
|
||||
|
||||
boolean useHttps = conf.getBoolean(CosNConfigKeys.COSN_USE_HTTPS_KEY,
|
||||
CosNConfigKeys.DEFAULT_USE_HTTPS);
|
||||
|
||||
ClientConfig config;
|
||||
if (null == region) {
|
||||
config = new ClientConfig(new Region(""));
|
||||
config.setEndPointSuffix(endpointSuffix);
|
||||
} else {
|
||||
config = new ClientConfig(new Region(region));
|
||||
}
|
||||
if (useHttps) {
|
||||
config.setHttpProtocol(HttpProtocol.https);
|
||||
}
|
||||
|
||||
config.setUserAgent(conf.get(CosNConfigKeys.USER_AGENT,
|
||||
CosNConfigKeys.DEFAULT_USER_AGENT) + " For " + " Hadoop "
|
||||
+ VersionInfo.getVersion());
|
||||
|
||||
this.maxRetryTimes = conf.getInt(CosNConfigKeys.COSN_MAX_RETRIES_KEY,
|
||||
CosNConfigKeys.DEFAULT_MAX_RETRIES);
|
||||
|
||||
config.setMaxConnectionsCount(
|
||||
conf.getInt(CosNConfigKeys.MAX_CONNECTION_NUM,
|
||||
CosNConfigKeys.DEFAULT_MAX_CONNECTION_NUM));
|
||||
|
||||
this.cosClient = new COSClient(cosCred, config);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize the CosNativeFileSystemStore object, including
|
||||
* its COS client and default COS bucket.
|
||||
*
|
||||
* @param uri The URI of the COS bucket accessed by default.
|
||||
* @param conf Hadoop configuration with COS configuration options.
|
||||
* @throws IOException Initialize the COS client failed.
|
||||
*/
|
||||
@Override
|
||||
public void initialize(URI uri, Configuration conf) throws IOException {
|
||||
try {
|
||||
initCOSClient(conf);
|
||||
this.bucketName = uri.getHost();
|
||||
} catch (Exception e) {
|
||||
handleException(e, "");
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Store a file into COS from the specified input stream, which would be
|
||||
* retried until the success or maximum number.
|
||||
*
|
||||
* @param key COS object key.
|
||||
* @param inputStream Input stream to be uploaded into COS.
|
||||
* @param md5Hash MD5 value of the content to be uploaded.
|
||||
* @param length Length of uploaded content.
|
||||
* @throws IOException Upload the file failed.
|
||||
*/
|
||||
private void storeFileWithRetry(String key, InputStream inputStream,
|
||||
byte[] md5Hash, long length) throws IOException {
|
||||
try {
|
||||
ObjectMetadata objectMetadata = new ObjectMetadata();
|
||||
objectMetadata.setContentMD5(Base64.encodeAsString(md5Hash));
|
||||
objectMetadata.setContentLength(length);
|
||||
PutObjectRequest putObjectRequest =
|
||||
new PutObjectRequest(bucketName, key, inputStream, objectMetadata);
|
||||
|
||||
PutObjectResult putObjectResult =
|
||||
(PutObjectResult) callCOSClientWithRetry(putObjectRequest);
|
||||
LOG.debug("Store file successfully. COS key: [{}], ETag: [{}], "
|
||||
+ "MD5: [{}].", key, putObjectResult.getETag(), new String(md5Hash));
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Store file failed. COS key: [%s], "
|
||||
+ "exception: [%s]", key, e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), key);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Store a local file into COS.
|
||||
*
|
||||
* @param key COS object key.
|
||||
* @param file The local file to be uploaded.
|
||||
* @param md5Hash The MD5 value of the file to be uploaded.
|
||||
* @throws IOException Upload the file failed.
|
||||
*/
|
||||
@Override
|
||||
public void storeFile(String key, File file, byte[] md5Hash)
|
||||
throws IOException {
|
||||
LOG.info("Store file from local path: [{}]. file length: [{}] COS key: " +
|
||||
"[{}] MD5: [{}].", file.getCanonicalPath(), file.length(), key,
|
||||
new String(md5Hash));
|
||||
storeFileWithRetry(key, new BufferedInputStream(new FileInputStream(file)),
|
||||
md5Hash, file.length());
|
||||
}
|
||||
|
||||
/**
|
||||
* Store a file into COS from the specified input stream.
|
||||
*
|
||||
* @param key COS object key.
|
||||
* @param inputStream The Input stream to be uploaded.
|
||||
* @param md5Hash The MD5 value of the content to be uploaded.
|
||||
* @param contentLength Length of uploaded content.
|
||||
* @throws IOException Upload the file failed.
|
||||
*/
|
||||
@Override
|
||||
public void storeFile(
|
||||
String key,
|
||||
InputStream inputStream,
|
||||
byte[] md5Hash,
|
||||
long contentLength) throws IOException {
|
||||
LOG.info("Store file from input stream. COS key: [{}], "
|
||||
+ "length: [{}], MD5: [{}].", key, contentLength, md5Hash);
|
||||
storeFileWithRetry(key, inputStream, md5Hash, contentLength);
|
||||
}
|
||||
|
||||
// For cos, storeEmptyFile means creating a directory
|
||||
@Override
|
||||
public void storeEmptyFile(String key) throws IOException {
|
||||
if (!key.endsWith(CosNFileSystem.PATH_DELIMITER)) {
|
||||
key = key + CosNFileSystem.PATH_DELIMITER;
|
||||
}
|
||||
|
||||
ObjectMetadata objectMetadata = new ObjectMetadata();
|
||||
objectMetadata.setContentLength(0);
|
||||
InputStream input = new ByteArrayInputStream(new byte[0]);
|
||||
PutObjectRequest putObjectRequest =
|
||||
new PutObjectRequest(bucketName, key, input, objectMetadata);
|
||||
try {
|
||||
PutObjectResult putObjectResult =
|
||||
(PutObjectResult) callCOSClientWithRetry(putObjectRequest);
|
||||
LOG.debug("Store empty file successfully. COS key: [{}], ETag: [{}].",
|
||||
key, putObjectResult.getETag());
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Store empty file failed. "
|
||||
+ "COS key: [%s], exception: [%s]", key, e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), key);
|
||||
}
|
||||
}
|
||||
|
||||
public PartETag uploadPart(File file, String key, String uploadId,
|
||||
int partNum) throws IOException {
|
||||
InputStream inputStream = new FileInputStream(file);
|
||||
return uploadPart(inputStream, key, uploadId, partNum, file.length());
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartETag uploadPart(InputStream inputStream, String key,
|
||||
String uploadId, int partNum, long partSize) throws IOException {
|
||||
UploadPartRequest uploadPartRequest = new UploadPartRequest();
|
||||
uploadPartRequest.setBucketName(this.bucketName);
|
||||
uploadPartRequest.setUploadId(uploadId);
|
||||
uploadPartRequest.setInputStream(inputStream);
|
||||
uploadPartRequest.setPartNumber(partNum);
|
||||
uploadPartRequest.setPartSize(partSize);
|
||||
uploadPartRequest.setKey(key);
|
||||
|
||||
try {
|
||||
UploadPartResult uploadPartResult =
|
||||
(UploadPartResult) callCOSClientWithRetry(uploadPartRequest);
|
||||
return uploadPartResult.getPartETag();
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Current thread: [%d], COS key: [%s], "
|
||||
+ "upload id: [%s], part num: [%d], exception: [%s]",
|
||||
Thread.currentThread().getId(), key, uploadId, partNum, e.toString());
|
||||
handleException(new Exception(errMsg), key);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
public void abortMultipartUpload(String key, String uploadId) {
|
||||
LOG.info("Abort the multipart upload. COS key: [{}], upload id: [{}].",
|
||||
key, uploadId);
|
||||
AbortMultipartUploadRequest abortMultipartUploadRequest =
|
||||
new AbortMultipartUploadRequest(bucketName, key, uploadId);
|
||||
cosClient.abortMultipartUpload(abortMultipartUploadRequest);
|
||||
}
|
||||
|
||||
/**
|
||||
* Initialize a multipart upload and return the upload id.
|
||||
*
|
||||
* @param key The COS object key initialized to multipart upload.
|
||||
* @return The multipart upload id.
|
||||
*/
|
||||
public String getUploadId(String key) {
|
||||
if (null == key || key.length() == 0) {
|
||||
return "";
|
||||
}
|
||||
|
||||
LOG.info("Initiate a multipart upload. bucket: [{}], COS key: [{}].",
|
||||
bucketName, key);
|
||||
InitiateMultipartUploadRequest initiateMultipartUploadRequest =
|
||||
new InitiateMultipartUploadRequest(bucketName, key);
|
||||
InitiateMultipartUploadResult initiateMultipartUploadResult =
|
||||
cosClient.initiateMultipartUpload(initiateMultipartUploadRequest);
|
||||
return initiateMultipartUploadResult.getUploadId();
|
||||
}
|
||||
|
||||
/**
|
||||
* Finish a multipart upload process, which will merge all parts uploaded.
|
||||
*
|
||||
* @param key The COS object key to be finished.
|
||||
* @param uploadId The upload id of the multipart upload to be finished.
|
||||
* @param partETagList The etag list of the part that has been uploaded.
|
||||
* @return The result object of completing the multipart upload process.
|
||||
*/
|
||||
public CompleteMultipartUploadResult completeMultipartUpload(
|
||||
String key, String uploadId, List<PartETag> partETagList) {
|
||||
Collections.sort(partETagList, new Comparator<PartETag>() {
|
||||
@Override
|
||||
public int compare(PartETag o1, PartETag o2) {
|
||||
return o1.getPartNumber() - o2.getPartNumber();
|
||||
}
|
||||
});
|
||||
LOG.info("Complete the multipart upload. bucket: [{}], COS key: [{}], "
|
||||
+ "upload id: [{}].", bucketName, key, uploadId);
|
||||
CompleteMultipartUploadRequest completeMultipartUploadRequest =
|
||||
new CompleteMultipartUploadRequest(
|
||||
bucketName, key, uploadId, partETagList);
|
||||
return cosClient.completeMultipartUpload(completeMultipartUploadRequest);
|
||||
}
|
||||
|
||||
private FileMetadata queryObjectMetadata(String key) throws IOException {
|
||||
GetObjectMetadataRequest getObjectMetadataRequest =
|
||||
new GetObjectMetadataRequest(bucketName, key);
|
||||
try {
|
||||
ObjectMetadata objectMetadata =
|
||||
(ObjectMetadata) callCOSClientWithRetry(getObjectMetadataRequest);
|
||||
long mtime = 0;
|
||||
if (objectMetadata.getLastModified() != null) {
|
||||
mtime = objectMetadata.getLastModified().getTime();
|
||||
}
|
||||
long fileSize = objectMetadata.getContentLength();
|
||||
FileMetadata fileMetadata = new FileMetadata(key, fileSize, mtime,
|
||||
!key.endsWith(CosNFileSystem.PATH_DELIMITER));
|
||||
LOG.debug("Retrieve file metadata. COS key: [{}], ETag: [{}], "
|
||||
+ "length: [{}].", key, objectMetadata.getETag(),
|
||||
objectMetadata.getContentLength());
|
||||
return fileMetadata;
|
||||
} catch (CosServiceException e) {
|
||||
if (e.getStatusCode() != HttpStatus.SC_NOT_FOUND) {
|
||||
String errorMsg = String.format("Retrieve file metadata file failed. "
|
||||
+ "COS key: [%s], CosServiceException: [%s].", key, e.toString());
|
||||
LOG.error(errorMsg);
|
||||
handleException(new Exception(errorMsg), key);
|
||||
}
|
||||
}
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public FileMetadata retrieveMetadata(String key) throws IOException {
|
||||
if (key.endsWith(CosNFileSystem.PATH_DELIMITER)) {
|
||||
key = key.substring(0, key.length() - 1);
|
||||
}
|
||||
|
||||
if (!key.isEmpty()) {
|
||||
FileMetadata fileMetadata = queryObjectMetadata(key);
|
||||
if (fileMetadata != null) {
|
||||
return fileMetadata;
|
||||
}
|
||||
}
|
||||
|
||||
// If the key is a directory.
|
||||
key = key + CosNFileSystem.PATH_DELIMITER;
|
||||
return queryObjectMetadata(key);
|
||||
}
|
||||
|
||||
/**
|
||||
* Download a COS object and return the input stream associated with it.
|
||||
*
|
||||
* @param key The object key that is being retrieved from the COS bucket
|
||||
* @return This method returns null if the key is not found
|
||||
* @throws IOException if failed to download.
|
||||
*/
|
||||
@Override
|
||||
public InputStream retrieve(String key) throws IOException {
|
||||
LOG.debug("Retrieve object key: [{}].", key);
|
||||
GetObjectRequest getObjectRequest =
|
||||
new GetObjectRequest(this.bucketName, key);
|
||||
try {
|
||||
COSObject cosObject =
|
||||
(COSObject) callCOSClientWithRetry(getObjectRequest);
|
||||
return cosObject.getObjectContent();
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Retrieving key: [%s] occurs "
|
||||
+ "an exception: [%s].", key, e.toString());
|
||||
LOG.error("Retrieving COS key: [{}] occurs an exception: [{}].", key, e);
|
||||
handleException(new Exception(errMsg), key);
|
||||
}
|
||||
// never will get here
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Retrieved a part of a COS object, which is specified the start position.
|
||||
*
|
||||
* @param key The object key that is being retrieved from
|
||||
* the COS bucket.
|
||||
* @param byteRangeStart The start position of the part to be retrieved in
|
||||
* the object.
|
||||
* @return The input stream associated with the retrieved object.
|
||||
* @throws IOException if failed to retrieve.
|
||||
*/
|
||||
@Override
|
||||
public InputStream retrieve(String key, long byteRangeStart)
|
||||
throws IOException {
|
||||
try {
|
||||
LOG.debug("Retrieve COS key:[{}]. range start:[{}].",
|
||||
key, byteRangeStart);
|
||||
long fileSize = getFileLength(key);
|
||||
long byteRangeEnd = fileSize - 1;
|
||||
GetObjectRequest getObjectRequest =
|
||||
new GetObjectRequest(this.bucketName, key);
|
||||
if (byteRangeEnd >= byteRangeStart) {
|
||||
getObjectRequest.setRange(byteRangeStart, fileSize - 1);
|
||||
}
|
||||
COSObject cosObject =
|
||||
(COSObject) callCOSClientWithRetry(getObjectRequest);
|
||||
return cosObject.getObjectContent();
|
||||
} catch (Exception e) {
|
||||
String errMsg =
|
||||
String.format("Retrieving COS key: [%s] occurs an exception. " +
|
||||
"byte range start: [%s], exception: [%s].",
|
||||
key, byteRangeStart, e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), key);
|
||||
}
|
||||
|
||||
// never will get here
|
||||
return null;
|
||||
}
|
||||
|
||||
/**
|
||||
* Download a part of a COS object, which is specified the start and
|
||||
* end position.
|
||||
*
|
||||
* @param key The object key that is being downloaded
|
||||
* @param byteRangeStart The start position of the part to be retrieved in
|
||||
* the object.
|
||||
* @param byteRangeEnd The end position of the part to be retrieved in
|
||||
* the object.
|
||||
* @return The input stream associated with the retrieved objects.
|
||||
* @throws IOException If failed to retrieve.
|
||||
*/
|
||||
@Override
|
||||
public InputStream retrieveBlock(String key, long byteRangeStart,
|
||||
long byteRangeEnd) throws IOException {
|
||||
try {
|
||||
GetObjectRequest request = new GetObjectRequest(this.bucketName, key);
|
||||
request.setRange(byteRangeStart, byteRangeEnd);
|
||||
COSObject cosObject = (COSObject) this.callCOSClientWithRetry(request);
|
||||
return cosObject.getObjectContent();
|
||||
} catch (CosServiceException e) {
|
||||
String errMsg =
|
||||
String.format("Retrieving key [%s] with byteRangeStart [%d] occurs " +
|
||||
"an CosServiceException: [%s].",
|
||||
key, byteRangeStart, e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), key);
|
||||
return null;
|
||||
} catch (CosClientException e) {
|
||||
String errMsg =
|
||||
String.format("Retrieving key [%s] with byteRangeStart [%d] "
|
||||
+ "occurs an exception: [%s].",
|
||||
key, byteRangeStart, e.toString());
|
||||
LOG.error("Retrieving COS key: [{}] with byteRangeStart: [{}] " +
|
||||
"occurs an exception: [{}].", key, byteRangeStart, e);
|
||||
handleException(new Exception(errMsg), key);
|
||||
}
|
||||
|
||||
return null;
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartialListing list(String prefix, int maxListingLength)
|
||||
throws IOException {
|
||||
return list(prefix, maxListingLength, null, false);
|
||||
}
|
||||
|
||||
@Override
|
||||
public PartialListing list(String prefix, int maxListingLength,
|
||||
String priorLastKey, boolean recurse) throws IOException {
|
||||
return list(prefix, recurse ? null : CosNFileSystem.PATH_DELIMITER,
|
||||
maxListingLength, priorLastKey);
|
||||
}
|
||||
|
||||
/**
|
||||
* List the metadata for all objects that
|
||||
* the object key has the specified prefix.
|
||||
*
|
||||
* @param prefix The prefix to be listed.
|
||||
* @param delimiter The delimiter is a sign, the same paths between
|
||||
* are listed.
|
||||
* @param maxListingLength The maximum number of listed entries.
|
||||
* @param priorLastKey The last key in any previous search.
|
||||
* @return A metadata list on the match.
|
||||
* @throws IOException If list objects failed.
|
||||
*/
|
||||
private PartialListing list(String prefix, String delimiter,
|
||||
int maxListingLength, String priorLastKey) throws IOException {
|
||||
LOG.debug("List objects. prefix: [{}], delimiter: [{}], " +
|
||||
"maxListLength: [{}], priorLastKey: [{}].",
|
||||
prefix, delimiter, maxListingLength, priorLastKey);
|
||||
|
||||
if (!prefix.startsWith(CosNFileSystem.PATH_DELIMITER)) {
|
||||
prefix += CosNFileSystem.PATH_DELIMITER;
|
||||
}
|
||||
ListObjectsRequest listObjectsRequest = new ListObjectsRequest();
|
||||
listObjectsRequest.setBucketName(bucketName);
|
||||
listObjectsRequest.setPrefix(prefix);
|
||||
listObjectsRequest.setDelimiter(delimiter);
|
||||
listObjectsRequest.setMarker(priorLastKey);
|
||||
listObjectsRequest.setMaxKeys(maxListingLength);
|
||||
ObjectListing objectListing = null;
|
||||
try {
|
||||
objectListing =
|
||||
(ObjectListing) callCOSClientWithRetry(listObjectsRequest);
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("prefix: [%s], delimiter: [%s], "
|
||||
+ "maxListingLength: [%d], priorLastKey: [%s]. "
|
||||
+ "List objects occur an exception: [%s].", prefix,
|
||||
(delimiter == null) ? "" : delimiter, maxListingLength, priorLastKey,
|
||||
e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), prefix);
|
||||
}
|
||||
ArrayList<FileMetadata> fileMetadataArray = new ArrayList<>();
|
||||
ArrayList<FileMetadata> commonPrefixArray = new ArrayList<>();
|
||||
|
||||
if (null == objectListing) {
|
||||
String errMsg = String.format("List the prefix: [%s] failed. " +
|
||||
"delimiter: [%s], max listing length:" +
|
||||
" [%s], prior last key: [%s]",
|
||||
prefix, delimiter, maxListingLength, priorLastKey);
|
||||
handleException(new Exception(errMsg), prefix);
|
||||
}
|
||||
|
||||
List<COSObjectSummary> summaries = objectListing.getObjectSummaries();
|
||||
for (COSObjectSummary cosObjectSummary : summaries) {
|
||||
String filePath = cosObjectSummary.getKey();
|
||||
if (!filePath.startsWith(CosNFileSystem.PATH_DELIMITER)) {
|
||||
filePath = CosNFileSystem.PATH_DELIMITER + filePath;
|
||||
}
|
||||
if (filePath.equals(prefix)) {
|
||||
continue;
|
||||
}
|
||||
long mtime = 0;
|
||||
if (cosObjectSummary.getLastModified() != null) {
|
||||
mtime = cosObjectSummary.getLastModified().getTime();
|
||||
}
|
||||
long fileLen = cosObjectSummary.getSize();
|
||||
fileMetadataArray.add(
|
||||
new FileMetadata(filePath, fileLen, mtime, true));
|
||||
}
|
||||
List<String> commonPrefixes = objectListing.getCommonPrefixes();
|
||||
for (String commonPrefix : commonPrefixes) {
|
||||
if (!commonPrefix.startsWith(CosNFileSystem.PATH_DELIMITER)) {
|
||||
commonPrefix = CosNFileSystem.PATH_DELIMITER + commonPrefix;
|
||||
}
|
||||
commonPrefixArray.add(
|
||||
new FileMetadata(commonPrefix, 0, 0, false));
|
||||
}
|
||||
|
||||
FileMetadata[] fileMetadata = new FileMetadata[fileMetadataArray.size()];
|
||||
for (int i = 0; i < fileMetadataArray.size(); ++i) {
|
||||
fileMetadata[i] = fileMetadataArray.get(i);
|
||||
}
|
||||
FileMetadata[] commonPrefixMetaData =
|
||||
new FileMetadata[commonPrefixArray.size()];
|
||||
for (int i = 0; i < commonPrefixArray.size(); ++i) {
|
||||
commonPrefixMetaData[i] = commonPrefixArray.get(i);
|
||||
}
|
||||
// when truncated is false, it means that listing is finished.
|
||||
if (!objectListing.isTruncated()) {
|
||||
return new PartialListing(
|
||||
null, fileMetadata, commonPrefixMetaData);
|
||||
} else {
|
||||
return new PartialListing(
|
||||
objectListing.getNextMarker(), fileMetadata, commonPrefixMetaData);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void delete(String key) throws IOException {
|
||||
LOG.debug("Delete object key: [{}] from bucket: {}.", key, this.bucketName);
|
||||
try {
|
||||
DeleteObjectRequest deleteObjectRequest =
|
||||
new DeleteObjectRequest(bucketName, key);
|
||||
callCOSClientWithRetry(deleteObjectRequest);
|
||||
} catch (Exception e) {
|
||||
String errMsg =
|
||||
String.format("Delete key: [%s] occurs an exception: [%s].",
|
||||
key, e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), key);
|
||||
}
|
||||
}
|
||||
|
||||
public void rename(String srcKey, String dstKey) throws IOException {
|
||||
LOG.debug("Rename source key: [{}] to dest key: [{}].", srcKey, dstKey);
|
||||
try {
|
||||
CopyObjectRequest copyObjectRequest =
|
||||
new CopyObjectRequest(bucketName, srcKey, bucketName, dstKey);
|
||||
callCOSClientWithRetry(copyObjectRequest);
|
||||
DeleteObjectRequest deleteObjectRequest =
|
||||
new DeleteObjectRequest(bucketName, srcKey);
|
||||
callCOSClientWithRetry(deleteObjectRequest);
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Rename object unsuccessfully. "
|
||||
+ "source cos key: [%s], dest COS " +
|
||||
"key: [%s], exception: [%s]",
|
||||
srcKey,
|
||||
dstKey, e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), srcKey);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void copy(String srcKey, String dstKey) throws IOException {
|
||||
LOG.debug("Copy source key: [{}] to dest key: [{}].", srcKey, dstKey);
|
||||
try {
|
||||
CopyObjectRequest copyObjectRequest =
|
||||
new CopyObjectRequest(bucketName, srcKey, bucketName, dstKey);
|
||||
callCOSClientWithRetry(copyObjectRequest);
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Copy object unsuccessfully. "
|
||||
+ "source COS key: %s, dest COS key: " +
|
||||
"%s, exception: %s",
|
||||
srcKey,
|
||||
dstKey, e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), srcKey);
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void purge(String prefix) throws IOException {
|
||||
throw new IOException("purge not supported");
|
||||
}
|
||||
|
||||
@Override
|
||||
public void dump() throws IOException {
|
||||
throw new IOException("dump not supported");
|
||||
}
|
||||
|
||||
// process Exception and print detail
|
||||
private void handleException(Exception e, String key) throws IOException {
|
||||
String cosPath = CosNFileSystem.SCHEME + "://" + bucketName + key;
|
||||
String exceptInfo = String.format("%s : %s", cosPath, e.toString());
|
||||
throw new IOException(exceptInfo);
|
||||
}
|
||||
|
||||
@Override
|
||||
public long getFileLength(String key) throws IOException {
|
||||
LOG.debug("Get file length. COS key: {}", key);
|
||||
GetObjectMetadataRequest getObjectMetadataRequest =
|
||||
new GetObjectMetadataRequest(bucketName, key);
|
||||
try {
|
||||
ObjectMetadata objectMetadata =
|
||||
(ObjectMetadata) callCOSClientWithRetry(getObjectMetadataRequest);
|
||||
return objectMetadata.getContentLength();
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Getting file length occurs an exception." +
|
||||
"COS key: %s, exception: %s", key,
|
||||
e.toString());
|
||||
LOG.error(errMsg);
|
||||
handleException(new Exception(errMsg), key);
|
||||
return 0; // never will get here
|
||||
}
|
||||
}
|
||||
|
||||
private <X> Object callCOSClientWithRetry(X request)
|
||||
throws CosServiceException, IOException {
|
||||
String sdkMethod = "";
|
||||
int retryIndex = 1;
|
||||
while (true) {
|
||||
try {
|
||||
if (request instanceof PutObjectRequest) {
|
||||
sdkMethod = "putObject";
|
||||
return this.cosClient.putObject((PutObjectRequest) request);
|
||||
} else if (request instanceof UploadPartRequest) {
|
||||
sdkMethod = "uploadPart";
|
||||
if (((UploadPartRequest) request).getInputStream()
|
||||
instanceof ByteBufferInputStream) {
|
||||
((UploadPartRequest) request).getInputStream()
|
||||
.mark((int) ((UploadPartRequest) request).getPartSize());
|
||||
}
|
||||
return this.cosClient.uploadPart((UploadPartRequest) request);
|
||||
} else if (request instanceof GetObjectMetadataRequest) {
|
||||
sdkMethod = "queryObjectMeta";
|
||||
return this.cosClient.getObjectMetadata(
|
||||
(GetObjectMetadataRequest) request);
|
||||
} else if (request instanceof DeleteObjectRequest) {
|
||||
sdkMethod = "deleteObject";
|
||||
this.cosClient.deleteObject((DeleteObjectRequest) request);
|
||||
return new Object();
|
||||
} else if (request instanceof CopyObjectRequest) {
|
||||
sdkMethod = "copyFile";
|
||||
return this.cosClient.copyObject((CopyObjectRequest) request);
|
||||
} else if (request instanceof GetObjectRequest) {
|
||||
sdkMethod = "getObject";
|
||||
return this.cosClient.getObject((GetObjectRequest) request);
|
||||
} else if (request instanceof ListObjectsRequest) {
|
||||
sdkMethod = "listObjects";
|
||||
return this.cosClient.listObjects((ListObjectsRequest) request);
|
||||
} else {
|
||||
throw new IOException("no such method");
|
||||
}
|
||||
} catch (CosServiceException cse) {
|
||||
String errMsg = String.format("Call cos sdk failed, "
|
||||
+ "retryIndex: [%d / %d], "
|
||||
+ "call method: %s, exception: %s",
|
||||
retryIndex, this.maxRetryTimes, sdkMethod, cse.toString());
|
||||
int statusCode = cse.getStatusCode();
|
||||
// Retry all server errors
|
||||
if (statusCode / 100 == 5) {
|
||||
if (retryIndex <= this.maxRetryTimes) {
|
||||
LOG.info(errMsg);
|
||||
long sleepLeast = retryIndex * 300L;
|
||||
long sleepBound = retryIndex * 500L;
|
||||
try {
|
||||
if (request instanceof UploadPartRequest) {
|
||||
if (((UploadPartRequest) request).getInputStream()
|
||||
instanceof ByteBufferInputStream) {
|
||||
((UploadPartRequest) request).getInputStream().reset();
|
||||
}
|
||||
}
|
||||
Thread.sleep(
|
||||
ThreadLocalRandom.current().nextLong(sleepLeast, sleepBound));
|
||||
++retryIndex;
|
||||
} catch (InterruptedException e) {
|
||||
throw new IOException(e.toString());
|
||||
}
|
||||
} else {
|
||||
LOG.error(errMsg);
|
||||
throw new IOException(errMsg);
|
||||
}
|
||||
} else {
|
||||
throw cse;
|
||||
}
|
||||
} catch (Exception e) {
|
||||
String errMsg = String.format("Call cos sdk failed, "
|
||||
+ "call method: %s, exception: %s", sdkMethod, e.toString());
|
||||
LOG.error(errMsg);
|
||||
throw new IOException(errMsg);
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() {
|
||||
if (null != this.cosClient) {
|
||||
this.cosClient.shutdown();
|
||||
}
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Holds basic metadata for a file stored in a {@link NativeFileSystemStore}.
|
||||
* </p>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
class FileMetadata {
|
||||
private final String key;
|
||||
private final long length;
|
||||
private final long lastModified;
|
||||
private final boolean isFile;
|
||||
|
||||
FileMetadata(String key, long length, long lastModified) {
|
||||
this(key, length, lastModified, true);
|
||||
}
|
||||
|
||||
FileMetadata(String key, long length, long lastModified, boolean isFile) {
|
||||
this.key = key;
|
||||
this.length = length;
|
||||
this.lastModified = lastModified;
|
||||
this.isFile = isFile;
|
||||
}
|
||||
|
||||
public String getKey() {
|
||||
return key;
|
||||
}
|
||||
|
||||
public long getLength() {
|
||||
return length;
|
||||
}
|
||||
|
||||
public long getLastModified() {
|
||||
return lastModified;
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "FileMetadata[" + key + ", " + length + ", " + lastModified + ", "
|
||||
+ "file?" + isFile + "]";
|
||||
}
|
||||
|
||||
public boolean isFile() {
|
||||
return isFile;
|
||||
}
|
||||
}
|
@ -0,0 +1,99 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.File;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
import java.net.URI;
|
||||
import java.util.List;
|
||||
|
||||
import com.qcloud.cos.model.CompleteMultipartUploadResult;
|
||||
import com.qcloud.cos.model.PartETag;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* An abstraction for a key-based {@link File} store.
|
||||
* </p>
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Stable
|
||||
interface NativeFileSystemStore {
|
||||
|
||||
void initialize(URI uri, Configuration conf) throws IOException;
|
||||
|
||||
void storeFile(String key, File file, byte[] md5Hash) throws IOException;
|
||||
|
||||
void storeFile(String key, InputStream inputStream, byte[] md5Hash,
|
||||
long contentLength) throws IOException;
|
||||
|
||||
void storeEmptyFile(String key) throws IOException;
|
||||
|
||||
CompleteMultipartUploadResult completeMultipartUpload(
|
||||
String key, String uploadId, List<PartETag> partETagList);
|
||||
|
||||
void abortMultipartUpload(String key, String uploadId);
|
||||
|
||||
String getUploadId(String key);
|
||||
|
||||
PartETag uploadPart(File file, String key, String uploadId, int partNum)
|
||||
throws IOException;
|
||||
|
||||
PartETag uploadPart(InputStream inputStream, String key, String uploadId,
|
||||
int partNum, long partSize) throws IOException;
|
||||
|
||||
FileMetadata retrieveMetadata(String key) throws IOException;
|
||||
|
||||
InputStream retrieve(String key) throws IOException;
|
||||
|
||||
InputStream retrieve(String key, long byteRangeStart) throws IOException;
|
||||
|
||||
InputStream retrieveBlock(String key, long byteRangeStart, long byteRangeEnd)
|
||||
throws IOException;
|
||||
|
||||
long getFileLength(String key) throws IOException;
|
||||
|
||||
PartialListing list(String prefix, int maxListingLength) throws IOException;
|
||||
|
||||
PartialListing list(String prefix, int maxListingLength,
|
||||
String priorLastKey, boolean recursive) throws IOException;
|
||||
|
||||
void delete(String key) throws IOException;
|
||||
|
||||
void copy(String srcKey, String dstKey) throws IOException;
|
||||
|
||||
/**
|
||||
* Delete all keys with the given prefix. Used for testing.
|
||||
*
|
||||
* @throws IOException if purge is not supported
|
||||
*/
|
||||
void purge(String prefix) throws IOException;
|
||||
|
||||
/**
|
||||
* Diagnostic method to dump state to the console.
|
||||
*
|
||||
* @throws IOException if dump is not supported
|
||||
*/
|
||||
void dump() throws IOException;
|
||||
|
||||
void close();
|
||||
}
|
@ -0,0 +1,64 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import org.apache.hadoop.classification.InterfaceAudience;
|
||||
import org.apache.hadoop.classification.InterfaceStability;
|
||||
|
||||
/**
|
||||
* <p>
|
||||
* Holds information on a directory listing for a
|
||||
* {@link NativeFileSystemStore}.
|
||||
* This includes the {@link FileMetadata files} and directories
|
||||
* (their names) contained in a directory.
|
||||
* </p>
|
||||
* <p>
|
||||
* This listing may be returned in chunks, so a <code>priorLastKey</code>
|
||||
* is provided so that the next chunk may be requested.
|
||||
* </p>
|
||||
*
|
||||
* @see NativeFileSystemStore#list(String, int, String, boolean)
|
||||
*/
|
||||
@InterfaceAudience.Private
|
||||
@InterfaceStability.Unstable
|
||||
class PartialListing {
|
||||
|
||||
private final String priorLastKey;
|
||||
private final FileMetadata[] files;
|
||||
private final FileMetadata[] commonPrefixes;
|
||||
|
||||
PartialListing(String priorLastKey, FileMetadata[] files,
|
||||
FileMetadata[] commonPrefixes) {
|
||||
this.priorLastKey = priorLastKey;
|
||||
this.files = files;
|
||||
this.commonPrefixes = commonPrefixes;
|
||||
}
|
||||
|
||||
public FileMetadata[] getFiles() {
|
||||
return files;
|
||||
}
|
||||
|
||||
public FileMetadata[] getCommonPrefixes() {
|
||||
return commonPrefixes;
|
||||
}
|
||||
|
||||
public String getPriorLastKey() {
|
||||
return priorLastKey;
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
/**
|
||||
* Constant definition of storage unit.
|
||||
*/
|
||||
public final class Unit {
|
||||
private Unit() {
|
||||
}
|
||||
|
||||
public static final int KB = 1024;
|
||||
public static final int MB = 1024 * KB;
|
||||
public static final int GB = 1024 * MB;
|
||||
public static final long TB = (long) 1024 * GB;
|
||||
public static final long PB = (long) 1024 * TB;
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.auth;
|
||||
|
||||
import java.io.Closeable;
|
||||
import java.util.ArrayList;
|
||||
import java.util.Collection;
|
||||
import java.util.List;
|
||||
import java.util.concurrent.atomic.AtomicBoolean;
|
||||
import java.util.concurrent.atomic.AtomicInteger;
|
||||
|
||||
import com.google.common.base.Preconditions;
|
||||
import com.qcloud.cos.auth.AnonymousCOSCredentials;
|
||||
import com.qcloud.cos.auth.COSCredentials;
|
||||
import com.qcloud.cos.auth.COSCredentialsProvider;
|
||||
import com.qcloud.cos.exception.CosClientException;
|
||||
import com.qcloud.cos.utils.StringUtils;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
/**
|
||||
* a list of cos credentials provider.
|
||||
*/
|
||||
public class COSCredentialProviderList implements
|
||||
COSCredentialsProvider, AutoCloseable {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(COSCredentialProviderList.class);
|
||||
|
||||
private static final String NO_COS_CREDENTIAL_PROVIDERS =
|
||||
"No COS Credential Providers";
|
||||
private static final String CREDENTIALS_REQUESTED_WHEN_CLOSED =
|
||||
"Credentials requested after provider list was closed";
|
||||
|
||||
private final List<COSCredentialsProvider> providers =
|
||||
new ArrayList<>(1);
|
||||
private boolean reuseLastProvider = true;
|
||||
private COSCredentialsProvider lastProvider;
|
||||
|
||||
private final AtomicInteger refCount = new AtomicInteger(1);
|
||||
private final AtomicBoolean isClosed = new AtomicBoolean(false);
|
||||
|
||||
public COSCredentialProviderList() {
|
||||
}
|
||||
|
||||
public COSCredentialProviderList(
|
||||
Collection<COSCredentialsProvider> providers) {
|
||||
this.providers.addAll(providers);
|
||||
}
|
||||
|
||||
public void add(COSCredentialsProvider provider) {
|
||||
this.providers.add(provider);
|
||||
}
|
||||
|
||||
public int getRefCount() {
|
||||
return this.refCount.get();
|
||||
}
|
||||
|
||||
public void checkNotEmpty() {
|
||||
if (this.providers.isEmpty()) {
|
||||
throw new NoAuthWithCOSException(NO_COS_CREDENTIAL_PROVIDERS);
|
||||
}
|
||||
}
|
||||
|
||||
public COSCredentialProviderList share() {
|
||||
Preconditions.checkState(!this.closed(), "Provider list is closed");
|
||||
this.refCount.incrementAndGet();
|
||||
return this;
|
||||
}
|
||||
|
||||
public boolean closed() {
|
||||
return this.isClosed.get();
|
||||
}
|
||||
|
||||
@Override
|
||||
public COSCredentials getCredentials() {
|
||||
if (this.closed()) {
|
||||
throw new NoAuthWithCOSException(CREDENTIALS_REQUESTED_WHEN_CLOSED);
|
||||
}
|
||||
|
||||
this.checkNotEmpty();
|
||||
|
||||
if (this.reuseLastProvider && this.lastProvider != null) {
|
||||
return this.lastProvider.getCredentials();
|
||||
}
|
||||
|
||||
for (COSCredentialsProvider provider : this.providers) {
|
||||
try {
|
||||
COSCredentials credentials = provider.getCredentials();
|
||||
if (!StringUtils.isNullOrEmpty(credentials.getCOSAccessKeyId())
|
||||
&& !StringUtils.isNullOrEmpty(credentials.getCOSSecretKey())
|
||||
|| credentials instanceof AnonymousCOSCredentials) {
|
||||
this.lastProvider = provider;
|
||||
return credentials;
|
||||
}
|
||||
} catch (CosClientException e) {
|
||||
LOG.warn("No credentials provided by {}: {}", provider, e.toString());
|
||||
}
|
||||
}
|
||||
|
||||
throw new NoAuthWithCOSException(
|
||||
"No COS Credentials provided by " + this.providers.toString());
|
||||
}
|
||||
|
||||
@Override
|
||||
public void close() throws Exception {
|
||||
if (this.closed()) {
|
||||
return;
|
||||
}
|
||||
|
||||
int remainder = this.refCount.decrementAndGet();
|
||||
if (remainder != 0) {
|
||||
return;
|
||||
}
|
||||
this.isClosed.set(true);
|
||||
|
||||
for (COSCredentialsProvider provider : this.providers) {
|
||||
if (provider instanceof Closeable) {
|
||||
((Closeable) provider).close();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
||||
}
|
@ -0,0 +1,55 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.auth;
|
||||
|
||||
import com.qcloud.cos.auth.BasicCOSCredentials;
|
||||
import com.qcloud.cos.auth.COSCredentials;
|
||||
import com.qcloud.cos.auth.COSCredentialsProvider;
|
||||
import com.qcloud.cos.exception.CosClientException;
|
||||
import com.qcloud.cos.utils.StringUtils;
|
||||
|
||||
import org.apache.hadoop.fs.cosn.Constants;
|
||||
|
||||
/**
|
||||
* the provider obtaining the cos credentials from the environment variables.
|
||||
*/
|
||||
public class EnvironmentVariableCredentialProvider
|
||||
implements COSCredentialsProvider {
|
||||
@Override
|
||||
public COSCredentials getCredentials() {
|
||||
String secretId = System.getenv(Constants.COSN_SECRET_ID_ENV);
|
||||
String secretKey = System.getenv(Constants.COSN_SECRET_KEY_ENV);
|
||||
|
||||
secretId = StringUtils.trim(secretId);
|
||||
secretKey = StringUtils.trim(secretKey);
|
||||
|
||||
if (!StringUtils.isNullOrEmpty(secretId)
|
||||
&& !StringUtils.isNullOrEmpty(secretKey)) {
|
||||
return new BasicCOSCredentials(secretId, secretKey);
|
||||
} else {
|
||||
throw new CosClientException(
|
||||
"Unable to load COS credentials from environment variables" +
|
||||
"(COS_SECRET_ID or COS_SECRET_KEY)");
|
||||
}
|
||||
}
|
||||
|
||||
@Override
|
||||
public String toString() {
|
||||
return "EnvironmentVariableCredentialProvider{}";
|
||||
}
|
||||
}
|
@ -0,0 +1,37 @@
|
||||
/**
|
||||
* Licensed to the Apache Software Foundation (ASF) under one
|
||||
* or more contributor license agreements. See the NOTICE file
|
||||
* distributed with this work for additional information
|
||||
* regarding copyright ownership. The ASF licenses this file
|
||||
* to you under the Apache License, Version 2.0 (the
|
||||
* "License"); you may not use this file except in compliance
|
||||
* with the License. You may obtain a copy of the License at
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.auth;
|
||||
|
||||
import com.qcloud.cos.exception.CosClientException;
|
||||
|
||||
/**
|
||||
* Exception thrown when no credentials can be obtained.
|
||||
*/
|
||||
public class NoAuthWithCOSException extends CosClientException {
|
||||
public NoAuthWithCOSException(String message, Throwable t) {
|
||||
super(message, t);
|
||||
}
|
||||
|
||||
public NoAuthWithCOSException(String message) {
|
||||
super(message);
|
||||
}
|
||||
|
||||
public NoAuthWithCOSException(Throwable t) {
|
||||
super(t);
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.auth;
|
||||
|
||||
import com.qcloud.cos.auth.BasicCOSCredentials;
|
||||
import com.qcloud.cos.auth.COSCredentials;
|
||||
import com.qcloud.cos.auth.COSCredentialsProvider;
|
||||
import com.qcloud.cos.exception.CosClientException;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.cosn.CosNConfigKeys;
|
||||
|
||||
/**
|
||||
* Get the credentials from the hadoop configuration.
|
||||
*/
|
||||
public class SimpleCredentialProvider implements COSCredentialsProvider {
|
||||
private String secretId;
|
||||
private String secretKey;
|
||||
|
||||
public SimpleCredentialProvider(Configuration conf) {
|
||||
this.secretId = conf.get(
|
||||
CosNConfigKeys.COSN_SECRET_ID_KEY
|
||||
);
|
||||
this.secretKey = conf.get(
|
||||
CosNConfigKeys.COSN_SECRET_KEY_KEY
|
||||
);
|
||||
}
|
||||
|
||||
@Override
|
||||
public COSCredentials getCredentials() {
|
||||
if (!StringUtils.isEmpty(this.secretId)
|
||||
&& !StringUtils.isEmpty(this.secretKey)) {
|
||||
return new BasicCOSCredentials(this.secretId, this.secretKey);
|
||||
}
|
||||
throw new CosClientException("secret id or secret key is unset");
|
||||
}
|
||||
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.auth;
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
@ -0,0 +1,33 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
/**
|
||||
* Configuration options for the CosN file system for testing.
|
||||
*/
|
||||
public final class CosNTestConfigKey {
|
||||
private CosNTestConfigKey() {
|
||||
}
|
||||
|
||||
public static final String TEST_COS_FILESYSTEM_CONF_KEY =
|
||||
"test.fs.cosn.name";
|
||||
public static final String DEFAULT_TEST_COS_FILESYSTEM_CONF_VALUE =
|
||||
"";
|
||||
public static final String TEST_UNIQUE_FORK_ID_KEY =
|
||||
"test.unique.fork.id";
|
||||
}
|
@ -0,0 +1,78 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.net.URI;
|
||||
|
||||
import org.apache.commons.lang3.StringUtils;
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.junit.internal.AssumptionViolatedException;
|
||||
|
||||
/**
|
||||
* Utilities for the CosN tests.
|
||||
*/
|
||||
public final class CosNTestUtils {
|
||||
|
||||
private CosNTestUtils() {
|
||||
}
|
||||
|
||||
/**
|
||||
* Create the file system for test.
|
||||
*
|
||||
* @param configuration hadoop's configuration
|
||||
* @return The file system for test
|
||||
* @throws IOException If fail to create or initialize the file system.
|
||||
*/
|
||||
public static CosNFileSystem createTestFileSystem(
|
||||
Configuration configuration) throws IOException {
|
||||
String fsName = configuration.getTrimmed(
|
||||
CosNTestConfigKey.TEST_COS_FILESYSTEM_CONF_KEY,
|
||||
CosNTestConfigKey.DEFAULT_TEST_COS_FILESYSTEM_CONF_VALUE);
|
||||
|
||||
boolean liveTest = StringUtils.isNotEmpty(fsName);
|
||||
URI testUri;
|
||||
if (liveTest) {
|
||||
testUri = URI.create(fsName);
|
||||
liveTest = testUri.getScheme().equals(CosNFileSystem.SCHEME);
|
||||
} else {
|
||||
throw new AssumptionViolatedException("no test file system in " +
|
||||
fsName);
|
||||
}
|
||||
|
||||
CosNFileSystem cosFs = new CosNFileSystem();
|
||||
cosFs.initialize(testUri, configuration);
|
||||
return cosFs;
|
||||
}
|
||||
|
||||
/**
|
||||
* Create a dir path for test.
|
||||
* The value of {@link CosNTestConfigKey#TEST_UNIQUE_FORK_ID_KEY}
|
||||
* will be used if it is set.
|
||||
*
|
||||
* @param defVal default value
|
||||
* @return The test path
|
||||
*/
|
||||
public static Path createTestPath(Path defVal) {
|
||||
String testUniqueForkId = System.getProperty(
|
||||
CosNTestConfigKey.TEST_UNIQUE_FORK_ID_KEY);
|
||||
return testUniqueForkId ==
|
||||
null ? defVal : new Path("/" + testUniqueForkId, "test");
|
||||
}
|
||||
}
|
@ -0,0 +1,167 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.junit.*;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
import static org.junit.Assert.assertTrue;
|
||||
|
||||
import java.io.IOException;
|
||||
import java.util.Random;
|
||||
|
||||
/**
|
||||
* CosNInputStream Tester.
|
||||
*/
|
||||
public class TestCosNInputStream {
|
||||
private static final Logger LOG =
|
||||
LoggerFactory.getLogger(TestCosNInputStream.class);
|
||||
|
||||
private FileSystem fs;
|
||||
|
||||
private Path testRootDir;
|
||||
|
||||
@Before
|
||||
public void setUp() throws IOException {
|
||||
Configuration configuration = new Configuration();
|
||||
this.fs = CosNTestUtils.createTestFileSystem(configuration);
|
||||
this.testRootDir = CosNTestUtils.createTestPath(new Path("/test"));
|
||||
LOG.info("test root dir: " + this.testRootDir);
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws IOException {
|
||||
if (null != this.fs) {
|
||||
this.fs.delete(this.testRootDir, true);
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Method: seek(long pos).
|
||||
*/
|
||||
@Test
|
||||
public void testSeek() throws Exception {
|
||||
Path seekTestFilePath = new Path(this.testRootDir + "/"
|
||||
+ "seekTestFile");
|
||||
long fileSize = 5 * Unit.MB;
|
||||
|
||||
ContractTestUtils.generateTestFile(
|
||||
this.fs, seekTestFilePath, fileSize, 256, 255);
|
||||
LOG.info("5MB file for seek test has created.");
|
||||
|
||||
FSDataInputStream inputStream = this.fs.open(seekTestFilePath);
|
||||
int seekTimes = 5;
|
||||
for (int i = 0; i != seekTimes; i++) {
|
||||
long pos = fileSize / (seekTimes - i) - 1;
|
||||
inputStream.seek(pos);
|
||||
assertTrue("expected position at: " +
|
||||
pos + ", but got: " + inputStream.getPos(),
|
||||
inputStream.getPos() == pos);
|
||||
LOG.info("completed seeking at pos: " + inputStream.getPos());
|
||||
}
|
||||
LOG.info("begin to random position seeking test...");
|
||||
Random random = new Random();
|
||||
for (int i = 0; i < seekTimes; i++) {
|
||||
long pos = Math.abs(random.nextLong()) % fileSize;
|
||||
LOG.info("seeking for pos: " + pos);
|
||||
inputStream.seek(pos);
|
||||
assertTrue("expected position at: " +
|
||||
pos + ", but got: " + inputStream.getPos(),
|
||||
inputStream.getPos() == pos);
|
||||
LOG.info("completed seeking at pos: " + inputStream.getPos());
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Method: getPos().
|
||||
*/
|
||||
@Test
|
||||
public void testGetPos() throws Exception {
|
||||
Path seekTestFilePath = new Path(this.testRootDir + "/" +
|
||||
"seekTestFile");
|
||||
long fileSize = 5 * Unit.MB;
|
||||
ContractTestUtils.generateTestFile(
|
||||
this.fs, seekTestFilePath, fileSize, 256, 255);
|
||||
LOG.info("5MB file for getPos test has created.");
|
||||
|
||||
FSDataInputStream inputStream = this.fs.open(seekTestFilePath);
|
||||
Random random = new Random();
|
||||
long pos = Math.abs(random.nextLong()) % fileSize;
|
||||
inputStream.seek(pos);
|
||||
assertTrue("expected position at: " +
|
||||
pos + ", but got: " + inputStream.getPos(),
|
||||
inputStream.getPos() == pos);
|
||||
LOG.info("completed get pos tests.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Method: seekToNewSource(long targetPos).
|
||||
*/
|
||||
@Ignore("Not ready yet")
|
||||
public void testSeekToNewSource() throws Exception {
|
||||
LOG.info("Currently it is not supported to " +
|
||||
"seek the offset in a new source.");
|
||||
}
|
||||
|
||||
/**
|
||||
* Method: read().
|
||||
*/
|
||||
@Test
|
||||
public void testRead() throws Exception {
|
||||
final int bufLen = 256;
|
||||
Path readTestFilePath = new Path(this.testRootDir + "/"
|
||||
+ "testReadSmallFile.txt");
|
||||
long fileSize = 5 * Unit.MB;
|
||||
|
||||
ContractTestUtils.generateTestFile(
|
||||
this.fs, readTestFilePath, fileSize, 256, 255);
|
||||
LOG.info("read test file: " + readTestFilePath + " has created.");
|
||||
|
||||
FSDataInputStream inputStream = this.fs.open(readTestFilePath);
|
||||
byte[] buf = new byte[bufLen];
|
||||
long bytesRead = 0;
|
||||
while (bytesRead < fileSize) {
|
||||
int bytes = 0;
|
||||
if (fileSize - bytesRead < bufLen) {
|
||||
int remaining = (int) (fileSize - bytesRead);
|
||||
bytes = inputStream.read(buf, 0, remaining);
|
||||
} else {
|
||||
bytes = inputStream.read(buf, 0, bufLen);
|
||||
}
|
||||
bytesRead += bytes;
|
||||
|
||||
if (bytesRead % (1 * Unit.MB) == 0) {
|
||||
int available = inputStream.available();
|
||||
assertTrue("expected remaining: " + (fileSize - bytesRead) +
|
||||
" but got: " + available, (fileSize - bytesRead) == available);
|
||||
LOG.info("Bytes read: " +
|
||||
Math.round((double) bytesRead / Unit.MB) + "MB");
|
||||
}
|
||||
}
|
||||
|
||||
assertTrue(inputStream.available() == 0);
|
||||
IOUtils.closeStream(inputStream);
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.contract.ContractTestUtils;
|
||||
import org.junit.Rule;
|
||||
import org.junit.Test;
|
||||
import org.junit.Before;
|
||||
import org.junit.After;
|
||||
import org.junit.rules.Timeout;
|
||||
|
||||
import java.io.IOException;
|
||||
|
||||
/**
|
||||
* CosNOutputStream Tester.
|
||||
* <p>
|
||||
* If the test.fs.cosn.name property is not set, all test case will fail.
|
||||
*/
|
||||
public class TestCosNOutputStream {
|
||||
private FileSystem fs;
|
||||
private Path testRootDir;
|
||||
|
||||
@Rule
|
||||
public Timeout timeout = new Timeout(3600 * 1000);
|
||||
|
||||
@Before
|
||||
public void setUp() throws Exception {
|
||||
Configuration configuration = new Configuration();
|
||||
configuration.setInt(
|
||||
CosNConfigKeys.COSN_BLOCK_SIZE_KEY, 2 * Unit.MB);
|
||||
configuration.setLong(
|
||||
CosNConfigKeys.COSN_UPLOAD_BUFFER_SIZE_KEY,
|
||||
CosNConfigKeys.DEFAULT_UPLOAD_BUFFER_SIZE);
|
||||
this.fs = CosNTestUtils.createTestFileSystem(configuration);
|
||||
this.testRootDir = new Path("/test");
|
||||
}
|
||||
|
||||
@After
|
||||
public void tearDown() throws Exception {
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEmptyFileUpload() throws IOException {
|
||||
ContractTestUtils.createAndVerifyFile(this.fs, this.testRootDir, 0);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testSingleFileUpload() throws IOException {
|
||||
ContractTestUtils.createAndVerifyFile(
|
||||
this.fs, this.testRootDir, 1 * Unit.MB - 1);
|
||||
ContractTestUtils.createAndVerifyFile(
|
||||
this.fs, this.testRootDir, 1 * Unit.MB);
|
||||
ContractTestUtils.createAndVerifyFile(
|
||||
this.fs, this.testRootDir, 2 * Unit.MB - 1);
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testLargeFileUpload() throws IOException {
|
||||
ContractTestUtils.createAndVerifyFile(
|
||||
this.fs, this.testRootDir, 2 * Unit.MB);
|
||||
ContractTestUtils.createAndVerifyFile(
|
||||
this.fs, this.testRootDir, 2 * Unit.MB + 1);
|
||||
ContractTestUtils.createAndVerifyFile(
|
||||
this.fs, this.testRootDir, 100 * Unit.MB);
|
||||
// In principle, a maximum boundary test (file size: 2MB * 10000 - 1)
|
||||
// should be provided here,
|
||||
// but it is skipped due to network bandwidth and test time constraints.
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.fs.contract.AbstractBondedFSContract;
|
||||
import org.apache.hadoop.fs.cosn.CosNFileSystem;
|
||||
import org.apache.hadoop.fs.cosn.CosNTestUtils;
|
||||
|
||||
/**
|
||||
* The contract of CosN: only enabled if the test bucket is provided.
|
||||
*/
|
||||
public class CosNContract extends AbstractBondedFSContract {
|
||||
private static final String CONTRACT_XML = "contract/cosn.xml";
|
||||
|
||||
protected CosNContract(Configuration conf) {
|
||||
super(conf);
|
||||
addConfResource(CONTRACT_XML);
|
||||
}
|
||||
|
||||
@Override
|
||||
public String getScheme() {
|
||||
return CosNFileSystem.SCHEME;
|
||||
}
|
||||
|
||||
@Override
|
||||
public Path getTestPath() {
|
||||
return CosNTestUtils.createTestPath(super.getTestPath());
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractCreateTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* CosN contract tests for creating files.
|
||||
*/
|
||||
public class TestCosNContractCreate extends AbstractContractCreateTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration configuration) {
|
||||
return new CosNContract(configuration);
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractDeleteTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* CosN contract tests for deleting files.
|
||||
*/
|
||||
public class TestCosNContractDelete extends AbstractContractDeleteTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration configuration) {
|
||||
return new CosNContract(configuration);
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
import org.apache.hadoop.fs.cosn.CosNConfigKeys;
|
||||
import org.apache.hadoop.fs.cosn.Unit;
|
||||
import org.apache.hadoop.tools.contract.AbstractContractDistCpTest;
|
||||
|
||||
/**
|
||||
* Contract test suit covering CosN integration with DistCp.
|
||||
*/
|
||||
public class TestCosNContractDistCp extends AbstractContractDistCpTest {
|
||||
|
||||
private static final int MULTIPART_SETTING = 2 * Unit.MB;
|
||||
private static final long UPLOAD_BUFFER_POOL_SIZE = 5 * 2 * Unit.MB;
|
||||
private static final int UPLOAD_THREAD_POOL_SIZE = 5;
|
||||
private static final int COPY_THREAD_POOL_SIZE = 3;
|
||||
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration conf) {
|
||||
return new CosNContract(conf);
|
||||
}
|
||||
|
||||
@Override
|
||||
protected Configuration createConfiguration() {
|
||||
Configuration newConf = super.createConfiguration();
|
||||
newConf.setInt(CosNConfigKeys.COSN_BLOCK_SIZE_KEY,
|
||||
MULTIPART_SETTING);
|
||||
newConf.setLong(CosNConfigKeys.COSN_UPLOAD_BUFFER_SIZE_KEY,
|
||||
UPLOAD_BUFFER_POOL_SIZE);
|
||||
newConf.setInt(CosNConfigKeys.UPLOAD_THREAD_POOL_SIZE_KEY,
|
||||
UPLOAD_THREAD_POOL_SIZE);
|
||||
newConf.setInt(CosNConfigKeys.COPY_THREAD_POOL_SIZE_KEY,
|
||||
COPY_THREAD_POOL_SIZE);
|
||||
return newConf;
|
||||
}
|
||||
}
|
@ -0,0 +1,33 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* CosN contract tests covering getFileStatus.
|
||||
*/
|
||||
public class TestCosNContractGetFileStatus
|
||||
extends AbstractContractGetFileStatusTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration conf) {
|
||||
return new CosNContract(conf);
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractMkdirTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* CosN contract tests for making directories.
|
||||
*/
|
||||
public class TestCosNContractMkdir extends AbstractContractMkdirTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration configuration) {
|
||||
return new CosNContract(configuration);
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractOpenTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* CosN contract tests for opening files.
|
||||
*/
|
||||
public class TestCosNContractOpen extends AbstractContractOpenTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration configuration) {
|
||||
return new CosNContract(configuration);
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractRenameTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* CosN contract tests for renaming a file.
|
||||
*/
|
||||
public class TestCosNContractRename extends AbstractContractRenameTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration configuration) {
|
||||
return new CosNContract(configuration);
|
||||
}
|
||||
}
|
@ -0,0 +1,33 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* root dir operations against an COS bucket.
|
||||
*/
|
||||
public class TestCosNContractRootDir
|
||||
extends AbstractContractRootDirectoryTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration configuration) {
|
||||
return new CosNContract(configuration);
|
||||
}
|
||||
}
|
@ -0,0 +1,32 @@
|
||||
/**
|
||||
* 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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
||||
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.contract.AbstractContractSeekTest;
|
||||
import org.apache.hadoop.fs.contract.AbstractFSContract;
|
||||
|
||||
/**
|
||||
* CosN contract tests for seeking a position in a file.
|
||||
*/
|
||||
public class TestCosNContractSeek extends AbstractContractSeekTest {
|
||||
@Override
|
||||
protected AbstractFSContract createContract(Configuration configuration) {
|
||||
return new CosNContract(configuration);
|
||||
}
|
||||
}
|
@ -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
|
||||
* <p>
|
||||
* http://www.apache.org/licenses/LICENSE-2.0
|
||||
* <p>
|
||||
* 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.cosn.contract;
|
@ -0,0 +1,120 @@
|
||||
<!--
|
||||
~ 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.is-blobstore</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.create-overwrites-directory</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.create-visibility-delayed</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.is-case-sensitive</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-returns-false-if-source-missing</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-append</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-atomic-directory-delete</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-atomic-rename</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-block-locality</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-concat</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-seek</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-seek-on-closed-file</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rejects-seek-past-eof</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.rename-overwrites-dest</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-getfilestatus</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-unix-permissions</name>
|
||||
<value>false</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.supports-strict-exceptions</name>
|
||||
<value>true</value>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.contract.test.random-seek-count</name>
|
||||
<value>10</value>
|
||||
</property>
|
||||
|
||||
</configuration>
|
@ -0,0 +1,107 @@
|
||||
<?xml version="1.0" encoding="UTF-8"?>
|
||||
<?xml-stylesheet type="text/xsl" href="configuration.xsl"?>
|
||||
<!--
|
||||
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.
|
||||
-->
|
||||
|
||||
<!-- Put site-specific property overrides in this file. -->
|
||||
|
||||
<configuration>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.credentials.provider</name>
|
||||
<value>org.apache.hadoop.fs.cosn.auth.SimpleCredentialProvider</value>
|
||||
<description>
|
||||
This option allows the user to specify how to get the credentials.
|
||||
Comma-separated class names of credential provider classes which
|
||||
implement
|
||||
com.qcloud.cos.auth.COSCredentialsProvider:
|
||||
|
||||
1.org.apache.hadoop.fs.cosn.auth.SimpleCredentialProvider: Obtain the
|
||||
secret id and secret key
|
||||
from fs.cosn.userinfo.secretId and fs.cosn.userinfo.secretKey in
|
||||
core-site.xml
|
||||
2.org.apache.hadoop.fs.cosn.auth.EnvironmentVariableCredentialProvider:
|
||||
Obtain the secret id and secret key
|
||||
from the system environment variables named COS_SECRET_ID and
|
||||
COS_SECRET_KEY
|
||||
|
||||
If unspecified, the default order of credential providers is:
|
||||
1. org.apache.hadoop.fs.cosn.auth.SimpleCredentialProvider
|
||||
2. org.apache.hadoop.fs.cosn.auth.EnvironmentVariableCredentialProvider
|
||||
</description>
|
||||
</property>
|
||||
<property>
|
||||
<name>fs.cosn.impl</name>
|
||||
<value>org.apache.hadoop.fs.cosn.CosNFileSystem</value>
|
||||
<description>
|
||||
The implementation class of the CosN Filesystem.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.AbstractFileSystem.cosn.impl</name>
|
||||
<value>org.apache.hadoop.fs.cosn.CosN</value>
|
||||
<description>
|
||||
The implementation class of the CosN AbstractFileSystem.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.tmp.dir</name>
|
||||
<value>/tmp/hadoop_cos</value>
|
||||
<description>
|
||||
Temporary files will be placed here.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.block.size</name>
|
||||
<value>8388608</value>
|
||||
<description>
|
||||
Block size to use cosn filesystem, which is the part size for
|
||||
MultipartUpload.
|
||||
Considering the COS supports up to 10000 blocks, user should
|
||||
estimate the maximum size of a single file.
|
||||
for example, 8MB part size can allow writing a 78GB single file.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.upload.buffer.size</name>
|
||||
<value>536870912</value>
|
||||
<description>The total size of the memory buffer pool.</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.read.ahead.block.size</name>
|
||||
<value>1048576</value>
|
||||
<description>
|
||||
Bytes to read ahead during a seek() before closing and
|
||||
re-opening the cosn HTTP connection.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<property>
|
||||
<name>fs.cosn.read.ahead.queue.size</name>
|
||||
<value>64</value>
|
||||
<description>
|
||||
The length of the pre-read queue.
|
||||
</description>
|
||||
</property>
|
||||
|
||||
<include xmlns="http://www.w3.org/2001/XInclude" href="auth-keys.xml">
|
||||
<fallback/>
|
||||
</include>
|
||||
|
||||
</configuration>
|
@ -0,0 +1,18 @@
|
||||
# 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=info,stdout
|
||||
log4j.threshhold=ALL
|
||||
log4j.appender.stdout=org.apache.log4j.ConsoleAppender
|
||||
log4j.appender.stdout.layout=org.apache.log4j.PatternLayout
|
||||
log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} %-5p %c{2} (%F:%M(%L)) - %m%n
|
@ -31,6 +31,7 @@
|
||||
|
||||
<modules>
|
||||
<module>hadoop-cloud-storage</module>
|
||||
<module>hadoop-cos</module>
|
||||
</modules>
|
||||
|
||||
<build>
|
||||
|
@ -196,6 +196,7 @@
|
||||
<item name="Azure Data Lake Storage"
|
||||
href="hadoop-azure-datalake/index.html"/>
|
||||
<item name="OpenStack Swift" href="hadoop-openstack/index.html"/>
|
||||
<item name="Tencent COS" href="hadoop-cos/index.html"/>
|
||||
</menu>
|
||||
|
||||
<menu name="Auth" inherit="top">
|
||||
|
Loading…
Reference in New Issue
Block a user