diff --git a/README.md b/README.md index 08999ad..ef9f8ad 100644 --- a/README.md +++ b/README.md @@ -2,3 +2,12 @@ 此仓库包含了两个项目 - hadoop-obs项目:基于华为云OBS服务实现了hadoop文件系统抽象,位于master分支hadoop-huaweicloud目录,见对应的readme.md - flink-obs项目:基于华为云OBS服务实现了Flink文件系统抽象,位于flink-obs分支flink-obs-fs-hadoop目录,见对应的readme.md +- ranger-obs项目:实现了在ranger系统中管理OBS权限的功能,位于ranger-obs分支,见对应的readme.md + +----- +### Overview +This warehouse contains two items + +- Hadoop-obs project: The Hadoop file system is abstracted based on HUAWEI CLOUD OBS. The Hadoop file system is stored in the hadoop-huaweicloud directory of the master branch. For details, see readme.md. +- Flink-obs project: The Flink file system is abstracted based on HUAWEI CLOUD OBS. The Flink file system is stored in the flink-obs-fs-hadoop directory of the flink-obs branch. For details, see the readme.md file. +- ranger-obs project: manages OBS permissions in the ranger system. It is located in the ranger-obs branch. For details, see the corresponding readme.md file. diff --git a/hadoop-huaweicloud/README.md b/hadoop-huaweicloud/README.md index 31d23d2..6d314fb 100644 --- a/hadoop-huaweicloud/README.md +++ b/hadoop-huaweicloud/README.md @@ -1,12 +1,195 @@ -Version 3.1.1.46/2.8.3.46/2.7.2.46 +示例:mvn clean install -Pdist -Dhadoop.plat.version=3.1.1 -Dhadoop.version=3.1.1 -Dmaven.test.skip=true + +Pdist:将hadoop-obs依赖的obs java sdk以及okttp等依赖进行了shade,编译部署安装使用 +Dhadoop.version:定义了依赖的hadoop版本,目前仅支持依赖hadoop-2.8.x及以上版本 +Dhadoop.plat.version:定义了jar的命名规范,和-Dhadoop.version保持一致即可 +jar包命名规范:hadoop-huaweicloud-x.x.x-hw-y.jar包含义:前三位x.x.x为依赖的hadoop版本;最后一位y为hadoop-obs版本,例如:hadoop-huaweicloud-3.1.1-hw-45.jar,3.1.1是配套的hadoop版本,45是hadoop-obs的版本 +========================================================================= +Version 3.1.1.54.5 +``` +【特性】支持各个接口的耗时、操作类型、userName、endpoint等请求信息 +【特性】支持setAttr、getAttr等与accesslabel属性配合使用文件目录属性设置和获取, +使用方式: + fs.setXAttr(filePath, "AccessLabel", setValue);// 设置filePath目录的AccessLabel属性值 + fs.getXAttr(filePath, "AccessLabel");// 获取filePath目录的AccessLabel属性值 +注意:setXAttr和getXAttr仅支持设置/获取AccessLabel属性值,其他属性值的设置和获取不支持 +【优化】升级SDK版本至3.24.12,将SDK的使用版本从esdk-obs-java-optimise升级至esdk-obs-java-bundle +【优化】多段上传的大小最小由5M下放到1M,即现在支持多段的最小段大小为1M +【BUG】上传对象的缓存文件命名增加时间戳,修复缓存文件被覆盖数据不一致问题 +``` +========================================================================= + +Version 3.1.1.54.3 + +【优化】修复53.8版本引入的创建文件夹后获取权限为rw-rw-rw-问题,恢复默认的为rwxrwxrwx + +========================================================================= + +Version 3.1.1.54.1 + +【优化】升级SDK版本至3.24.3.1 + +========================================================================= + +Version 3.1.1.54.0/2.8.3.54.0 + +1. 【优化】支持append场景去除head行为,提升append的性能表现 +2. 【注】该版本性能优化特性需要手动配置core-site.xml的配置项,新增如下配置 +```txt + +fs.obs.appendRemoveHead +true + +``` +========================================================================= + +Version 3.1.1.53.8/2.8.3.53.8 + +1. 【优化】写数据流程加入sha256校验配置fs.obs.fast.upload.checksum.type,默认none +2. 【特性】支持hadoop distcp命令保留文件的属主(u)、属组(g)、权限(p)信息, 使用配置fs.obs.permissions.mode=disguise打开此特性 + +========================================================================= + +Version 3.1.1.53.7/2.8.3.53.7 + +1. 【BUG】修复MemArtsCC场景下,Cache miss时,读统计数据放大的问题 +2. 【优化】大数据回收站trashRoot目录路径优化,减少配置生命周期规则条数。使用配置fs.obs.hdfs.trash.version=2,默认路径前缀fs.obs.hdfs.trash.prefix=/user/.Trash + +========================================================================= + +Version 3.1.1.53.6.1/2.8.3.53.6.1 +【优化】升级OBS SDK版本为3.23.3.1, 支持建链重试, 解决运满满依赖冲突 + +========================================================================= + +Version 3.1.1.53.6/2.8.3.53.6 +【优化】OBSA适配MemArtsCC SDK接口修改 + +========================================================================= + +Version 3.1.1.53.5/2.8.3.53.5 +【优化】在write和read中补充缺失的上报异常信息 + +========================================================================= + +Version 3.1.1.53.4/2.8.3.53.4 +【特性】实现WithErrCode接口 实现MRS+DLI+OBS客户侧故障感知能力 + +========================================================================= + +Version 3.1.1.53.3/2.8.3.53.3 +【BUG】修复MemArtsCCClient初始化失败仍然调用流量上报接口的Bug + +========================================================================= + +Version 3.1.1.53.1/2.8.3.53.1 +【优化】新增对memartscc上报流量的开关 + +========================================================================= + +Version 3.1.1.53/2.8.3.53 +【特性】新增OBSA对memartscc上报流量统计的功能 + +========================================================================= + +Version 3.1.1.52.2/2.8.3.52.2 +【优化】memartscc跳过pyspark与spark引擎联动 + +========================================================================= + +Version 3.1.1.52.1/2.8.3.52.1 +【优化】新增memartscc跳过pyspark功能添加开关 + +========================================================================= + +Version 3.1.1.52/2.8.3.52 +【优化】增加写obs时disk缓存同步刷盘功能 + +========================================================================= + +Version 3.1.1.51/2.8.3.51 +【特性】支持obs select接口,试验阶段不可商用 + +========================================================================= + +Version 3.1.1.50.9/2.8.3.50.9 +【优化】写数据流程加入md5校验选项 + +========================================================================= + +Version 3.1.1.50.8.1/2.8.3.50.8.1 +【优化】新增MemArtsCC跳过pyspark开关 + +========================================================================= + +Version 3.1.1.50.8/2.8.3.50.8 +【优化】MemArtsCC Input Stream添加buffer pool + +========================================================================= + +Version 3.1.1.50.7/2.8.3.50.7 +【BUG】修复OBSA ReadFully 偶现EOFException + +========================================================================= + +Version 3.1.1.50.6/2.8.3.50.6 +【特性】支持 fs trash +【特性】去除Apache common lang 2.6依赖 使得OBSA支持 3.3.1 +【特性】支持delegationToken机制 +【BUG】override getHomeDirectory and fix mkdirs and fix putobject/appendobject retry + +========================================================================= + +Version 3.1.1.50.3/2.8.3.50.3 +【优化】新metric监控打点框架修改点:所有接口在流控错误均上报监控打点 + +========================================================================= + +Version 3.1.1.50.2/2.8.3.50.2 +【优化】OBSA getContentSummary在服务端不支持的情况下,不重试,静默回退至v1版本 +【优化】新metric监控打点框架 + +========================================================================= + +Version 3.1.1.50.1/2.8.3.50.1 +【特性】对接MemArtsCC特性(适配memartscc_3.22.4.T1版本) +【优化】将流控状态下的重试机制独立出来 +【优化】优化range读取流程中多出1字节 + +========================================================================= + +Version 3.1.1.50/2.8.3.50 +【特性】对接MemArtsCC特性(适配memartscc_3.22.3.B001版本) +【特性】getContentSummary性能提升 +【优化】升级obs sdk为3.22.3.1,支持getContentSummary性能提升 +【优化】缩减元数据访问次数 +【优化】代码可信整改 +【优化】遇到503流控时展示详细流控状态码 +【优化】为快速删除特性参数fs.obs.trash.dir添加默认值:FileSystem.getTrashRoot() + “FastDelete” +【优化】遇到400状态码时不重试 +【优化】异常日志中添加详细错误码,利用问题快速定位 +【优化】改变maven-shade-plugin插件行为,避免集成方对obs java sdk的重复引用 +【BUG】seek超出文件末尾时抛出EOFException + +========================================================================= + +Version 3.1.1.46.1/2.8.3.46.1 +【优化】inputStream seek 超出文件末尾时抛出EOFException + +========================================================================= + +Version 3.1.1.46/2.8.3.46 【BUG】“对象桶”场景下在服务端持续返回503的情况下rename操作依然显示成功 【优化】在flink或是flume等场景中将频繁调用append接口,此接口因输出流position判断不准确导致出现很多不必要的warn级别的日志 【优化】hadoop-obs 访问OBS的TCP建链超时由120s改为5s 【优化】添加aksk获取方式的INFO级别日志 【优化】升级obs sdk为3.21.8.2,解决XXE漏洞 【新增】新增fs.obs.outputstream.hflush.policy参数控制hflush和hsync方法的行为 +【新增】ranger-obs,ranger控制OBS鉴权,目前仅支持MRS集群集成此特性 -Version 3.1.1.45/2.8.3.45/2.7.2.45 +========================================================================= + +Version 3.1.1.45/2.8.3.45 修复问题: 1. 【功能】新增预读策略,优化顺序读性能 @@ -277,3 +460,156 @@ Version 2.8.3.19/2.7.2.19 修复问题: 1. 【功能】HBASE场景,未关闭JAVA SDK的连接,导致EOF异常; + +------ + +Example: mvn clean install -Pdist -Dhadoop.plat.version=3.1.1 -Dhadoop.version=3.1.1 -Dmaven.test.skip=true + +Pdist: Shades the dependencies such as obs java sdk and okttp on which hadoop-obs depends for compilation, deployment, and installation. +Dhadoop.version: defines the dependent Hadoop version. Currently, only hadoop-2.8.x and later versions are supported. +Dhadoop.plat.version: defines the jar naming rule. The value must be the same as that of -Dhadoop.version. +The JAR package name format is hadoop-huaweicloud-x.x.x-hw-y.jar. The first three characters x.x.x indicate the dependent Hadoop version. The last y indicates the hadoop-obs version, for example, hadoop-huaweicloud-3.1.1-hw-45.jar. 3.1.1 indicates the matching Hadoop version, and 45 indicates the hadoop-obs version. + +========================================================================= + +Version 3.1.1.53.8/2.8.3.53.8/2.7.2.53.8 + +1. [Optimization] The SHA256 check configuration fs.obs.fast.upload.checksum.type is added to the data write process. The default value is none. +2. [Feature] The hadoop distcp command can retain the file owner (u), owner group (g), and permission (p). This feature is enabled by configuring fs.obs.permissions.mode=disguise. + +========================================================================= + +Version 3.1.1.53.7/2.8.3.53.7/2.7.2.53.7 + +1. [BUG] In the MemArtsCC scenario, the read statistics are magnified when the cache misses. +2. [Optimization] The trashRoot directory path in the big data recycle bin is optimized to reduce the number of lifecycle rules to be configured. Use the configuration fs.obs.hdfs.trash.version=2. The default path prefix is fs.obs.hdfs.trash.prefix=/user/.Trash. + +========================================================================= + +Version 3.1.1.53.6.1/2.8.3.53.6.1/2.7.2.53.6.1 +[Optimization] The OBS SDK version is upgraded to 3.23.3.1, which supports link setup retry and resolves dependency conflicts when the system is full. + +========================================================================= + +Version 3.1.1.53.6/2.8.3.53.6/2.7.2.53.6 +[Optimization] The OBSA adapts to the MemArtsCC SDK interface. + +========================================================================= + +Version 3.1.1.53.5/2.8.3.53.5/2.7.2.53.5 +[Optimization] The missing exception information is added to the write and read fields. + +========================================================================= + +Version 3.1.1.53.4/2.8.3.53.4/2.7.2.53.4 +[Feature] The WithErrCode interface is used to detect faults on the customer side of MRS, DLI, and OBS. + +========================================================================= + +Version 3.1.1.53.3/2.8.3.53.3/2.7.2.53.3 +[BUG] Fix the bug that the traffic reporting interface is invoked after the MemArtsCCClient fails to be initialized. + +========================================================================= + +Version 3.1.1.53.1/2.8.3.53.1/2.7.2.53.1 +[Optimization] The switch for reporting the traffic volume of the memartscc is added. + +========================================================================= + +Version 3.1.1.53/2.8.3.53/2.7.2.53 +[Feature] The OBSA can collect statistics on the traffic reported by the memartscc. + +========================================================================= + +Version 3.1.1.52.2/2.8.3.52.2/2.7.2.52.2 +[Optimization] The memartscc skips the association between the Pyspark and Spark engine. + +========================================================================= + +Version 3.1.1.52.1/2.8.3.52.1/2.7.2.52.1 +[Optimization] The switch for skipping the pyspark function is added for the memartscc service. + +========================================================================= + +Version 3.1.1.52/2.8.3.52/2.7.2.52 +[Optimization] The disk cache flushing function is added when data is written to OBS. + +========================================================================= + +Version 3.1.1.51/2.8.3.51/2.7.2.51 +[Feature] The OBS select interface is supported. It cannot be put into commercial use in the trial phase. + +========================================================================= + +Version 3.1.1.50.9/2.8.3.50.9/2.7.2.50.9 +[Optimization] The MD5 check option is added to the data write process. + +========================================================================= + +Version 3.1.1.50.8.1/2.8.3.50.8.1/2.7.2.50.8.1 +[Optimization] The function of skipping pyspark for MemArtsCC is added. + +========================================================================= + +Version 3.1.1.50.8/2.8.3.50.8/2.7.2.50.8 +[Optimization] The buffer pool is added to the MemArtsCC Input Stream. + +========================================================================= + +Version 3.1.1.50.7/2.8.3.50.7/2.7.2.50.7 +[BUG] Occasionally EOFException Occurs in OBSA ReadFully + +========================================================================= + +Version 3.1.1.50.6/2.8.3.50.6/2.7.2.50.6 +[Feature] fs trash +[Feature] The dependency on Apache common lang 2.6 is removed so that OBSA supports 3.3. 1. +[Feature] The delegationToken mechanism is supported. +[BUG] override getHomeDirectory and fix mkdirs and fix putobject/appendobject retry + +========================================================================= + +Version 3.1.1.50.3/2.8.3.50.3/2.7.2.50.3 +[Optimization] In the new metric monitoring dotting framework, all interfaces report monitoring dotting when traffic control errors occur. + +========================================================================= + +Version 3.1.1.50.2/2.8.3.50.2/2.7.2.50.2 +[Optimization] When OBSA getContentSummary is not supported by the server, the system does not retry and silently rolls back to V1. +[Optimization] New metric monitoring dotting framework + +========================================================================= + +Version 3.1.1.50.1/2.8.3.50.1/2.7.2.50.1 +[Feature] Interconnection with MemArtsCC (memartscc_3.22.4.T1) +[Optimization] The retry mechanism in the flow control state is independent. +[Optimization] One extra byte is added in the range reading process. + +========================================================================= + +Version 3.1.1.50/2.8.3.50/2.7.2.50 +[Feature] Interconnection with MemArtsCC (memartscc_3.22.3.B001) +[Feature] The getContentSummary performance is improved. +[Optimization] The OBS SDK is upgraded to 3.22.3.1 to improve the getContentSummary performance. +[Optimization] Reduce the number of metadata access times. +[Optimization] Code trustworthiness rectification +[Optimization] The detailed flow control status code is displayed when the 503 flow control occurs. +[Optimization] The default value FileSystem.getTrashRoot() + FastDelete is added for the fs.obs.trash.dir parameter. +[Optimization] No retry is performed when status code 400 is received. +[Optimization] Detailed error codes are added to exception logs to quickly locate faults. +[Optimization] The behavior of the maven-shade-plugin plug-in is changed to prevent the integrator from repeatedly referencing the obs java SDK. +[BUG] EOFException is thrown when the seek exceeds the end of the file. + +========================================================================= + +Version 3.1.1.46.1/2.8.3.46.1/2.7.2.46.1 +[Optimization] When inputStream seek exceeds the end of the file, EOFException is thrown. + +========================================================================= + +Version 3.1.1.46/2.8.3.46/2.7.2.46 +[BUG] In the object bucket scenario, the rename operation is still successful even if the server continuously returns 503. +[Optimization] In scenarios such as Flink and Flume, the append interface is frequently invoked. This interface incorrectly determines the position of the output stream. As a result, many unnecessary warn-level logs are generated. +[Optimization] The TCP connection setup timeout for hadoop-obs to access OBS is changed from 120s to 5s. +[Optimization] The INFO-level log for obtaining the AK/SK is added. +[Optimization] The OBS SDK is upgraded to 3.21.8.2. diff --git a/hadoop-huaweicloud/dev-support/findbugs-exclude.xml b/hadoop-huaweicloud/dev-support/findbugs-exclude.xml index fa2556b..4a10562 100644 --- a/hadoop-huaweicloud/dev-support/findbugs-exclude.xml +++ b/hadoop-huaweicloud/dev-support/findbugs-exclude.xml @@ -1,30 +1,30 @@ - - - - - - - - - - - - - - + + + + + + + + + + + + + + \ No newline at end of file diff --git a/hadoop-huaweicloud/pom.xml b/hadoop-huaweicloud/pom.xml index c0e2e3f..e34da5f 100644 --- a/hadoop-huaweicloud/pom.xml +++ b/hadoop-huaweicloud/pom.xml @@ -16,9 +16,9 @@ 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"> 4.0.0 - com.huaweicloud + com.huaweicloud.obs hadoop-huaweicloud - 2.8.3-hw-46 + 3.1.1-hw-53.8 hadoop-huaweicloud This module contains code to support integration with OBS. @@ -26,12 +26,14 @@ jar + UTF-8 UTF-8 - 2.8.3.0101-hw-ei-14 - 3.21.8.2 - 2.8.3 - 46 + 3.1.1-hw-ei-315005 + 3.23.3.1 + 3.1.1 + 53.8 obs.shaded + 2.13.4.1 @@ -44,7 +46,7 @@ maven-shade-plugin 3.2.1 - false + true @@ -55,21 +57,21 @@ - - - com.jamesmurty.utils - ${shading.prefix}.com.jamesmurty.utils - - - - okio - ${shading.prefix}.okio - - - - okhttp3 - ${shading.prefix}.okhttp3 - + + + + + + + + + + + + + + + com.fasterxml.jackson @@ -81,30 +83,30 @@ - com.jamesmurty.utils:* - com.squareup.okio:* - com.squareup.okhttp3:* + + + com.huaweicloud:esdk-obs-java-optimised com.fasterxml.jackson.core:* - - com.squareup.okhttp3:* - - okhttp3/internal/connection/ExchangeFinder.class - okhttp3/internal/connection/Transmitter.class - okhttp3/internal/http/RetryAndFollowUpInterceptor.class - - - - com.squareup.okio:* - - okio/AsyncTimeout.class - okio/SegmentPool.class - - + + + + + + + + + + + + + + + *:* @@ -185,7 +187,7 @@ 8 8 - -Xlint:unchecked + -Xlint:all @@ -193,6 +195,33 @@ versions-maven-plugin 2.8.1 + + com.huawei.fuxi + fuxicov-maven-plugin + 1.0.9-SNAPSHOT + + + + + prepare-agent + + + + + com.huawei.* + + + + + + report + test + + report + + + + @@ -267,8 +296,17 @@ jetty-webapp org.eclipse.jetty + + commons-lang + commons-lang + + + com.fasterxml.jackson.core + jackson-databind + ${jackson.databind.version} + org.apache.hadoop hadoop-hdfs-client diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BasicMetricsConsumer.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BasicMetricsConsumer.java index ac6ded8..5313fda 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BasicMetricsConsumer.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BasicMetricsConsumer.java @@ -26,111 +26,70 @@ */ public interface BasicMetricsConsumer extends Closeable { + enum MetricKind { + normal,abnormal + } + class MetricRecord { /** * Operation name, such as listStatus. */ - private String opName; - - /** - * Operation result: true for success, or false for failure. - */ - private boolean success; + private OBSOperateAction obsOperateAction; /** * Operation cost time in ms. */ private long costTime; + /** + * Obs detail error msg + */ + private Exception exception; - private String opType; - - //opName - public static final String READ = "read"; - - public static final String CLOSE = "close"; - - public static final String READFULLY = "readFully"; - - static final String HFLUSH = "hflush"; - - static final String WRITE = "write"; - - static final String CREATE = "create"; - - static final String CREATE_NR = "createNonRecursive"; - - static final String APPEND = "append"; - - static final String RENAME = "rename"; - - static final String DELETE = "delete"; - - static final String LIST_STATUS = "listStatus"; - - static final String MKDIRS = "mkdirs"; - - static final String GET_FILE_STATUS = "getFileStatus"; - - static final String GET_CONTENT_SUMMARY = "getContentSummary"; - - static final String COPYFROMLOCAL = "copyFromLocalFile"; - - static final String LIST_FILES = "listFiles"; - - static final String LIST_LOCATED_STS = "listLocatedStatus"; - - static final String OPEN = "open"; - - //opType - public static final String ONEBYTE = "1byte"; - - public static final String BYTEBUF = "byteBuf"; - - public static final String INPUT = "input"; - - public static final String RANDOM = "random"; - - public static final String SEQ = "seq"; - - static final String OUTPUT = "output"; - - static final String FLAGS = "flags"; - - static final String NONRECURSIVE = "nonrecursive"; - - static final String RECURSIVE = "recursive"; - - static final String FS = "fs"; + /** + * normal logic or exception logic + */ + private MetricKind kind; - static final String OVERWRITE = "overwrite"; - public MetricRecord(String opType, String opName, boolean success, long costTime) { - this.opName = opName; - this.opType = opType; - this.success = success; + //正常: kind:指示是什么性能追踪类信息还是异常;opName 操作接口; + public MetricRecord(OBSOperateAction opName, long costTime, MetricKind kind) { + this.obsOperateAction = opName; this.costTime = costTime; + this.kind = kind; } - public String getOpName() { - return opName; + //异常:opName 操作类型; opDetail 代表obs错误码 + public MetricRecord(OBSOperateAction opName, Exception exception, MetricKind kind) { + this.obsOperateAction = opName; + this.exception = exception; + this.kind = kind; } - public boolean isSuccess() { - return success; + public OBSOperateAction getObsOperateAction() { + return obsOperateAction; } public long getCostTime() { return costTime; } - public String getOpType() { - return opType; + //获取异常实例 + public Exception getExceptionIns() { + return exception; + } + + public MetricKind getKind() { + return kind; + } + + public void setKind(MetricKind kind) { + this.kind = kind; } @Override public String toString() { - return "MetricRecord{" + "opName='" + opName + ", success=" + success + ", costTime=" + costTime - + ", opType=" + opType + '}'; + return "MetricRecord{" + "opName='" + obsOperateAction + ", costTime=" + costTime + + ", exception=" + exception + ",kind=" + kind + '}'; } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BlockingThreadPoolExecutorService.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BlockingThreadPoolExecutorService.java index 7583478..e3697e8 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BlockingThreadPoolExecutorService.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/BlockingThreadPoolExecutorService.java @@ -106,27 +106,11 @@ static ThreadFactory newDaemonThreadFactory(final String prefix) { }; } - /** - * A thread pool that that blocks clients submitting additional tasks if - * there are already {@code activeTasks} running threads and {@code - * waitingTasks} tasks waiting in its queue. - * - * @param activeTasks maximum number of active tasks - * @param waitingTasks maximum number of waiting tasks - * @param keepAliveTime time until threads are cleaned up in seconds - * @param prefixName prefix of name for threads - * @return new instance of BlockingThreadPoolExecutorService - */ static BlockingThreadPoolExecutorService newInstance(final int activeTasks, final int waitingTasks, final long keepAliveTime, final String prefixName) { - - /* Although we generally only expect up to waitingTasks tasks in the - queue, we need to be able to buffer all tasks in case dequeueing is - slower than enqueueing. */ final BlockingQueue workQueue = new LinkedBlockingQueue<>(waitingTasks + activeTasks); ThreadPoolExecutor eventProcessingExecutor = new ThreadPoolExecutor(activeTasks, activeTasks, keepAliveTime, TimeUnit.SECONDS, workQueue, newDaemonThreadFactory(prefixName), (r, executor) -> { - // This is not expected to happen. LOG.error("Could not submit task to executor {}", executor.toString()); }); eventProcessingExecutor.allowCoreThreadTimeOut(true); diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultMetricsConsumer.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultMetricsConsumer.java index 4aff4af..2af0536 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultMetricsConsumer.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultMetricsConsumer.java @@ -64,8 +64,14 @@ class DefaultMetricsConsumer implements BasicMetricsConsumer { @Override public boolean putMetrics(MetricRecord metricRecord) { if (LOG.isDebugEnabled()) { - LOG.debug("[Metrics]: url[{}], opName [{}], costTime[{}], " + "opResult[{}]", this.uri, - metricRecord.getOpName(), metricRecord.getCostTime(), metricRecord.isSuccess()); + if (metricRecord.getKind().equals(MetricKind.normal)) { + LOG.debug("[Metrics]: url[{}], action [{}], kind[{}], costTime[{}] ", this.uri, + metricRecord.getObsOperateAction(), metricRecord.getKind(), metricRecord.getCostTime()); + }else { + LOG.debug("[Metrics]: url[{}], action [{}], kind[{}], exception[{}] ", this.uri, + metricRecord.getObsOperateAction(), metricRecord.getKind(), metricRecord.getExceptionIns()); + } + } return true; } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultOBSClientFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultOBSClientFactory.java index 3bffb3a..a2fe3b5 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultOBSClientFactory.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/DefaultOBSClientFactory.java @@ -20,21 +20,16 @@ import com.obs.services.IObsCredentialsProvider; import com.obs.services.ObsClient; -import com.obs.services.ObsConfiguration; import com.obs.services.internal.ext.ExtObsConfiguration; import com.obs.services.model.AuthTypeEnum; -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.conf.Configured; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.IOException; -import java.lang.reflect.Constructor; -import java.lang.reflect.InvocationTargetException; import java.net.URI; -import java.util.Optional; /** * The default factory implementation, which calls the OBS SDK to configure and @@ -74,6 +69,9 @@ private static void initConnectionSettings(final Configuration conf, final ExtOb conf.set(OBSConstants.ENDPOINT, newEndPointWithSchema); } + String endPoint = conf.getTrimmed(OBSConstants.ENDPOINT, ""); + obsConf.setEndPoint(endPoint); + obsConf.setMaxErrorRetry( OBSCommonUtils.intOption(conf, OBSConstants.MAX_ERROR_RETRIES, OBSConstants.DEFAULT_MAX_ERROR_RETRIES, 0)); @@ -181,119 +179,6 @@ private static void initProxySupport(final Configuration conf, final ExtObsConfi } } - /** - * Creates an {@link ObsClient} from the established configuration. - * - * @param conf Hadoop configuration - * @param obsConf ObsConfiguration - * @param name URL - * @return ObsClient client - * @throws IOException on any failure to create Huawei OBS client - */ - private static ObsClient createHuaweiObsClient(final Configuration conf, final ObsConfiguration obsConf, - final URI name) throws IOException { - Class credentialsProviderClass; - BasicSessionCredential credentialsProvider; - ObsClient obsClient; - - try { - credentialsProviderClass = conf.getClass(OBSConstants.OBS_CREDENTIALS_PROVIDER, null); - } catch (RuntimeException e) { - Throwable c = e.getCause() != null ? e.getCause() : e; - throw new IOException("From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + c, c); - } - - if (credentialsProviderClass == null) { - return createObsClientWithoutCredentialsProvider(conf, obsConf, name); - } - - try { - Constructor cons = credentialsProviderClass.getDeclaredConstructor(URI.class, Configuration.class); - credentialsProvider = (BasicSessionCredential) cons.newInstance(name, conf); - } catch (NoSuchMethodException | SecurityException | IllegalAccessException | InstantiationException | InvocationTargetException e) { - Throwable c = e.getCause() != null ? e.getCause() : e; - throw new IOException("From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + c, c); - } - - LOG.info("create ObsClient using credentialsProvider: {}", credentialsProviderClass.getName()); - String sessionToken = credentialsProvider.getSessionToken(); - String ak = credentialsProvider.getOBSAccessKeyId(); - String sk = credentialsProvider.getOBSSecretKey(); - String endPoint = conf.getTrimmed(OBSConstants.ENDPOINT, ""); - obsConf.setEndPoint(endPoint); - if (sessionToken != null && sessionToken.length() != 0) { - obsClient = new ObsClient(ak, sk, sessionToken, obsConf); - } else { - obsClient = new ObsClient(ak, sk, obsConf); - } - return obsClient; - } - - private static ObsClient createObsClientWithoutCredentialsProvider(final Configuration conf, - final ObsConfiguration obsConf, final URI name) throws IOException { - ObsClient obsClient; - OBSLoginHelper.Login creds = OBSCommonUtils.getOBSAccessKeys(name, conf); - - String ak = creds.getUser(); - String sk = creds.getPassword(); - String token = creds.getToken(); - - String endPoint = conf.getTrimmed(OBSConstants.ENDPOINT, ""); - obsConf.setEndPoint(endPoint); - - if (!StringUtils.isEmpty(ak) || !StringUtils.isEmpty(sk)) { - LOG.info("create ObsClient using aksk from configuration"); - obsClient = new ObsClient(ak, sk, token, obsConf); - return obsClient; - } - - Class securityProviderClass; - try { - securityProviderClass = conf.getClass(OBSConstants.OBS_SECURITY_PROVIDER, null); - LOG.info("From option {} get {}", OBSConstants.OBS_SECURITY_PROVIDER, securityProviderClass); - } catch (RuntimeException e) { - Throwable c = e.getCause() != null ? e.getCause() : e; - throw new IOException("From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, c); - } - - if (securityProviderClass == null) { - LOG.info("create ObsClient when securityProviderClass is null"); - obsClient = new ObsClient(ak, sk, token, obsConf); - return obsClient; - } - - LOG.info("create ObsClient using securityProvider {}", securityProviderClass.getName()); - IObsCredentialsProvider securityProvider; - try { - Optional cons = tryGetConstructor(securityProviderClass, - new Class[] {URI.class, Configuration.class}); - - if (cons.isPresent()) { - securityProvider = (IObsCredentialsProvider) cons.get().newInstance(name, conf); - } else { - securityProvider = (IObsCredentialsProvider) securityProviderClass.getDeclaredConstructor() - .newInstance(); - } - - } catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException | RuntimeException e) { - Throwable c = e.getCause() != null ? e.getCause() : e; - throw new IOException("From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, c); - } - obsClient = new ObsClient(securityProvider, obsConf); - - return obsClient; - } - - public static Optional tryGetConstructor(final Class mainClss, final Class[] args) { - try { - Constructor constructor = mainClss.getDeclaredConstructor(args); - return Optional.ofNullable(constructor); - } catch (NoSuchMethodException e) { - // ignore - return Optional.empty(); - } - } - @Override public ObsClient createObsClient(final URI name) throws IOException { Configuration conf = getConf(); @@ -301,6 +186,7 @@ public ObsClient createObsClient(final URI name) throws IOException { initConnectionSettings(conf, obsConf); initProxySupport(conf, obsConf); - return createHuaweiObsClient(conf, obsConf, name); + IObsCredentialsProvider securityProvider = OBSSecurityProviderUtil.createObsSecurityProvider(conf, name); + return new ObsClient(securityProvider, obsConf); } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/AlreadyBeingCreatedException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSAlreadyBeingCreatedException.java similarity index 78% rename from hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/AlreadyBeingCreatedException.java rename to hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSAlreadyBeingCreatedException.java index e73d7fa..c82c8c5 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/AlreadyBeingCreatedException.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSAlreadyBeingCreatedException.java @@ -17,21 +17,16 @@ package org.apache.hadoop.fs.obs; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - import java.io.IOException; /** * The exception that happens when you ask to create a file that already is * being created, but is not closed yet. */ -@InterfaceAudience.Private -@InterfaceStability.Evolving -public class AlreadyBeingCreatedException extends IOException { +public class OBSAlreadyBeingCreatedException extends IOException { static final long serialVersionUID = 0x12308AD009L; - public AlreadyBeingCreatedException(String msg) { + public OBSAlreadyBeingCreatedException(String msg) { super(msg); } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java index 2d98bce..fd2cf8c 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSBlockOutputStream.java @@ -36,11 +36,13 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Syncable; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.File; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.OutputStream; @@ -203,37 +205,19 @@ private synchronized OBSDataBlocks.DataBlock createBlockIfNeeded() throws IOExce return activeBlock; } - /** - * Synchronized accessor to the active block. - * - * @return the active block; null if there isn't one. - */ synchronized OBSDataBlocks.DataBlock getActiveBlock() { return activeBlock; } - /** - * Set mock error. - * - * @param isException mock error - */ @VisibleForTesting void mockPutPartError(final boolean isException) { this.mockUploadPartError = isException; } - /** - * Predicate to query whether or not there is an active block. - * - * @return true if there is an active block. - */ private synchronized boolean hasActiveBlock() { return activeBlock != null; } - /** - * Clear the active block. - */ private synchronized void clearActiveBlock() { if (activeBlock != null) { LOG.debug("Clearing active block"); @@ -241,14 +225,11 @@ private synchronized void clearActiveBlock() { activeBlock = null; } - /** - * Check for the filesystem being open. - * - * @throws IOException if the filesystem is closed. - */ private void checkStreamOpen() throws IOException { if (closed) { - throw new IOException(uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + IOException ioe = new IOException(uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.write, ioe); + throw ioe; } } @@ -269,13 +250,6 @@ public synchronized void flush() throws IOException { } } - /** - * Writes a byte to the destination. If this causes the buffer to reach its - * limit, the actual upload is submitted to the threadpool. - * - * @param b the int of which the lowest byte is written - * @throws IOException on any problem - */ @Override public synchronized void write(final int b) throws IOException { fs.checkOpen(); @@ -284,27 +258,16 @@ public synchronized void write(final int b) throws IOException { write(singleCharWrite, 0, 1); } - /** - * Writes a range of bytes from to the memory buffer. If this causes the - * buffer to reach its limit, the actual upload is submitted to the - * threadpool and the remainder of the array is written to memory - * (recursively). - * - * @param source byte array containing - * @param offset offset in array where to start - * @param len number of bytes to be written - * @throws IOException on any problem - */ @Override public synchronized void write(@NotNull final byte[] source, final int offset, final int len) throws IOException { fs.checkOpen(); checkStreamOpen(); - long startTime = System.currentTimeMillis(); - long endTime; if (hasException.get()) { String closeWarning = String.format("write has error. bs : pre upload obs[%s] has error.", key); + IOException ioe = new IOException(closeWarning); + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.write, ioe); LOG.warn(closeWarning); - throw new IOException(closeWarning); + throw ioe; } OBSDataBlocks.validateWriteArgs(source, offset, len); if (len == 0) { @@ -316,19 +279,8 @@ public synchronized void write(@NotNull final byte[] source, final int offset, f int remainingCapacity = block.remainingCapacity(); try { innerWrite(source, offset, len, written, remainingCapacity); - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.WRITE, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } } catch (IOException e) { - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.WRITE, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.write, e); LOG.error("Write data for key {} of bucket {} error, error message {}", key, fs.getBucket(), e.getMessage()); throw e; @@ -380,7 +332,6 @@ private synchronized void innerWrite(final byte[] source, final int offset, fina private synchronized void uploadCurrentBlock() throws IOException { Preconditions.checkState(hasActiveBlock(), "No active block"); LOG.debug("Writing block # {}", blockCount); - try { if (multiPartUpload == null) { LOG.debug("Initiating Multipart upload"); @@ -392,7 +343,6 @@ private synchronized void uploadCurrentBlock() throws IOException { LOG.error("Upload current block on ({}/{}) failed.", fs.getBucket(), key, e); throw e; } finally { - // set the block to null, so the next write will create a new block. clearActiveBlock(); } } @@ -408,8 +358,6 @@ private synchronized void uploadCurrentBlock() throws IOException { */ @Override public synchronized void close() throws IOException { - long startTime = System.currentTimeMillis(); - long endTime; if (closed) { // already closed LOG.debug("Ignoring close() as stream is already closed"); @@ -419,14 +367,10 @@ public synchronized void close() throws IOException { if (hasException.get()) { String closeWarning = String.format("closed has error. bs : pre write obs[%s] has error.", key); LOG.warn(closeWarning); - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.OUTPUT, BasicMetricsConsumer.MetricRecord.CLOSE, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } - throw new IOException(closeWarning); + fs.removeFileBeingWritten(key); + IOException ioe = new IOException(closeWarning); + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.write, ioe); + throw ioe; } fs.checkOpen(); @@ -441,13 +385,6 @@ public synchronized void close() throws IOException { // directories writeOperationHelper.writeSuccessful(key); fs.removeFileBeingWritten(key); - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.OUTPUT, BasicMetricsConsumer.MetricRecord.CLOSE, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } closed = true; } @@ -457,8 +394,19 @@ public synchronized void close() throws IOException { * @throws IOException any problem in append or put object */ private synchronized void putObjectIfNeedAppend() throws IOException { - if (appendAble.get() && fs.exists(OBSCommonUtils.keyToQualifiedPath(fs, key))) { - appendFsFile(); + FileStatus fileStatus = null; + boolean exists = true; + if (appendAble.get()) { + try { + fileStatus = fs.getFileStatus(OBSCommonUtils.keyToPath(key)); + exists = fileStatus != null; + } catch (FileNotFoundException e) { + exists = false; + } + } + + if (appendAble.get() && exists) { + appendFsFile(fileStatus); } else { putObject(); } @@ -469,17 +417,26 @@ private synchronized void putObjectIfNeedAppend() throws IOException { * * @throws IOException any problem */ - private synchronized void appendFsFile() throws IOException { + private synchronized void appendFsFile(FileStatus fileStatus) throws IOException { LOG.debug("bucket is posix, to append file. key is {}", key); final OBSDataBlocks.DataBlock block = getActiveBlock(); + if (block == null) { + throw new IOException("block is null"); + } WriteFileRequest writeFileReq; if (block instanceof OBSDataBlocks.DiskBlock) { - writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key, objectLen, (File) block.startUpload()); + writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key, objectLen, (File) block.startUpload(), fileStatus); } else { - writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key, objectLen, (InputStream) block.startUpload()); + writeFileReq = OBSCommonUtils.newAppendFileRequest(fs, key, objectLen, (InputStream) block.startUpload(), fileStatus); + } + try { + OBSCommonUtils.appendFile(fs, writeFileReq); + objectLen += block.dataSize(); + }finally { + if (writeFileReq.getInput() != null) { + writeFileReq.getInput().close(); + } } - OBSCommonUtils.appendFile(fs, writeFileReq); - objectLen += block.dataSize(); } /** @@ -494,14 +451,17 @@ private synchronized void putObject() throws IOException { final OBSDataBlocks.DataBlock block = getActiveBlock(); clearActiveBlock(); + if (block == null) { + throw new IOException("block is null"); + } final int size = block.dataSize(); final PutObjectRequest putObjectRequest; if (block instanceof OBSDataBlocks.DiskBlock) { - putObjectRequest = writeOperationHelper.newPutRequest(key, (File) block.startUpload()); - + putObjectRequest = writeOperationHelper.newPutRequest(key, (File) block.startUpload(), + block.getChecksumType(), block.getChecksum()); // getChecksum should be called after startUpload } else { - putObjectRequest = writeOperationHelper.newPutRequest(key, (InputStream) block.startUpload(), size); - + putObjectRequest = writeOperationHelper.newPutRequest(key, (InputStream) block.startUpload(), size, + block.getChecksumType(), block.getChecksum()); // getChecksum should be called after startUpload } putObjectRequest.setAcl(fs.getCannedACL()); fs.getSchemeStatistics().incrementWriteOps(1); @@ -511,6 +471,9 @@ private synchronized void putObject() throws IOException { writeOperationHelper.putObject(putObjectRequest); objectLen += size; } finally { + if (putObjectRequest.getInput() != null) { + putObjectRequest.getInput().close(); + } OBSCommonUtils.closeAll(block); } } @@ -538,7 +501,6 @@ public synchronized void sync() throws IOException { public synchronized void hflush() throws IOException { fs.checkOpen(); checkStreamOpen(); - long startTime = System.currentTimeMillis(); switch (this.hflushPolicy) { case OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_SYNC: // hflush hsyn same @@ -555,13 +517,6 @@ public synchronized void hflush() throws IOException { default: throw new IOException(String.format("unsupported downgrade policy '%s'", this.hflushPolicy)); } - - long endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.HFLUSH, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } } /** @@ -617,9 +572,15 @@ private synchronized void uploadWriteBlocks(final OBSDataBlocks.DataBlock block, } else { // there has already been at least one block scheduled for upload; // put up the current then wait - if (hasBlock && block.hasData()) { - // send last part - uploadCurrentBlock(); + if (hasBlock) { + if (block == null) { + throw new IOException("block is null"); + } + + if (block.hasData()) { + // send last part + uploadCurrentBlock(); + } } // wait for the partial uploads to finish final List> partETags = multiPartUpload.waitForAllPartUploads(); @@ -652,6 +613,7 @@ private synchronized void completeCurrentBlock() throws IOException { } finally { OBSCommonUtils.closeAll(block); clearActiveBlock(); + fs.removeFileBeingWritten(key); } } @@ -679,7 +641,6 @@ private synchronized void flushCurrentBlock() throws IOException { public synchronized void hsync() throws IOException { fs.checkOpen(); checkStreamOpen(); - long startTime = System.currentTimeMillis(); switch (this.hflushPolicy) { case OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_SYNC: // hflush hsyn same @@ -696,12 +657,6 @@ public synchronized void hsync() throws IOException { default: throw new IOException(String.format("unsupported downgrade policy '%s'", this.hflushPolicy)); } - long endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.HFLUSH, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } } /** @@ -731,19 +686,20 @@ private class MultiPartUpload { * @param block block to upload * @throws IOException upload failure */ - private void uploadBlockAsync(final OBSDataBlocks.DataBlock block) throws IOException { + private void uploadBlockAsync(@NotNull final OBSDataBlocks.DataBlock block) throws IOException { LOG.debug("Queueing upload of {}", block); - + if (block == null) { + throw new IOException("block is null"); + } final int size = block.dataSize(); final int currentPartNumber = partETagsFutures.size() + 1; final UploadPartRequest request; if (block instanceof OBSDataBlocks.DiskBlock) { request = writeOperationHelper.newUploadPartRequest(key, uploadId, currentPartNumber, size, - (File) block.startUpload()); + (File) block.startUpload(), block.getChecksumType(), block.getChecksum()); // getChecksum should be called after startUpload } else { request = writeOperationHelper.newUploadPartRequest(key, uploadId, currentPartNumber, size, - (InputStream) block.startUpload()); - + (InputStream) block.startUpload(), block.getChecksumType(), block.getChecksum()); // getChecksum should be called after startUpload } ListenableFuture> partETagFuture = executorService.submit(() -> { // this is the queued upload operation @@ -759,14 +715,11 @@ private void uploadBlockAsync(final OBSDataBlocks.DataBlock block) throws IOExce if (LOG.isDebugEnabled()) { LOG.debug("Completed upload of {} to part {}", block, partETag); } - } catch (ObsException e) { - // catch all exception + } catch (IOException e) { hasException.set(true); - IOException ioException = OBSCommonUtils.translateException("UploadPart", key, e); - LOG.error("UploadPart failed (ObsException). {}", ioException.getMessage()); - throw ioException; + throw e; } finally { - // close the stream and block + request.getInput().close(); OBSCommonUtils.closeAll(block); } return new Pair(partETag, size); @@ -774,12 +727,6 @@ private void uploadBlockAsync(final OBSDataBlocks.DataBlock block) throws IOExce partETagsFutures.add(partETagFuture); } - /** - * Block awaiting all outstanding uploads to complete. - * - * @return list of results - * @throws IOException IO Problems - */ private List> waitForAllPartUploads() throws IOException { LOG.debug("Waiting for {} uploads to complete", partETagsFutures.size()); try { diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCallable.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCallable.java new file mode 100644 index 0000000..5de7884 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCallable.java @@ -0,0 +1,14 @@ +package org.apache.hadoop.fs.obs; + +import java.io.IOException; + +/** + * description + * + * @since 2022-04-13 + */ +@FunctionalInterface +public interface OBSCallable { + + T call() throws IOException; +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java index ca29671..9b8276c 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSCommonUtils.java @@ -7,6 +7,7 @@ import com.obs.services.model.AbortMultipartUploadRequest; import com.obs.services.model.DeleteObjectsRequest; import com.obs.services.model.DeleteObjectsResult; +import com.obs.services.model.GetObjectMetadataRequest; import com.obs.services.model.KeyAndVersion; import com.obs.services.model.ListMultipartUploadsRequest; import com.obs.services.model.ListObjectsRequest; @@ -20,13 +21,9 @@ import com.obs.services.model.UploadPartRequest; import com.obs.services.model.UploadPartResult; import com.obs.services.model.fs.FSStatusEnum; -import com.obs.services.model.fs.GetAttributeRequest; import com.obs.services.model.fs.GetBucketFSStatusRequest; -import com.obs.services.model.fs.GetBucketFSStatusResult; -import com.obs.services.model.fs.ObsFSAttribute; import com.obs.services.model.fs.WriteFileRequest; -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FileAlreadyExistsException; @@ -37,6 +34,8 @@ import org.apache.hadoop.fs.ParentNotDirectoryException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.fs.obs.security.ObsDelegationTokenManger; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.ProviderUtils; import org.apache.hadoop.util.DiskChecker; @@ -45,6 +44,7 @@ import java.io.EOFException; import java.io.File; +import java.io.FileInputStream; import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; @@ -54,12 +54,14 @@ import java.util.Date; import java.util.EnumSet; import java.util.List; +import java.util.Locale; import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutionException; import java.util.concurrent.ExecutorService; /** - * Common utils for {@link OBSFileSystem}. + * Common utils */ //CHECKSTYLE:OFF public final class OBSCommonUtils { @@ -73,7 +75,12 @@ public final class OBSCommonUtils { /** * Moved permanently response code. */ - static final int MOVED_PERMANENTLY_CODE = 301; + static final int OTHER_CODE = -1; + + /** + * IllegalArgument response code. + */ + static final int IllEGALARGUMENT_CODE = 400; /** * Unauthorized response code. @@ -90,6 +97,11 @@ public final class OBSCommonUtils { */ static final int NOT_FOUND_CODE = 404; + /** + * Method not allowed + */ + static final int NOT_ALLOWED_CODE = 405; + /** * File conflict. */ @@ -106,25 +118,36 @@ public final class OBSCommonUtils { static final int EOF_CODE = 416; /** - * Core property for provider path. Duplicated here for consistent code - * across Hadoop version: {@value}. + * error response code. */ - static final String CREDENTIAL_PROVIDER_PATH = "hadoop.security.credential.provider.path"; + static final int ERROR_CODE = 503; /** - * Max time in milliseconds to retry when request failed. + * qos error code */ - public static long MAX_TIME_IN_MILLISECONDS_TO_RETRY = 180000; + public static final String DETAIL_QOS_CODE = "GetQosTokenException"; /** - * Min time in milliseconds to sleep between retry intervals. + * qos indicator code */ - static final int MIN_TIME_IN_MILLISECONDS_TO_SLEEP = 50; + static final String DETAIL_QOS_INDICATOR_601 = "601"; + + static final String DETAIL_QOS_INDICATOR_602 = "602"; + + static OBSInvoker obsInvoker; + /** - * Max time in milliseconds to sleep between retry intervals. + * Core property for provider path. Duplicated here for consistent code + * across Hadoop version: {@value}. */ - static final int MAX_TIME_IN_MILLISECONDS_TO_SLEEP = 30000; + static final String CREDENTIAL_PROVIDER_PATH = "hadoop.security.credential.provider.path"; + + static long retryMaxTime = OBSConstants.DEFAULT_RETRY_MAXTIME; + + static long retrySleepBaseTime = OBSConstants.DEFAULT_RETRY_SLEEP_BASETIME; + + static long retrySleepMaxTime = OBSConstants.DEFAULT_RETRY_SLEEP_MAXTIME; /** * Variable base of power function. @@ -144,16 +167,40 @@ public final class OBSCommonUtils { private OBSCommonUtils() { } + + public static void init(OBSFileSystem fs, Configuration conf) { + obsInvoker = new OBSInvoker(fs, new OBSRetryPolicy(conf), OBSInvoker.LOG_EVENT); + OBSPosixBucketUtils.init(conf); + } + + public static ObsDelegationTokenManger initDelegationTokenManger(OBSFileSystem fs, URI uri, Configuration conf) + throws IOException { + ObsDelegationTokenManger obsDelegationTokenManger = null; + if (ObsDelegationTokenManger.hasDelegationTokenProviders(conf)) { + LOG.debug("Initializing ObsDelegationTokenManager for {}", uri); + obsDelegationTokenManger = new ObsDelegationTokenManger(); + obsDelegationTokenManger.initialize(fs, uri, conf); + } + return obsDelegationTokenManger; + } + + public static OBSInvoker getOBSInvoker() { + return obsInvoker; + } + /** * Set the max time in millisecond to retry on error. - * @param maxTime max time in millisecond to set for retry + * + * @param retryMaxTime max time in millisecond to set for retry */ - static void setMaxTimeInMillisecondsToRetry(long maxTime) { - if (maxTime <= 0) { - LOG.warn("Invalid time[{}] to set for retry on error.", maxTime); - maxTime = OBSConstants.DEFAULT_TIME_IN_MILLISECOND_TO_RETRY; + static void setRetryTime(long retryMaxTime, long retrySleepBaseTime, long retrySleepMaxTime) { + if (retryMaxTime <= 0) { + LOG.warn("Invalid time[{}] to set for retry on error.", retryMaxTime); + retryMaxTime = OBSConstants.DEFAULT_RETRY_MAXTIME; } - MAX_TIME_IN_MILLISECONDS_TO_RETRY = maxTime; + OBSCommonUtils.retryMaxTime = retryMaxTime; + OBSCommonUtils.retrySleepBaseTime = retrySleepBaseTime; + OBSCommonUtils.retrySleepMaxTime = retrySleepMaxTime; } /** @@ -167,57 +214,17 @@ static void setMaxTimeInMillisecondsToRetry(long maxTime) { */ static boolean getBucketFsStatus(final ObsClient obs, final String bucketName) throws FileNotFoundException, IOException { - GetBucketFSStatusRequest request = new GetBucketFSStatusRequest(); - request.setBucketName(bucketName); - FSStatusEnum fsStatus = innerGetBucketFsStatus(obs, request); - return FSStatusEnum.ENABLED == fsStatus; - } - - /** - * Get the fs status of the bucket with failure retry. - * - * @param obs OBS client instance - * @param request information to get bucket FsStatus - * @return boolean value indicating if this bucket is a posix bucket - */ - private static FSStatusEnum innerGetBucketFsStatus(final ObsClient obs, final GetBucketFSStatusRequest request) - throws IOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - GetBucketFSStatusResult getBucketFsStatusResult; - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - getBucketFsStatusResult = obs.getBucketFSStatus(request); - return getBucketFsStatusResult.getStatus(); - } catch (ObsException e) { - LOG.debug("Failed to getBucketFsStatus for [{}], retry time [{}], " + "exception [{}]", - request.getBucketName(), retryTime, e); - - IOException ioException = OBSCommonUtils.translateException("getBucketFSStatus", - request.getBucketName(), e); - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - - delayMs = getSleepTimeInMs(retryTime); - retryTime++; - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw e; - } - } - } - + GetBucketFSStatusRequest request = new GetBucketFSStatusRequest(bucketName); + FSStatusEnum fsStatus; try { - getBucketFsStatusResult = obs.getBucketFSStatus(request); - } catch (ObsException e) { - LOG.debug("Failed to getBucketFsStatus for [{}], retry time [{}], " + "exception [{}]", - request.getBucketName(), retryTime, e); - throw OBSCommonUtils.translateException("getBucketFSStatus", request.getBucketName(), e); + fsStatus = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.getBucketFsStatus, + request.getBucketName(), () -> { + return obs.getBucketFSStatus(request).getStatus(); + },true); + } catch (FileNotFoundException e) { + throw new FileNotFoundException("Bucket " + bucketName + " does not exist"); } - return getBucketFsStatusResult.getStatus(); + return FSStatusEnum.ENABLED == fsStatus; } /** @@ -310,7 +317,7 @@ static String pathToKey(final OBSFileSystem owner, final Path path) { * @return the with a trailing "/", or, if it is the root key, "", */ static String maybeAddTrailingSlash(final String key) { - if (!StringUtils.isEmpty(key) && !key.endsWith("/")) { + if (!isStringEmpty(key) && !key.endsWith("/")) { return key + '/'; } else { return key; @@ -318,7 +325,7 @@ static String maybeAddTrailingSlash(final String key) { } /** - * Convert a path back to a key. + * Convert a key back to a Path. * * @param key input key * @return the path from this key @@ -356,7 +363,7 @@ static Path qualify(final OBSFileSystem owner, final Path path) { * @return new key */ static String maybeDeleteBeginningSlash(final String key) { - return !StringUtils.isEmpty(key) && key.startsWith("/") ? key.substring(1) : key; + return !isStringEmpty(key) && key.startsWith("/") ? key.substring(1) : key; } /** @@ -366,7 +373,7 @@ static String maybeDeleteBeginningSlash(final String key) { * @return new key */ static String maybeAddBeginningSlash(final String key) { - return !StringUtils.isEmpty(key) && !key.startsWith("/") ? "/" + key : key; + return !isStringEmpty(key) && !key.startsWith("/") ? "/" + key : key; } /** @@ -381,52 +388,74 @@ static String maybeAddBeginningSlash(final String key) { */ public static IOException translateException(final String operation, final String path, final ObsException exception) { - String message = String.format( - "%s%s: status [%d] - request id [%s] " + "- error code [%s] - error message [%s] - trace :%s ", operation, - path != null ? " on " + path : "", exception.getResponseCode(), exception.getErrorRequestId(), - exception.getErrorCode(), exception.getErrorMessage(), exception); + String headerErrorCode = null; + Map responseHeaders = exception.getResponseHeaders(); + if (responseHeaders != null) { + for (Map.Entry entry : responseHeaders.entrySet()) { + if (entry.getKey().equals("error-code")) { + headerErrorCode = entry.getValue(); + break; + } + } + } + String indicatorCode = exception.getErrorIndicator(); + String bodyErrorCode = exception.getErrorCode(); + String errorCode = bodyErrorCode != null ? bodyErrorCode : + headerErrorCode != null ? headerErrorCode : indicatorCode; + + String message = String.format(Locale.ROOT,"%s%s: ResponseCode[%d],ErrorCode[%s],ErrorMessage[%s],RequestId[%s]", + operation, path == null || path.length() == 0 ? "" : " on " + path, exception.getResponseCode(), + errorCode, exception.getErrorMessage(),exception.getErrorRequestId()); IOException ioe; int status = exception.getResponseCode(); switch (status) { - case MOVED_PERMANENTLY_CODE: - message = String.format("Received permanent redirect response, " + "status [%d] - request id [%s] - " - + "error code [%s] - message [%s]", exception.getResponseCode(), exception.getErrorRequestId(), - exception.getErrorCode(), exception.getErrorMessage()); - ioe = new OBSIOException(message, exception); + case IllEGALARGUMENT_CODE: + OBSIllegalArgumentException illegalArgumentException = new OBSIllegalArgumentException(message); + illegalArgumentException.initCause(exception); + illegalArgumentException.setErrCode(errorCode); + ioe = illegalArgumentException; break; - // permissions case UNAUTHORIZED_CODE: case FORBIDDEN_CODE: ioe = new AccessControlException(message); ioe.initCause(exception); break; - - // the object isn't there case NOT_FOUND_CODE: case GONE_CODE: ioe = new FileNotFoundException(message); ioe.initCause(exception); break; - + case NOT_ALLOWED_CODE: + OBSMethodNotAllowedException methodNotAllowedException = new OBSMethodNotAllowedException(message); + methodNotAllowedException.initCause(exception); + methodNotAllowedException.setErrCode(errorCode); + ioe = methodNotAllowedException; + break; case CONFLICT_CODE: - ioe = new FileConflictException(message); - ioe.initCause(exception); + OBSFileConflictException fileConflictException = new OBSFileConflictException(message); + fileConflictException.initCause(exception); + fileConflictException.setErrCode(errorCode); + ioe = fileConflictException; break; - - // out of range. This may happen if an object is overwritten with - // a shorter one while it is being read. case EOF_CODE: ioe = new EOFException(message); ioe.initCause(exception); break; - default: - // no specific exit code. Choose an IOE subclass based on the - // class - // of the caught exception - ioe = new OBSIOException(message, exception); + if (ERROR_CODE == status && ( + DETAIL_QOS_CODE.equals(errorCode) || + DETAIL_QOS_INDICATOR_601.equals(errorCode) || + DETAIL_QOS_INDICATOR_602.equals(errorCode))) { + OBSQosException qosException = new OBSQosException(message, exception); + qosException.setErrCode(errorCode); + ioe = qosException; + } else { + OBSIOException ioException = new OBSIOException(message, exception); + ioException.setErrCode(errorCode); + ioe = ioException; + } break; } return ioe; @@ -457,52 +486,10 @@ static void blockRootDelete(final String bucket, final String key) throws Invali */ static void deleteObject(final OBSFileSystem owner, final String key) throws IOException { blockRootDelete(owner.getBucket(), key); - try { - innerDeleteObject(owner, key); - return; - } catch (ObsException e) { - LOG.error("Failed to deleteObject for [{}], exception [{}]", key, translateException("delete", key, e)); - throw translateException("delete", key, e); - } - } - - /** - * Delete an object with failure retry. Increments the {@code - * OBJECT_DELETE_REQUESTS} and write operation statistics. - * - * @param owner the owner OBSFileSystem instance. - * @param key key to blob to delete. - */ - private static void innerDeleteObject(final OBSFileSystem owner, final String key) throws IOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - owner.getObsClient().deleteObject(owner.getBucket(), key); - owner.getSchemeStatistics().incrementWriteOps(1); - return; - } catch (ObsException e) { - LOG.debug("Delete path failed with [{}], " + "retry time [{}] - request id [{}] - " - + "error code [{}] - error message [{}]", e.getResponseCode(), retryTime, e.getErrorRequestId(), - e.getErrorCode(), e.getErrorMessage()); - - IOException ioException = OBSCommonUtils.translateException("innerDeleteObject", key, e); - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - - delayMs = getSleepTimeInMs(retryTime); - retryTime++; - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw ioException; - } - } - } - - owner.getObsClient().deleteObject(owner.getBucket(), key); + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.delete, key, () -> { + owner.getObsClient().deleteObject(owner.getBucket(), key); + return null; + }, true); owner.getSchemeStatistics().incrementWriteOps(1); } @@ -521,8 +508,9 @@ static void deleteObjects(final OBSFileSystem owner, final DeleteObjectsRequest result = owner.getObsClient().deleteObjects(deleteRequest); owner.getSchemeStatistics().incrementWriteOps(1); } catch (ObsException e) { - LOG.warn("delete objects failed, request [{}], request id [{}] - " + "error code [{}] - error message [{}]", - deleteRequest, e.getErrorRequestId(), e.getErrorCode(), e.getErrorMessage()); + LOG.warn("bulk delete objects failed: request [{}], response code [{}], error code [{}], " + + "error message [{}], request id [{}]", deleteRequest, e.getResponseCode(), e.getErrorCode(), + e.getErrorMessage(), e.getErrorRequestId()); for (KeyAndVersion keyAndVersion : deleteRequest.getKeyAndVersionsList()) { deleteObject(owner, keyAndVersion.getKey()); } @@ -533,8 +521,11 @@ static void deleteObjects(final OBSFileSystem owner, final DeleteObjectsRequest if (result != null) { List errorResults = result.getErrorResults(); if (!errorResults.isEmpty()) { - LOG.warn("bulk delete {} objects, {} failed, begin to delete " + "one by one.", - deleteRequest.getKeyAndVersionsList().size(), errorResults.size()); + LOG.warn("bulk delete {} objects: {} failed, request id [{}].begin to delete one by one.detail info " + + "example:key[{}],error code [{}],error message [{}]", + deleteRequest.getKeyAndVersionsList().size(), errorResults.size(), result.getRequestId(), + errorResults.get(0).getObjectKey(), errorResults.get(0).getErrorCode(), + errorResults.get(0).getMessage()); for (DeleteObjectsResult.ErrorResult errorResult : errorResults) { deleteObject(owner, errorResult.getObjectKey()); } @@ -552,9 +543,11 @@ static void deleteObjects(final OBSFileSystem owner, final DeleteObjectsRequest * @return the request */ static PutObjectRequest newPutObjectRequest(final OBSFileSystem owner, final String key, - final ObjectMetadata metadata, final File srcfile) { + final ObjectMetadata metadata, final File srcfile) throws FileNotFoundException { Preconditions.checkNotNull(srcfile); - PutObjectRequest putObjectRequest = new PutObjectRequest(owner.getBucket(), key, srcfile); + PutObjectRequest putObjectRequest = new PutObjectRequest(owner.getBucket(), key); + putObjectRequest.setInput(new FileInputStream(srcfile)); + putObjectRequest.setAutoClose(false); putObjectRequest.setAcl(owner.getCannedACL()); putObjectRequest.setMetadata(metadata); if (owner.getSse().isSseCEnable()) { @@ -601,15 +594,17 @@ static PutObjectRequest newPutObjectRequest(final OBSFileSystem owner, final Str * @throws ObsException on problems */ static PutObjectResult putObjectDirect(final OBSFileSystem owner, final PutObjectRequest putObjectRequest) - throws ObsException { - long len; - if (putObjectRequest.getFile() != null) { - len = putObjectRequest.getFile().length(); - } else { - len = putObjectRequest.getMetadata().getContentLength(); - } + throws IOException { + PutObjectResult result = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.putObject, + putObjectRequest.getObjectKey(), () -> { + if (putObjectRequest.getInput() instanceof FileInputStream) { + ((FileInputStream) putObjectRequest.getInput()).getChannel().position(0); + } + return owner.getObsClient().putObject(putObjectRequest); + },true); - PutObjectResult result = owner.getObsClient().putObject(putObjectRequest); + long len = putObjectRequest.getFile() != null ? putObjectRequest.getFile().length() : + putObjectRequest.getMetadata().getContentLength(); owner.getSchemeStatistics().incrementWriteOps(1); owner.getSchemeStatistics().incrementBytesWritten(len); return result; @@ -625,11 +620,16 @@ static PutObjectResult putObjectDirect(final OBSFileSystem owner, final PutObjec * @return the result of the operation. * @throws ObsException on problems */ - static UploadPartResult uploadPart(final OBSFileSystem owner, final UploadPartRequest request) throws ObsException { - long len = request.getPartSize(); - UploadPartResult uploadPartResult = owner.getObsClient().uploadPart(request); + static UploadPartResult uploadPart(final OBSFileSystem owner, final UploadPartRequest request) throws IOException { + UploadPartResult uploadPartResult = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.uploadPart, + request.getObjectKey(), () -> { + if (request.getInput() instanceof FileInputStream) { + ((FileInputStream) request.getInput()).getChannel().position(0); + } + return owner.getObsClient().uploadPart(request); + },true); owner.getSchemeStatistics().incrementWriteOps(1); - owner.getSchemeStatistics().incrementBytesWritten(len); + owner.getSchemeStatistics().incrementBytesWritten(request.getPartSize()); return uploadPartResult; } @@ -703,7 +703,7 @@ static IOException translateException(final String operation, final Path path, f * @throws IOException due to an IO problem. * @throws ObsException on failures inside the OBS SDK */ - static FileStatus[] innerListStatus(final OBSFileSystem owner, final Path f, final boolean recursive) + static FileStatus[] listStatus(final OBSFileSystem owner, final Path f, final boolean recursive) throws FileNotFoundException, IOException, ObsException { Path path = qualify(owner, f); String key = pathToKey(owner, path); @@ -711,8 +711,8 @@ static FileStatus[] innerListStatus(final OBSFileSystem owner, final Path f, fin List result; final FileStatus fileStatus; try { - fileStatus = innerGetFileStatusWithRetry(owner, path); - } catch (FileConflictException e) { + fileStatus = getFileStatusWithRetry(owner, path); + } catch (OBSFileConflictException e) { throw new AccessControlException(e); } @@ -724,7 +724,7 @@ static FileStatus[] innerListStatus(final OBSFileSystem owner, final Path f, fin OBSListing.FileStatusListingIterator files = owner.getObsListing() .createFileStatusListingIterator(path, request, OBSListing.ACCEPT_ALL, - new OBSListing.AcceptAllButSelfAndS3nDirs(path)); + new OBSListing.AcceptAllButSelfAndOBSDirs(path)); result = new ArrayList<>(files.getBatchSize()); while (files.hasNext()) { result.add(files.next()); @@ -806,12 +806,12 @@ static boolean rejectRootDirectoryDelete(final String bucket, final boolean isEm * @throws IOException other IO problems * @throws ObsException on failures inside the OBS SDK */ - static boolean innerMkdirs(final OBSFileSystem owner, final Path path) + static boolean mkdirs(final OBSFileSystem owner, final Path path) throws IOException, FileAlreadyExistsException, ObsException { LOG.debug("Making directory: {}", path); FileStatus fileStatus; try { - fileStatus = innerGetFileStatusWithRetry(owner, path); + fileStatus = getFileStatusWithRetry(owner, path); if (fileStatus.isDirectory()) { return true; @@ -819,33 +819,37 @@ static boolean innerMkdirs(final OBSFileSystem owner, final Path path) throw new FileAlreadyExistsException("Path is a file: " + path); } } catch (FileNotFoundException e) { - Path fPart = path.getParent(); - do { - try { - fileStatus = innerGetFileStatusWithRetry(owner, fPart); - if (fileStatus.isDirectory()) { - break; - } - if (fileStatus.isFile()) { - throw new FileAlreadyExistsException( - String.format("Can't make directory for path '%s'" + " since it is a file.", fPart)); - } - } catch (FileNotFoundException fnfe) { - LOG.debug("file {} not fount, but ignore.", path); - } catch (FileConflictException fce) { - throw new ParentNotDirectoryException(fce.getMessage()); - } - fPart = fPart.getParent(); - } while (fPart != null); - String key = pathToKey(owner, path); if (owner.isFsBucket()) { - OBSPosixBucketUtils.fsCreateFolder(owner, key); + try { + OBSPosixBucketUtils.fsCreateFolder(owner, key); + } catch (OBSFileConflictException e1) { + throw new ParentNotDirectoryException(e1.getMessage()); + } } else { + Path fPart = path.getParent(); + do { + try { + fileStatus = getFileStatusWithRetry(owner, fPart); + if (fileStatus.isDirectory()) { + break; + } + if (fileStatus.isFile()) { + throw new FileAlreadyExistsException( + String.format("Can't make directory for path '%s'" + " since it is a file.", fPart)); + } + } catch (FileNotFoundException fnfe) { + LOG.debug("file {} not fount, but ignore.", path); + } catch (OBSFileConflictException fce) { + throw new ParentNotDirectoryException(fce.getMessage()); + } + fPart = fPart.getParent(); + } while (fPart != null); + OBSObjectBucketUtils.createFakeDirectory(owner, key); } return true; - } catch (FileConflictException e) { + } catch (OBSFileConflictException e) { throw new ParentNotDirectoryException(e.getMessage()); } } @@ -870,38 +874,11 @@ static ObjectListing listObjects(final OBSFileSystem owner, final ListObjectsReq static ObjectListing commonListObjects(final OBSFileSystem owner, final ListObjectsRequest request) throws IOException { - int retryTime = 0; - long delayMs; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - owner.getSchemeStatistics().incrementReadOps(1); - return owner.getObsClient().listObjects(request); - } catch (ObsException e) { - LOG.debug("Failed to commonListObjects for request[{}], retry " + "time [{}], due to exception[{}]", - request, retryTime, e); - - IOException ioException = OBSCommonUtils.translateException("listObjects (" + request + ")", - request.getPrefix(), e); - - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - - delayMs = getSleepTimeInMs(retryTime); - retryTime++; - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - LOG.error("Failed to commonListObjects for request[{}], " + "retry time [{}], due to exception[{}]", - request, retryTime, ioException); - throw ioException; - } - } - } - + ObjectListing listObjects = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.listObjects, request.getPrefix(), () -> { + return owner.getObsClient().listObjects(request); + },true); owner.getSchemeStatistics().incrementReadOps(1); - return owner.getObsClient().listObjects(request); + return listObjects; } /** @@ -942,38 +919,11 @@ private static ObjectListing commonContinueListObjects(final OBSFileSystem owner static ObjectListing commonContinueListObjects(final OBSFileSystem owner, final ListObjectsRequest request) throws IOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - owner.getSchemeStatistics().incrementReadOps(1); - return owner.getObsClient().listObjects(request); - } catch (ObsException e) { - LOG.debug("Continue list objects failed for request[{}], retry" + " time[{}], due to exception[{}]", - request, retryTime, e); - - IOException ioException = OBSCommonUtils.translateException("listObjects (" + request + ")", - request.getPrefix(), e); - - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - - delayMs = getSleepTimeInMs(retryTime); - retryTime++; - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - LOG.error("Continue list objects failed for request[{}], " + "retry time[{}], due to exception[{}]", - request, retryTime, ioException); - throw ioException; - } - } - } - + ObjectListing listObjects = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.listObjects, request.getPrefix(), () -> { + return owner.getObsClient().listObjects(request); + },true); owner.getSchemeStatistics().incrementReadOps(1); - return owner.getObsClient().listObjects(request); + return listObjects; } /** @@ -1004,42 +954,14 @@ public static long dateToLong(final Date date) { // Used to check if a folder is empty or not. static boolean isFolderEmpty(final OBSFileSystem owner, final String key) throws IOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - return innerIsFolderEmpty(owner, key); - } catch (ObsException e) { - LOG.debug("Failed to check empty folder for [{}], retry time [{}], " + "exception [{}]", key, retryTime, - e); - - IOException ioException = OBSCommonUtils.translateException("innerIsFolderEmpty", key, e); - - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - - delayMs = getSleepTimeInMs(retryTime); - retryTime++; - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw ioException; - } - } - } - - try { + return OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.isFolderEmpty, key, () -> { return innerIsFolderEmpty(owner, key); - } catch (ObsException e) { - throw OBSCommonUtils.translateException("innerIsFolderEmpty", key, e); - } + },true); } // Used to check if a folder is empty or not by counting the number of // sub objects in list. - private static boolean isFolderEmpty(final String key, final ObjectListing objects) { + private static boolean innerIsFolderEmptyDepth(final String key, final ObjectListing objects) { int count = objects.getObjects().size(); if (count >= 2) { return false; @@ -1068,7 +990,7 @@ static boolean innerIsFolderEmpty(final OBSFileSystem owner, final String key) ObjectListing objects = owner.getObsClient().listObjects(request); if (!objects.getCommonPrefixes().isEmpty() || !objects.getObjects().isEmpty()) { - if (isFolderEmpty(obsKey, objects)) { + if (innerIsFolderEmptyDepth(obsKey, objects)) { LOG.debug("Found empty directory {}", obsKey); return true; } @@ -1122,23 +1044,19 @@ static LocatedFileStatus toLocatedFileStatus(final OBSFileSystem owner, final Fi * @throws IOException any problem */ static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner, final String key, final long recordPosition, - final File tmpFile) throws IOException { + final File tmpFile, final FileStatus fileStatus) throws IOException { Preconditions.checkNotNull(key); Preconditions.checkNotNull(tmpFile); - ObsFSAttribute obsFsAttribute; - try { - GetAttributeRequest getAttributeReq = new GetAttributeRequest(owner.getBucket(), key); - obsFsAttribute = owner.getObsClient().getAttribute(getAttributeReq); - } catch (ObsException e) { - throw translateException("GetAttributeRequest", key, e); - } - long appendPosition = Math.max(recordPosition, obsFsAttribute.getContentLength()); - if (recordPosition != obsFsAttribute.getContentLength()) { + long appendPosition = Math.max(recordPosition, fileStatus.getLen()); + if (recordPosition != fileStatus.getLen()) { LOG.warn("append url[{}] position[{}], file contentLength[{}] not" + " equal to recordPosition[{}].", key, - appendPosition, obsFsAttribute.getContentLength(), recordPosition); + appendPosition, fileStatus.getLen(), recordPosition); } - WriteFileRequest writeFileReq = new WriteFileRequest(owner.getBucket(), key, tmpFile, appendPosition); + WriteFileRequest writeFileReq = new WriteFileRequest(owner.getBucket(), key); + writeFileReq.setInput(new FileInputStream(tmpFile)); + writeFileReq.setPosition(appendPosition); + writeFileReq.setAutoClose(false); writeFileReq.setAcl(owner.getCannedACL()); return writeFileReq; } @@ -1154,21 +1072,15 @@ static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner, final St * @throws IOException any problem */ static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner, final String key, final long recordPosition, - final InputStream inputStream) throws IOException { + final InputStream inputStream, final FileStatus fileStatus) { Preconditions.checkNotNull(key); Preconditions.checkNotNull(inputStream); - ObsFSAttribute obsFsAttribute; - try { - GetAttributeRequest getAttributeReq = new GetAttributeRequest(owner.getBucket(), key); - obsFsAttribute = owner.getObsClient().getAttribute(getAttributeReq); - } catch (ObsException e) { - throw translateException("GetAttributeRequest", key, e); - } + Preconditions.checkNotNull(fileStatus); - long appendPosition = Math.max(recordPosition, obsFsAttribute.getContentLength()); - if (recordPosition != obsFsAttribute.getContentLength()) { + long appendPosition = Math.max(recordPosition, fileStatus.getLen()); + if (recordPosition != fileStatus.getLen()) { LOG.warn("append url[{}] position[{}], file contentLength[{}] not" + " equal to recordPosition[{}].", key, - appendPosition, obsFsAttribute.getContentLength(), recordPosition); + appendPosition, fileStatus.getLen(), recordPosition); } WriteFileRequest writeFileReq = new WriteFileRequest(owner.getBucket(), key, inputStream, appendPosition); writeFileReq.setAcl(owner.getCannedACL()); @@ -1183,20 +1095,16 @@ static WriteFileRequest newAppendFileRequest(final OBSFileSystem owner, final St * @throws IOException on any failure to append file */ static void appendFile(final OBSFileSystem owner, final WriteFileRequest appendFileRequest) throws IOException { - long len = 0; - if (appendFileRequest.getFile() != null) { - len = appendFileRequest.getFile().length(); - } + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.appendFile, appendFileRequest.getObjectKey(), () -> { + if (appendFileRequest.getInput() instanceof FileInputStream) { + ((FileInputStream) appendFileRequest.getInput()).getChannel().position(0); + } + return owner.getObsClient().writeFile(appendFileRequest); + },true); - try { - LOG.debug("Append file, key {} position {} size {}", appendFileRequest.getObjectKey(), - appendFileRequest.getPosition(), len); - owner.getObsClient().writeFile(appendFileRequest); - owner.getSchemeStatistics().incrementWriteOps(1); - owner.getSchemeStatistics().incrementBytesWritten(len); - } catch (ObsException e) { - throw translateException("AppendFile", appendFileRequest.getObjectKey(), e); - } + long len = appendFileRequest.getFile() != null ? appendFileRequest.getFile().length() : 0; + owner.getSchemeStatistics().incrementWriteOps(1); + owner.getSchemeStatistics().incrementBytesWritten(len); } /** @@ -1205,7 +1113,7 @@ static void appendFile(final OBSFileSystem owner, final WriteFileRequest appendF * * @param closeables the objects to close */ - static void closeAll(final java.io.Closeable... closeables) { + public static void closeAll(final java.io.Closeable... closeables) { for (java.io.Closeable c : closeables) { if (c != null) { try { @@ -1251,18 +1159,26 @@ static IOException extractException(final String operation, final String path, f * @param blockSize block size to declare. * @param owner owner of the file * @return a status entry + * @throws IOException If an I/O error occurred */ static OBSFileStatus createFileStatus(final Path keyPath, final ObsObject summary, final long blockSize, - final String owner) { + final OBSFileSystem owner) throws IOException { + OBSFileStatus status = null; if (objectRepresentsDirectory(summary.getObjectKey(), summary.getMetadata().getContentLength())) { long lastModified = summary.getMetadata().getLastModified() == null ? System.currentTimeMillis() : OBSCommonUtils.dateToLong(summary.getMetadata().getLastModified()); - return new OBSFileStatus(keyPath, lastModified, owner); + status = new OBSFileStatus(keyPath, lastModified, owner.getShortUserName()); } else { - return new OBSFileStatus(summary.getMetadata().getContentLength(), - dateToLong(summary.getMetadata().getLastModified()), keyPath, blockSize, owner); + status = new OBSFileStatus(summary.getMetadata().getContentLength(), + dateToLong(summary.getMetadata().getLastModified()), keyPath, blockSize, owner.getShortUserName()); } + + if (owner.supportDisguisePermissionsMode()) { + OBSCommonUtils.setAccessControlAttrForFileStatus(owner, status, + getObjectMetadata(owner, summary.getObjectKey())); + } + return status; } /** @@ -1296,17 +1212,9 @@ static OBSLoginHelper.Login getOBSAccessKeys(final URI name, final Configuration * @throws IOException on any problem */ private static String getPassword(final Configuration conf, final String key, final String val) throws IOException { - return StringUtils.isEmpty(val) ? lookupPassword(conf, key) : val; + return isStringEmpty(val) ? lookupPassword(conf, key) : val; } - /** - * Get a password from a configuration/configured credential providers. - * - * @param conf configuration - * @param key key to look up - * @return a password or the value in {@code defVal} - * @throws IOException on any problem - */ private static String lookupPassword(final Configuration conf, final String key) throws IOException { try { final char[] pass = conf.getPassword(key); @@ -1327,56 +1235,35 @@ static String stringify(final ObsObject summary) { } /** - * Get a integer option not smaller than the minimum allowed value. - * - * @param conf configuration - * @param key key to look up - * @param defVal default value - * @param min minimum value - * @return the value - * @throws IllegalArgumentException if the value is below the minimum + * Get a integer not smaller than the minimum allowed value. */ public static int intOption(final Configuration conf, final String key, final int defVal, final int min) { int v = conf.getInt(key, defVal); Preconditions.checkArgument(v >= min, - String.format("Value of %s: %d is below the minimum value %d", key, v, min)); + String.format(Locale.ROOT, "Value of %s: %d is below the minimum value %d", key, v, min)); LOG.debug("Value of {} is {}", key, v); return v; } /** - * Get a long option not smaller than the minimum allowed value. - * - * @param conf configuration - * @param key key to look up - * @param defVal default value - * @param min minimum value - * @return the value - * @throws IllegalArgumentException if the value is below the minimum + * Get a long not smaller than the minimum allowed value. */ static long longOption(final Configuration conf, final String key, final long defVal, final long min) { long v = conf.getLong(key, defVal); Preconditions.checkArgument(v >= min, - String.format("Value of %s: %d is below the minimum value %d", key, v, min)); + String.format(Locale.ROOT, "Value of %s: %d is below the minimum value %d", key, v, min)); LOG.debug("Value of {} is {}", key, v); return v; } /** - * Get a long option not smaller than the minimum allowed value, supporting + * Get a long not smaller than the minimum allowed value, supporting * memory prefixes K,M,G,T,P. - * - * @param conf configuration - * @param key key to look up - * @param defVal default value - * @param min minimum value - * @return the value - * @throws IllegalArgumentException if the value is below the minimum */ public static long longBytesOption(final Configuration conf, final String key, final long defVal, final long min) { long v = conf.getLongBytes(key, defVal); Preconditions.checkArgument(v >= min, - String.format("Value of %s: %d is below the minimum value %d", key, v, min)); + String.format(Locale.ROOT, "Value of %s: %d is below the minimum value %d", key, v, min)); LOG.debug("Value of {} is {}", key, v); return v; } @@ -1442,26 +1329,20 @@ static int ensureOutputParameterInRange(final String name, final long size) { */ static Configuration propagateBucketOptions(final Configuration source, final String bucket) { - Preconditions.checkArgument(StringUtils.isNotEmpty(bucket), "bucket"); + Preconditions.checkArgument(isStringNotEmpty(bucket), "bucket"); final String bucketPrefix = OBSConstants.FS_OBS_BUCKET_PREFIX + bucket + '.'; LOG.debug("Propagating entries under {}", bucketPrefix); final Configuration dest = new Configuration(source); for (Map.Entry entry : source) { final String key = entry.getKey(); - // get the (unexpanded) value. final String value = entry.getValue(); if (!key.startsWith(bucketPrefix) || bucketPrefix.equals(key)) { continue; } - // there's a bucket prefix, so strip it final String stripped = key.substring(bucketPrefix.length()); if (stripped.startsWith("bucket.") || "impl".equals(stripped)) { - // tell user off LOG.debug("Ignoring bucket option {}", key); } else { - // propagate the value, building a new origin field. - // to track overwrites, the generic key is overwritten even if - // already matches the new one. final String generic = OBSConstants.FS_OBS_PREFIX + stripped; LOG.debug("Updating {}", generic); dest.set(generic, value, key); @@ -1486,7 +1367,7 @@ static void patchSecurityCredentialProviders(final Configuration conf) { if (!customCredentials.isEmpty()) { List all = Lists.newArrayList(customCredentials); all.addAll(hadoopCredentials); - String joined = StringUtils.join(all, ','); + String joined = String.join(",", all); LOG.debug("Setting {} to {}", CREDENTIAL_PROVIDER_PATH, joined); conf.set(CREDENTIAL_PROVIDER_PATH, joined, "patch of " + OBSConstants.OBS_SECURITY_CREDENTIAL_PROVIDER_PATH); @@ -1524,60 +1405,6 @@ static void verifyBufferDirAccessible(final Configuration conf) throws IOExcepti } } - /** - * Verify that the bucket exists. This does not check permissions, not even - * read access. - * - * @param owner the owner OBSFileSystem instance - * @throws FileNotFoundException the bucket is absent - * @throws IOException any other problem talking to OBS - */ - static void verifyBucketExists(final OBSFileSystem owner) throws FileNotFoundException, IOException { - try { - if (!innerVerifyBucketExists(owner)) { - throw new FileNotFoundException("Bucket " + owner.getBucket() + " does not exist"); - } - } catch (IOException e) { - LOG.error("Failed to head bucket for [{}] , exception [{}]", owner.getBucket(), e); - throw e; - } - } - - /** - * Verify that the bucket exists with failure retry. - * - * @param owner the owner OBSFileSystem instance - * @return boolean whether the bucket exists - */ - private static boolean innerVerifyBucketExists(final OBSFileSystem owner) throws IOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - return owner.getObsClient().headBucket(owner.getBucket()); - } catch (ObsException e) { - IOException ioException = OBSCommonUtils.translateException("verifyBucketExists", owner.getBucket(), e); - LOG.debug("Failed to head bucket for [{}], retry time [{}], " + "exception [{}]", owner.getBucket(), - retryTime, ioException); - - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - - delayMs = getSleepTimeInMs(retryTime); - retryTime++; - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw ioException; - } - } - } - - return owner.getObsClient().headBucket(owner.getBucket()); - } - /** * initialize multi-part upload, purge larger than the value of * PURGE_EXISTING_MULTIPART_AGE. @@ -1652,49 +1479,101 @@ static void shutdownAll(final ExecutorService... executors) { * @throws FileNotFoundException when the path does not exist * @throws IOException on other problems */ - static FileStatus innerGetFileStatusWithRetry(final OBSFileSystem owner, final Path f) + static OBSFileStatus getFileStatusWithRetry(final OBSFileSystem owner, final Path f) throws FileNotFoundException, IOException { - long delayMs; - int retryTime = 0; + return OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.getFileStatus, OBSCommonUtils.pathToKey(owner, f), () -> { + return owner.innerGetFileStatus(f); + },true); + } + + public static long getSleepTimeInMs(final int retryCount) { + long sleepTime = OBSCommonUtils.retrySleepBaseTime * (long) ((int) Math.pow( + OBSCommonUtils.VARIABLE_BASE_OF_POWER_FUNCTION, retryCount)); + return sleepTime > OBSCommonUtils.retrySleepMaxTime ? OBSCommonUtils.retrySleepMaxTime : sleepTime; + } + + public static void setMetricsAbnormalInfo(OBSFileSystem fs, OBSOperateAction opName, Exception exception) { long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - return owner.innerGetFileStatus(f); - } catch (OBSIOException e) { - OBSFileSystem.LOG.debug("Failed to get file status for [{}], retry time [{}], " + "exception [{}]", f, - retryTime, e); - - delayMs = getSleepTimeInMs(retryTime); - retryTime++; - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw e; - } + if (fs.getMetricSwitch()) { + BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(opName, exception, BasicMetricsConsumer.MetricKind.abnormal); + fs.getMetricsConsumer().putMetrics(record); + long endTime = System.currentTimeMillis(); + + long costTime = (endTime - startTime) / 1000; + if (costTime >= fs.getInvokeCountThreshold() && !(fs.getMetricsConsumer() instanceof DefaultMetricsConsumer)) { + LOG.warn("putMetrics cosetTime too much:exception: {},opName: {} " + "costTime: {}", record.getExceptionIns().getMessage(), + record.getObsOperateAction(), costTime); + } + } + } + + public static void setMetricsNormalInfo(OBSFileSystem fs, OBSOperateAction opName, long start) { + long startTime = System.currentTimeMillis(); + if (fs.getMetricSwitch()) { + BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( + opName, System.currentTimeMillis() - start, BasicMetricsConsumer.MetricKind.normal); + fs.getMetricsConsumer().putMetrics(record); + long endTime = System.currentTimeMillis(); + + long costTime = (endTime - startTime) / 1000; + if (costTime >= fs.getInvokeCountThreshold() && !(fs.getMetricsConsumer() instanceof DefaultMetricsConsumer)) { + LOG.warn("putMetrics cosetTime too much:opName: {} " + "costTime: {}", record.getObsOperateAction(), costTime); } } + } - return owner.innerGetFileStatus(f); + public static void putQosMetric(OBSFileSystem fs, OBSOperateAction action, IOException e) { + if (e instanceof OBSQosException) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, action, e); + } } - public static long getSleepTimeInMs(final int retryTime) { - long sleepTime = OBSCommonUtils.MIN_TIME_IN_MILLISECONDS_TO_SLEEP * (long) ((int) Math.pow( - OBSCommonUtils.VARIABLE_BASE_OF_POWER_FUNCTION, retryTime)); - sleepTime = sleepTime > OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_SLEEP - ? OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_SLEEP - : sleepTime; - return sleepTime; + public static boolean isStringEmpty(String str) { + return str == null || str.length() == 0; } - public static void setMetricsInfo(OBSFileSystem fs, BasicMetricsConsumer.MetricRecord record) { - long startTime = System.currentTimeMillis(); - fs.getMetricsConsumer().putMetrics(record); - long endTime = System.currentTimeMillis(); + public static boolean isStringNotEmpty(String str) { + return !isStringEmpty(str); + } + + public static boolean stringEqualsIgnoreCase(String str1, String str2) { + return str1 == null ? str2 == null : str1.equalsIgnoreCase(str2); + } + + public static String toHex(byte[] bytes) { + StringBuilder result = new StringBuilder(); + for (byte b : bytes) { + String hex = Integer.toHexString(b & 0xFF); + if (hex.length() == 1) { + result.append('0'); + } + + result.append(hex); + } + return result.toString(); + } + + static Map getObjectMetadata(final OBSFileSystem fs, final String key) throws IOException { + GetObjectMetadataRequest req = new GetObjectMetadataRequest(fs.getBucket(), key); + ObjectMetadata metadata = getOBSInvoker().retryByMaxTime(OBSOperateAction.getObjectMetadata, key, + () -> fs.getObsClient().getObjectMetadata(req), true); + return metadata.getAllMetadata(); + } - long costTime = (endTime - startTime) / 1000; - if (costTime >= fs.getInvokeCountThreshold() && !(fs.getMetricsConsumer() instanceof DefaultMetricsConsumer)) { - LOG.warn("putMetrics cosetTime too much:opType: {},opName: {} " + "costTime: {}", record.getOpType(), - record.getOpName(), costTime); + static void setAccessControlAttrForFileStatus(final OBSFileSystem fs, final OBSFileStatus status, + final Map objMeta) { + FsPermission fsPermission = null; + try { + fsPermission = Optional.ofNullable(objMeta.get("permission")).map(Object::toString) + .map(Short::valueOf).map(FsPermission::new).orElse(null); + } catch (NumberFormatException e) { + LOG.debug("File {} permission is invalid, use default permission.", status.getPath()); } + + status.setAccessControlAttr( + Optional.ofNullable(objMeta.get("user")).map(Object::toString).orElse(fs.getShortUserName()), + Optional.ofNullable(objMeta.get("group")).map(Object::toString).orElse(fs.getShortUserName()), + fsPermission + ); } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java index b06940b..9669a30 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSConstants.java @@ -430,6 +430,13 @@ public final class OBSConstants { */ static final String USER_AGENT_PREFIX = "fs.obs.user.agent.prefix"; + public static final String DELEGATION_TOKEN_ONLY = "fs.obs.delegation.token.only"; + + public static final boolean DEFAULT_DELEGATION_TOKEN_ONLY = false; + + public static final String DELEGATION_TOKEN_PROVIDERS = "fs.obs.delegation.token.providers"; + + public static final String DEFAULT_DELEGATION_TOKEN_PROVIDERS = ""; /** * what read policy to use. Default is {@link #READAHEAD_POLICY_PRIMARY} Value: * {@value} @@ -443,6 +450,17 @@ public final class OBSConstants { @InterfaceStability.Unstable public static final String READAHEAD_POLICY_ADVANCE = "advance"; + @InterfaceStability.Unstable + public static final String READAHEAD_POLICY_MEMARTSCC = "memArtsCC"; + + public static final String MEMARTSCC_LOCALITY_ENABLE = "fs.obs.memartscc.locality.enable"; + + public static final String CACHE_CONFIG_PREFIX = "fs.obs.cache.config.prefix"; + + public static final String DEFAULT_CACHE_CONFIG_PREFIX = "fs.obs.memartscc.config"; + + public static final boolean DEFAULT_MEMARTSCC_LOCALITY_ENABLE = false; + /** * Read ahead buffer size to prevent connection re-establishments. */ @@ -453,10 +471,70 @@ public final class OBSConstants { */ public static final long DEFAULT_READAHEAD_RANGE = 1024 * 1024; + /** + * the prefetch range sent to memartscc + */ + public static final String MEMARTSCC_READAHEAD_RANGE = "fs.obs.memartscc.readahead.range"; + + public static final long DEFAULT_MEMARTSCC_READAHEAD_RANGE = 8 * 1024 * 1024; + + public static final String MEMARTSCC_BUFFER_SIZE = "fs.obs.memartscc.buffer.size"; + + public static final int DEFAULT_MEMARTSCC_BUFFER_SIZE = 8192; + + public static final String MEMARTSCC_DIRECTBUFFER_SIZE = "fs.obs.memartscc.directbuffer.size"; + + public static final int DEFAULT_MEMARTSCC_DIRECTBUFFER_SIZE = 1024 * 1024; + + public static final String MEMARTSCC_AKSK_REFRESH_INTERVAL = "fs.obs.memartscc.aksk.refresh.interval"; + + public static final int DEFAULT_MEMARTSCC_AKSK_REFRESH_INTERVAL = 60; // 60 sec + + public static final String MEMARTSCC_CACHE_IMPL = "fs.obs.memartscc.cache.impl"; + public static final String READAHEAD_MAX_NUM = "fs.obs.readahead.max.number"; public static final int DEFAULT_READAHEAD_MAX_NUM = 4; + public static final String MEMARTSCC_INPUTSTREAM_BUFFER_TYPE = "fs.obs.memartscc.inputstream.buffer.type"; + + public static final String MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_BIND = "bind"; + + public static final String MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_POOL = "pool"; + + public static final String MEMARTSCC_INPUTSTREAM_BUFFER_POOL_MAX_SIZE = "fs.obs.memartscc.inputstream.buffer.pool.maxsize"; + + public static final int MEMARTSCC_INPUTSTREAM_BUFFER_POOL_DEFAULT_MAX_SIZE = 128; + + public static final String MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_TIMEOUT = "fs.obs.memartscc.inputstream.buffer.poll.timeout"; + + public static final int MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_DEFAULT_TIMEOUT = 5000; // ms + + public static final String MEMARTSCC_PYSPARK_OPTIMIZED = "fs.obs.memartscc.pyspark.optimized"; + + public static final boolean DEFAULT_MEMARTSCC_PYSPARK_OPTIMIZED = true; + + public static final String MEMARTSCC_TRAFFIC_REPORT_ENABLE = "fs.obs.memartscc.inputstream.statistics.report.enable"; + + public static final boolean DEFAULT_MEMARTSCC_TRAFFIC_REPORT_ENABLE = false; + + /** + * The interval of reporting traffic statistics to CC SDK, unit: seconds + */ + public static final String MEMARTSCC_TRAFFIC_REPORT_INTERVAL = "fs.obs.memartscc.inputstream.statistics.report.interval"; + + public static final long MEMARTSCC_TRAFFIC_REPORT_DEFAULT_INTERVAL = 30; + + /* + * memartscc duplications num, set to 1 for now + * */ + public static final int MAX_DUPLICATION_NUM = 1; + + /* + * memartscc errorcode getShardInfo success + * */ + public static final int GET_SHARD_INFO_SUCCESS = 0; + /** * Flag indicating if * {@link OBSInputStream#read(long, byte[], int, int)} @@ -466,6 +544,21 @@ public final class OBSConstants { */ public static final String READAHEAD_TRANSFORM_ENABLE = "fs.obs.read.transform.enable"; + /** + * obs file system permission mode settings, only take effect on posix file system. + */ + public static final String PERMISSIONS_MODE = "fs.obs.permissions.mode"; + + /** + * default permission mode, doesn't support permissions. + */ + public static final String DEFAULT_PERMISSIONS_MODE = "none"; + + /** + * disguise permission mode, support file owner, group, permission attribute. + */ + public static final String PERMISSIONS_MODE_DISGUISE = "disguise"; + /** * OBS client factory implementation class. */ @@ -654,29 +747,47 @@ public final class OBSConstants { static final int DEFAULT_LIST_PARALLEL_FACTOR = 30; /** - * Switch for the fast delete. + * Multi list contentsummary parallel factor */ - static final String TRASH_ENABLE = "fs.obs.trash.enable"; + static final String MULTILISTCS_PARALLEL_FACTOR = "fs.obs.multilistcs.parallel.factor"; /** - * Enable obs content summary or not. + * Default value of {@link #MULTILISTCS_PARALLEL_FACTOR}. */ - static final String OBS_CONTENT_SUMMARY_ENABLE = "fs.obs.content.summary.enable"; + static final int DEFAULT_MULTILISTCS_PARALLEL_FACTOR = 30; /** - * Enable obs client dfs list or not. + * Switch for the fast delete. */ - static final String OBS_CLIENT_DFS_LIST_ENABLE = "fs.obs.client.dfs.list.enable"; + static final String FAST_DELETE_ENABLE = "fs.obs.trash.enable"; /** * Default trash : false. */ - static final boolean DEFAULT_TRASH = false; + static final boolean DEFAULT_FAST_DELETE_ENABLE = false; /** * The fast delete recycle directory. */ - static final String TRASH_DIR = "fs.obs.trash.dir"; + static final String FAST_DELETE_DIR = "fs.obs.trash.dir"; + + /** + * Enable obs content summary or not. + */ + static final String OBS_CONTENT_SUMMARY_ENABLE = "fs.obs.content.summary.enable"; + + static final String OBS_CONTENT_SUMMARY_VERSION = "fs.obs.content.summary.version"; + + static final String OBS_CONTENT_SUMMARY_VERSION_V1 = "1"; + + static final String OBS_CONTENT_SUMMARY_VERSION_V2 = "2"; + + static final int OBS_CONTENT_SUMMARY_FALLBACK_THRESHOLD = 1000; + + /** + * Enable obs client dfs list or not. + */ + static final String OBS_CLIENT_DFS_LIST_ENABLE = "fs.obs.client.dfs.list.enable"; /** * Encryption type is sse-kms or sse-c. @@ -744,12 +855,38 @@ public final class OBSConstants { */ static final boolean DEFAULT_GET_CANONICAL_SERVICE_NAME_ENABLE = false; - static final String MAX_TIME_IN_MILLISECOND_TO_RETRY = "fs.obs.retry.maxtime"; + static final String RETRY_MAXTIME = "fs.obs.retry.maxtime"; - /** - * Default value of {@link #MAX_TIME_IN_MILLISECOND_TO_RETRY} - */ - static final long DEFAULT_TIME_IN_MILLISECOND_TO_RETRY = 180000; + static final long DEFAULT_RETRY_MAXTIME = 180000; + + static final String RETRY_SLEEP_BASETIME = "fs.obs.retry.sleep.basetime"; + + static final long DEFAULT_RETRY_SLEEP_BASETIME = 50; + + static final String RETRY_SLEEP_MAXTIME = "fs.obs.retry.sleep.maxtime"; + + static final long DEFAULT_RETRY_SLEEP_MAXTIME = 30000; + + + static final String RETRY_QOS_MAXTIME = "fs.obs.retry.qos.maxtime"; + + static final long DEFAULT_RETRY_QOS_MAXTIME = 180000; + + static final String RETRY_QOS_SLEEP_BASETIME = "fs.obs.retry.qos.sleep.basetime"; + + static final long DEFAULT_RETRY_QOS_SLEEP_BASETIME = 1000; + + static final String RETRY_QOS_SLEEP_MAXTIME = "fs.obs.retry.qos.sleep.maxtime"; + + static final long DEFAULT_RETRY_QOS_SLEEP_MAXTIME = 30000; + + public static final String RETRY_LIMIT = "fs.obs.retry.limit"; + + public static final int DEFAULT_RETRY_LIMIT = 7; + + public static final String RETRY_QOS_LIMIT = "fs.obs.retry.qos.limit"; + + public static final int DEFAULT_RETRY_QOS_LIMIT = 7; /** * File visibility after create interface switch. @@ -764,10 +901,10 @@ public final class OBSConstants { public static final String AUTHORIZER_PROVIDER = "fs.obs.authorize.provider"; - public static final String AUTHORIZE_FAIL_FALLBACK= "fs.obs.authorize.fail.fallback"; + public static final String AUTHORIZE_FAIL_FALLBACK = "fs.obs.authorize.fail.fallback"; public static final boolean DEFAULT_AUTHORIZE_FAIL_FALLBACK = false; - public static final String AUTHORIZE_EXCEPTION_FALLBACK= "fs.obs.authorize.exception.fallback"; + public static final String AUTHORIZE_EXCEPTION_FALLBACK = "fs.obs.authorize.exception.fallback"; public static final boolean DEFAULT_AUTHORIZE_EXCEPTION_FALLBACK = true; /** @@ -798,6 +935,48 @@ public final class OBSConstants { static final String OUTPUT_STREAM_HFLUSH_POLICY_EMPTY = "empty"; // downgrade hflush/hsync to empty func, which means calling hflush/hsync will do nothing + static final String OUTPUT_STREAM_ATTACH_MD5 = "fs.obs.outputstream.attach.md5"; + + static final Boolean DEFAULT_OUTPUT_STREAM_ATTACH_MD5 = false; + + /** + * Use which type to validate consistency of uploaded block data. Default value is {@link #FAST_UPLOAD_CHECKSUM_TYPE_NONE}. + * Normally replace {@link #OUTPUT_STREAM_ATTACH_MD5}. + * Recommend {@link #FAST_UPLOAD_CHECKSUM_TYPE_SHA256} for more secure. + */ + static final String FAST_UPLOAD_CHECKSUM_TYPE = "fs.obs.fast.upload.checksum.type"; + + static final String FAST_UPLOAD_CHECKSUM_TYPE_NONE = "none"; + + static final String FAST_UPLOAD_CHECKSUM_TYPE_MD5 = "md5"; + + static final String FAST_UPLOAD_CHECKSUM_TYPE_SHA256 = "sha256"; + + static final String OUTPUT_STREAM_DISK_FORCE_FLUSH = "fs.obs.outputstream.disk.force.flush"; + + static final Boolean DEFAULT_OUTPUT_STREAM_DISK_FORCE_FLUSH = true; + + static final String FAST_DELETE_VERSION = "fs.obs.fast.delete.version"; + + static final String FAST_DELETE_VERSION_V1 = "1"; + + static final String FAST_DELETE_VERSION_V2 = "2"; + + static final String FAST_DELETE_VERSION_V2_CHECKPOINT_FORMAT = "yyyyMMddHH"; + + /** + * Determines the HDFS trash behavior. Default value is {@link #HDFS_TRASH_VERSION_V1}. + */ + static final String HDFS_TRASH_VERSION = "fs.obs.hdfs.trash.version"; + + static final String HDFS_TRASH_VERSION_V1 = "1"; + + static final String HDFS_TRASH_VERSION_V2 = "2"; + + static final String HDFS_TRASH_PREFIX = "fs.obs.hdfs.trash.prefix"; + + static final String DEFAULT_HDFS_TRASH_PREFIX = "/user/.Trash"; + private OBSConstants() { } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java index bb2b061..0ddb1b5 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSDataBlocks.java @@ -23,6 +23,8 @@ import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; import org.apache.hadoop.util.DirectBufferPool; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -38,6 +40,10 @@ import java.io.IOException; import java.io.InputStream; import java.nio.ByteBuffer; +import java.security.MessageDigest; +import java.security.NoSuchAlgorithmException; +import java.util.Base64; +import java.util.Locale; import java.util.concurrent.atomic.AtomicBoolean; import java.util.concurrent.atomic.AtomicInteger; @@ -55,20 +61,14 @@ final class OBSDataBlocks { private OBSDataBlocks() { } - /** - * Validate args to a write command. These are the same validation checks - * expected for any implementation of {@code OutputStream.write()}. - * - * @param b byte array containing data - * @param off offset in array where to start - * @param len number of bytes to be written - * @throws NullPointerException for a null buffer - * @throws IndexOutOfBoundsException if indices are out of range - */ static void validateWriteArgs(final byte[] b, final int off, final int len) { Preconditions.checkNotNull(b); - if (off < 0 || off > b.length || len < 0 || off + len > b.length || off + len < 0) { - throw new IndexOutOfBoundsException("write (b[" + b.length + "], " + off + ", " + len + ')'); + boolean offsetInvalid = off < 0 || off > b.length; + boolean lenInvalid = len < 0; + boolean endposInvalid = off + len > b.length || off + len < 0; + boolean invalid = offsetInvalid || lenInvalid || endposInvalid; + if (invalid) { + throw new IndexOutOfBoundsException(String.format(Locale.ROOT, "write (b[%d], %d, %d)", b.length, off, len)); } } @@ -93,6 +93,39 @@ static BlockFactory createFactory(final OBSFileSystem owner, final String name) } } + enum ChecksumType { + NONE("NONE") { + @Override + public String encode(byte[] bytes) { + throw new UnsupportedOperationException("Not supported when checksum type is none"); + } + }, + MD5("MD5") { + @Override + public String encode(byte[] bytes) { + return Base64.getEncoder().encodeToString(bytes); + } + }, + SHA256("SHA-256") { + @Override + public String encode(byte[] bytes) { + return OBSCommonUtils.toHex(bytes); + } + }; + + private String algorithm; + + ChecksumType(String algorithm) { + this.algorithm = algorithm; + } + + public String getAlgorithm() { + return algorithm; + } + + public abstract String encode(byte[] bytes); + } + /** * Base class for block factories. */ @@ -116,19 +149,33 @@ protected BlockFactory(final OBSFileSystem obsFileSystem) { */ abstract DataBlock create(long index, int limit) throws IOException; - /** - * Owner. - * - * @return obsFileSystem instance - */ protected OBSFileSystem getOwner() { return owner; } + + protected boolean calcMd5() { + return this.getOwner() + .getConf() + .getBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, OBSConstants.DEFAULT_OUTPUT_STREAM_ATTACH_MD5); + } + + protected ChecksumType determineChecksumType() throws IOException { + String type = this.getOwner() + .getConf() + .get(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_NONE); + if (OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_SHA256.equals(type)) { + return ChecksumType.SHA256; + } else if (OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_MD5.equals(type) || calcMd5()) { + return ChecksumType.MD5; + } else if (OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_NONE.equals(type)) { + return ChecksumType.NONE; + } else { + throw new IOException(String.format("Unsupported fast upload checksum type '%s'", type)); + } + } + } - /** - * This represents a block being uploaded. - */ abstract static class DataBlock implements Closeable { /** @@ -136,13 +183,38 @@ abstract static class DataBlock implements Closeable { */ private final long index; + private ChecksumType checksumType; + + private MessageDigest digest; + + private String checksum; + /** * Dest state can be : writing/upload/closed. */ private volatile DestState state = DestState.Writing; - protected DataBlock(final long dataIndex) { + protected DataBlock(final long dataIndex, final ChecksumType checksumType) { this.index = dataIndex; + if (checksumType == null) { + this.checksumType = ChecksumType.NONE; + } else { + this.checksumType = checksumType; + } + + if (checksumType != ChecksumType.NONE) { + LOG.debug("init data block digest state to calculate checksum. checksumType: {}", checksumType); + initDigestState(); + } + } + + private void initDigestState() { + try { + digest = MessageDigest.getInstance(checksumType.getAlgorithm()); + } catch (NoSuchAlgorithmException e) { + LOG.warn("load digest algorithm failed", e); + this.checksumType = ChecksumType.NONE; + } } /** @@ -160,12 +232,6 @@ protected final synchronized void enterState(final DestState current, final Dest state = next; } - /** - * Verify that the block is in the declared state. - * - * @param expected expected state. - * @throws IllegalStateException if the DataBlock is in the wrong state - */ protected final void verifyState(final DestState expected) throws IllegalStateException { if (expected != null && state != expected) { throw new IllegalStateException( @@ -173,11 +239,6 @@ protected final void verifyState(final DestState expected) throws IllegalStateEx } } - /** - * Current state. - * - * @return the current state. - */ protected final DestState getState() { return state; } @@ -186,49 +247,16 @@ protected long getIndex() { return index; } - /** - * Return the current data size. - * - * @return the size of the data - */ abstract int dataSize(); - /** - * Predicate to verify that the block has the capacity to write the - * given set of bytes. - * - * @param bytes number of bytes desired to be written. - * @return true if there is enough space. - */ abstract boolean hasCapacity(long bytes); - /** - * Predicate to check if there is data in the block. - * - * @return true if there is - */ boolean hasData() { return dataSize() > 0; } - /** - * The remaining capacity in the block before it is full. - * - * @return the number of bytes remaining. - */ abstract int remainingCapacity(); - /** - * Write a series of bytes from the buffer, from the offset. Returns the - * number of bytes written. Only valid in the state {@code Writing}. - * Base class verifies the state but does no writing. - * - * @param buffer buffer - * @param offset offset - * @param length length of write - * @return number of bytes written - * @throws IOException trouble - */ int write(final byte[] buffer, final int offset, final int length) throws IOException { verifyState(DestState.Writing); Preconditions.checkArgument(buffer != null, "Null buffer"); @@ -236,46 +264,43 @@ int write(final byte[] buffer, final int offset, final int length) throws IOExce Preconditions.checkArgument(offset >= 0, "offset is negative"); Preconditions.checkArgument(!(buffer.length - offset < length), "buffer shorter than amount of data to write"); + if (checksumType != ChecksumType.NONE) { + digest.update(buffer, offset, length); + } return 0; } - /** - * Flush the output. Only valid in the state {@code Writing}. In the - * base class, this is a no-op - * - * @throws IOException any IO problem. - */ void flush() throws IOException { verifyState(DestState.Writing); } - /** - * Switch to the upload state and return a stream for uploading. Base - * class calls {@link #enterState(DestState, DestState)} to manage the - * state machine. - * - * @return the stream - * @throws IOException trouble - */ Object startUpload() throws IOException { LOG.debug("Start datablock[{}] upload", index); enterState(DestState.Writing, DestState.Upload); + if (checksumType != ChecksumType.NONE) { + finalDigest(); + this.digest = null; + } return null; } - /** - * Enter the closed state. - * - * @return true if the class was in any other state, implying that the - * subclass should do its close operations - */ - protected synchronized boolean enterClosedState() { - if (!state.equals(DestState.Closed)) { - enterState(null, DestState.Closed); - return true; - } else { - return false; + private void finalDigest() { + if (digest == null) { + LOG.warn("digest is null"); } + if (!getState().equals(DestState.Upload)) { + throw new IllegalStateException("finalDigest() should in Upload state"); + } + byte[] bytes = digest.digest(); + checksum = checksumType.encode(bytes); + } + + public ChecksumType getChecksumType() { + return checksumType; + } + + public String getChecksum() { + return checksum; } @Override @@ -286,62 +311,43 @@ public void close() throws IOException { } } - /** - * Inner close logic for subclasses to implement. - * - * @throws IOException on any failure to close - */ protected abstract void innerClose() throws IOException; - /** - * Destination state definition for a data block. - */ + protected synchronized boolean enterClosedState() { + if (!state.equals(DestState.Closed)) { + enterState(null, DestState.Closed); + return true; + } else { + return false; + } + } + enum DestState { - /** - * destination state : writing. - */ Writing, - /** - * destination state : upload. - */ Upload, - /** - * destination state : closed. - */ Closed } } - /** - * Use byte arrays on the heap for storage. - */ static class ByteArrayBlockFactory extends BlockFactory { ByteArrayBlockFactory(final OBSFileSystem owner) { super(owner); } @Override - DataBlock create(final long index, final int limit) { + DataBlock create(final long index, final int limit) throws IOException { int firstBlockSize = super.owner.getConf() .getInt(OBSConstants.FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE, OBSConstants.FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT); - return new ByteArrayBlock(0, limit, firstBlockSize); + return new ByteArrayBlock(0, limit, firstBlockSize, determineChecksumType()); } } - /** - * OBS specific byte array output stream. - */ static class OBSByteArrayOutputStream extends ByteArrayOutputStream { OBSByteArrayOutputStream(final int size) { super(size); } - /** - * InputStream backed by the internal byte array. - * - * @return input stream - */ ByteArrayInputStream getInputStream() { ByteArrayInputStream bin = new ByteArrayInputStream(this.buf, 0, count); this.reset(); @@ -350,77 +356,30 @@ ByteArrayInputStream getInputStream() { } } - /** - * Stream to memory via a {@code ByteArrayOutputStream}. - * - *

This was taken from {@code OBSBlockOutputStream} and has the same - * problem which surfaced there: it can consume a lot of heap space - * proportional to the mismatch between writes to the stream and the - * JVM-wide upload bandwidth to the OBS endpoint. The memory consumption can - * be limited by tuning the filesystem settings to restrict the number of - * queued/active uploads. - */ static class ByteArrayBlock extends DataBlock { - /** - * Memory limit. - */ + private final int limit; - /** - * Output stream. - */ private OBSByteArrayOutputStream buffer; - /** - * Cache data size so that it is consistent after the buffer is reset. - */ private Integer dataSize; - /** - * Block first size. - */ private int firstBlockSize; - /** - * Input stream. - */ private ByteArrayInputStream inputStream = null; - ByteArrayBlock(final long index, final int limitBlockSize, final int blockSize) { - super(index); + ByteArrayBlock(final long index, final int limitBlockSize, final int blockSize, final ChecksumType checksumType) { + super(index, checksumType); this.limit = limitBlockSize; this.buffer = new OBSByteArrayOutputStream(blockSize); this.firstBlockSize = blockSize; } - /** - * Returns the block first block size. - * - * @return the block first block size - */ - @VisibleForTesting - public int firstBlockSize() { - return this.firstBlockSize; - } - - /** - * Get the amount of data; if there is no buffer then the size is 0. - * - * @return the amount of data available to upload. - */ @Override int dataSize() { return dataSize != null ? dataSize : buffer.size(); } - @Override - InputStream startUpload() throws IOException { - super.startUpload(); - dataSize = buffer.size(); - inputStream = buffer.getInputStream(); - return inputStream; - } - @Override boolean hasCapacity(final long bytes) { return dataSize() + bytes <= limit; @@ -431,10 +390,15 @@ int remainingCapacity() { return limit - dataSize(); } + @VisibleForTesting + public int firstBlockSize() { + return this.firstBlockSize; + } + @Override int write(final byte[] b, final int offset, final int len) throws IOException { - super.write(b, offset, len); int written = Math.min(remainingCapacity(), len); + super.write(b, offset, written); buffer.write(b, offset, written); return written; } @@ -452,6 +416,14 @@ protected void innerClose() throws IOException { } } + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = buffer.size(); + inputStream = buffer.getInputStream(); + return inputStream; + } + @Override public String toString() { return "ByteArrayBlock{" + "index=" + getIndex() + ", state=" + getState() + ", limit=" + limit @@ -459,20 +431,10 @@ public String toString() { } } - /** - * Stream via Direct ByteBuffers; these are allocated off heap via {@link - * DirectBufferPool}. - */ static class ByteBufferBlockFactory extends BlockFactory { - /** - * The directory buffer pool. - */ private static final DirectBufferPool BUFFER_POOL = new DirectBufferPool(); - /** - * Count of outstanding buffers. - */ private static final AtomicInteger BUFFERS_OUTSTANDING = new AtomicInteger(0); ByteBufferBlockFactory(final OBSFileSystem owner) { @@ -480,8 +442,8 @@ static class ByteBufferBlockFactory extends BlockFactory { } @Override - ByteBufferBlock create(final long index, final int limit) { - return new ByteBufferBlock(index, limit); + ByteBufferBlock create(final long index, final int limit) throws IOException { + return new ByteBufferBlock(index, limit, determineChecksumType()); } public static ByteBuffer requestBuffer(final int limit) { @@ -496,104 +458,41 @@ public static void releaseBuffer(final ByteBuffer buffer) { BUFFERS_OUTSTANDING.decrementAndGet(); } - /** - * Get count of outstanding buffers. - * - * @return the current buffer count - */ public int getOutstandingBufferCount() { return BUFFERS_OUTSTANDING.get(); } @Override public String toString() { - return "ByteBufferBlockFactory{" + "buffersOutstanding=" + BUFFERS_OUTSTANDING + '}'; + return String.format(Locale.ROOT, "ByteBufferBlockFactory{ buffersOutstanding = %d }", BUFFERS_OUTSTANDING.get()); } } - /** - * A DataBlock which requests a buffer from pool on creation; returns it - * when it is closed. - */ static class ByteBufferBlock extends DataBlock { - /** - * Set the buffer size. - */ + private final int bufferSize; - /** - * Create block buffer. - */ private ByteBuffer blockBuffer; - /** - * Cache data size so that it is consistent after the buffer is reset. - */ private Integer dataSize; - /** - * Create input stream. - */ private ByteBufferInputStream inputStream; - /** - * Instantiate. This will request a ByteBuffer of the desired size. - * - * @param index block index - * @param initBufferSize buffer size - */ - ByteBufferBlock(final long index, final int initBufferSize) { - super(index); + ByteBufferBlock(final long index, final int initBufferSize, final ChecksumType checksumType) { + super(index, checksumType); this.bufferSize = initBufferSize; blockBuffer = ByteBufferBlockFactory.requestBuffer(initBufferSize); } - /** - * Get the amount of data; if there is no buffer then the size is 0. - * - * @return the amount of data available to upload. - */ - @Override - int dataSize() { - return dataSize != null ? dataSize : bufferCapacityUsed(); - } - - @Override - InputStream startUpload() throws IOException { - super.startUpload(); - dataSize = bufferCapacityUsed(); - // set the buffer up from reading from the beginning - blockBuffer.limit(blockBuffer.position()); - blockBuffer.position(0); - inputStream = new ByteBufferInputStream(dataSize, blockBuffer); - return inputStream; - } - - @Override - public boolean hasCapacity(final long bytes) { - return bytes <= remainingCapacity(); - } - - @Override - public int remainingCapacity() { - return blockBuffer != null ? blockBuffer.remaining() : 0; - } - - private int bufferCapacityUsed() { + private int capacityUsed() { return blockBuffer.capacity() - blockBuffer.remaining(); } @Override - int write(final byte[] b, final int offset, final int len) throws IOException { - super.write(b, offset, len); - int written = Math.min(remainingCapacity(), len); - blockBuffer.put(b, offset, written); - return written; + int dataSize() { + return dataSize != null ? dataSize : capacityUsed(); } - /** - * Closing the block will release the buffer. - */ @Override protected void innerClose() { if (blockBuffer != null) { @@ -606,27 +505,34 @@ protected void innerClose() { } } + @Override + public int remainingCapacity() { + return blockBuffer != null ? blockBuffer.remaining() : 0; + } + + @Override + public boolean hasCapacity(final long bytes) { + return bytes <= remainingCapacity(); + } + @Override public String toString() { return "ByteBufferBlock{" + "index=" + getIndex() + ", state=" + getState() + ", dataSize=" + dataSize() + ", limit=" + bufferSize + ", remainingCapacity=" + remainingCapacity() + '}'; } - /** - * Provide an input stream from a byte buffer; supporting {@link - * #mark(int)}, which is required to enable replay of failed PUT - * attempts. - */ + @Override + int write(final byte[] b, final int offset, final int len) throws IOException { + int written = Math.min(remainingCapacity(), len); + super.write(b, offset, written); + blockBuffer.put(b, offset, written); + return written; + } + class ByteBufferInputStream extends InputStream { - /** - * Set the input stream size. - */ private final int size; - /** - * Set the byte buffer. - */ private ByteBuffer byteBuffer; ByteBufferInputStream(final int streamSize, final ByteBuffer streamByteBuffer) { @@ -635,48 +541,38 @@ class ByteBufferInputStream extends InputStream { this.byteBuffer = streamByteBuffer; } - /** - * After the stream is closed, set the local reference to the byte - * buffer to null; this guarantees that future attempts to use - * stream methods will fail. - */ - @Override - public synchronized void close() { - LOG.debug("ByteBufferInputStream.close() for {}", ByteBufferBlock.super.toString()); - byteBuffer = null; - } - - /** - * Verify that the stream is open. - * - * @throws IOException if the stream is closed - */ private void verifyOpen() throws IOException { if (byteBuffer == null) { throw new IOException(FSExceptionMessages.STREAM_IS_CLOSED); } } - public synchronized int read() { - if (available() > 0) { - return byteBuffer.get() & OBSCommonUtils.BYTE_TO_INT_MASK; - } else { - return -1; - } - } - @Override public synchronized long skip(final long offset) throws IOException { verifyOpen(); - long newPos = position() + offset; - if (newPos < 0) { + long pos = position() + offset; + if (pos < 0) { throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK); } - if (newPos > size) { + if (pos > size) { throw new EOFException(FSExceptionMessages.CANNOT_SEEK_PAST_EOF); } - byteBuffer.position((int) newPos); - return newPos; + byteBuffer.position((int) pos); + return pos; + } + + @Override + public synchronized void close() { + LOG.debug("ByteBufferInputStream.close() for {}", ByteBufferBlock.super.toString()); + byteBuffer = null; + } + + public synchronized int read() { + if (available() > 0) { + return byteBuffer.get() & OBSCommonUtils.BYTE_TO_INT_MASK; + } else { + return -1; + } } @Override @@ -685,53 +581,20 @@ public synchronized int available() { return byteBuffer.remaining(); } - /** - * Get the current buffer position. - * - * @return the buffer position - */ public synchronized int position() { return byteBuffer.position(); } - /** - * Check if there is data left. - * - * @return true if there is data remaining in the buffer. - */ public synchronized boolean hasRemaining() { return byteBuffer.hasRemaining(); } - @Override - public synchronized void mark(final int readlimit) { - LOG.debug("mark at {}", position()); - byteBuffer.mark(); - } - @Override public synchronized void reset() { LOG.debug("reset"); byteBuffer.reset(); } - @Override - public boolean markSupported() { - return true; - } - - /** - * Read in data. - * - * @param b destination buffer - * @param offset offset within the buffer - * @param length length of bytes to read - * @return read size - * @throws EOFException if the position is negative - * @throws IndexOutOfBoundsException if there isn't space for the - * amount of data requested. - * @throws IllegalArgumentException other arguments are invalid. - */ public synchronized int read(final byte[] b, final int offset, final int length) throws IOException { Preconditions.checkArgument(length >= 0, "length is negative"); Preconditions.checkArgument(b != null, "Null buffer"); @@ -750,107 +613,99 @@ public synchronized int read(final byte[] b, final int offset, final int length) return toRead; } + @Override + public synchronized void mark(final int readLimit) { + LOG.debug("mark at {}", position()); + byteBuffer.mark(); + } + @Override public String toString() { final StringBuilder sb = new StringBuilder("ByteBufferInputStream{"); sb.append("size=").append(size); - ByteBuffer buf = this.byteBuffer; - if (buf != null) { - sb.append(", available=").append(buf.remaining()); + if (this.byteBuffer != null) { + sb.append(", available=").append(this.byteBuffer.remaining()); } - sb.append(", ").append(ByteBufferBlock.super.toString()); - sb.append('}'); + sb.append(", ").append(ByteBufferBlock.super.toString()).append('}'); return sb.toString(); } + + @Override + public boolean markSupported() { + return true; + } + } + + @Override + InputStream startUpload() throws IOException { + super.startUpload(); + dataSize = capacityUsed(); + blockBuffer.limit(blockBuffer.position()); + blockBuffer.position(0); + inputStream = new ByteBufferInputStream(dataSize, blockBuffer); + return inputStream; } } - /** - * Buffer blocks to disk. - */ static class DiskBlockFactory extends BlockFactory { - /** - * Allocator the local directory. - */ - private static OBSLocalDirAllocator directoryAllocator; + + private static volatile LocalDirAllocator directoryAllocator; DiskBlockFactory(final OBSFileSystem owner) { super(owner); } - /** - * Create a temp file and a {@link DiskBlock} instance to manage it. - * - * @param index block index - * @param limit limit of the block. - * @return the new block - * @throws IOException IO problems - */ + protected boolean diskForce() { + return this.getOwner() + .getConf() + .getBoolean(OBSConstants.OUTPUT_STREAM_DISK_FORCE_FLUSH , OBSConstants.DEFAULT_OUTPUT_STREAM_DISK_FORCE_FLUSH); + } + @Override DataBlock create(final long index, final int limit) throws IOException { File destFile = createTmpFileForWrite(String.format("obs-block-%04d-", index), limit, getOwner().getConf()); - return new DiskBlock(destFile, limit, index); + return new DiskBlock(destFile, limit, index, determineChecksumType(), diskForce()); } - /** - * Demand create the directory allocator, then create a temporary file. - * {@link OBSLocalDirAllocator#createTmpFileForWrite(String, long, - * Configuration)}. - * - * @param pathStr prefix for the temporary file - * @param size the size of the file that is going to be written - * @param conf the Configuration object - * @return a unique temporary file - * @throws IOException IO problems - */ - static File createTmpFileForWrite(final String pathStr, final long size, final Configuration conf) + static synchronized File createTmpFileForWrite(final String pathStr, final long size, final Configuration conf) throws IOException { if (directoryAllocator == null) { String bufferDir = conf.get(OBSConstants.BUFFER_DIR) != null - ? OBSConstants.BUFFER_DIR - : "hadoop.tmp.dir"; - directoryAllocator = new OBSLocalDirAllocator(bufferDir); + ? OBSConstants.BUFFER_DIR + : "hadoop.tmp.dir"; + directoryAllocator = new LocalDirAllocator(bufferDir); } - return directoryAllocator.createTmpFileForWrite(pathStr, size, conf); + Path path = directoryAllocator.getLocalPathForWrite(pathStr, size, conf); + File dir = new File(path.getParent().toUri().getPath()); + String prefix = path.getName(); + return File.createTempFile(prefix, null, dir); } } - /** - * Stream to a file. This will stop at the limit; the caller is expected to - * create a new block. - */ static class DiskBlock extends DataBlock { - /** - * Create buffer file. - */ private final File bufferFile; - /** - * Buffer size limit. - */ private final int limit; - /** - * Verify block has closed or not. - */ private final AtomicBoolean closed = new AtomicBoolean(false); - /** - * Written bytes count. - */ private int bytesWritten; - /** - * Out put stream buffer. - */ - private BufferedOutputStream out; + private boolean diskForce; + + private BufferedOutputStream outputStream; - DiskBlock(final File destBufferFile, final int limitSize, final long index) throws FileNotFoundException { - super(index); + private FileOutputStream fileoutputstream; + + DiskBlock(final File destBufferFile, final int limitSize, final long index, final ChecksumType checksumType, + final boolean diskForce) throws FileNotFoundException { + super(index, checksumType); this.limit = limitSize; + this.diskForce = diskForce; this.bufferFile = destBufferFile; - out = new BufferedOutputStream(new FileOutputStream(destBufferFile)); + this.fileoutputstream = new FileOutputStream(destBufferFile); + outputStream = new BufferedOutputStream(fileoutputstream); } @Override @@ -859,8 +714,12 @@ int dataSize() { } @Override - boolean hasCapacity(final long bytes) { - return dataSize() + bytes <= limit; + int write(final byte[] b, final int offset, final int len) throws IOException { + int writeLen = Math.min(remainingCapacity(), len); + super.write(b, offset, writeLen); + outputStream.write(b, offset, writeLen); + bytesWritten += writeLen; + return writeLen; } @Override @@ -868,67 +727,56 @@ int remainingCapacity() { return limit - bytesWritten; } - @Override - int write(final byte[] b, final int offset, final int len) throws IOException { - super.write(b, offset, len); - int written = Math.min(remainingCapacity(), len); - out.write(b, offset, written); - bytesWritten += written; - return written; - } - - @Override - File startUpload() throws IOException { - super.startUpload(); - try { - out.flush(); - } finally { - out.close(); - out = null; + void closeBlock() { + LOG.debug("block[{}]: closeBlock()", getIndex()); + if (!closed.getAndSet(true)) { + if (!bufferFile.delete() && bufferFile.exists()) { + LOG.warn("delete({}) returned false", bufferFile.getAbsoluteFile()); + } + } else { + LOG.debug("block[{}]: skipping re-entrant closeBlock()", getIndex()); } - return bufferFile; } - /** - * The close operation will delete the destination file if it still - * exists. - */ @Override protected void innerClose() { final DestState state = getState(); LOG.debug("Closing {}", this); switch (state) { - case Writing: - if (bufferFile.exists()) { - // file was not uploaded - LOG.debug("Block[{}]: Deleting buffer file as upload " + "did not start", getIndex()); - closeBlock(); - } + case Closed: + closeBlock(); break; case Upload: LOG.debug("Block[{}]: Buffer file {} exists —close upload stream", getIndex(), bufferFile); break; - case Closed: - closeBlock(); + case Writing: + if (bufferFile.exists()) { + LOG.debug("Block[{}]: Deleting buffer file as upload did not start", getIndex()); + closeBlock(); + } break; default: - // this state can never be reached, but checkstyle - // complains, so it is here. + } } - /** - * Flush operation will flush to disk. - * - * @throws IOException IOE raised on FileOutputStream - */ @Override - void flush() throws IOException { - super.flush(); - out.flush(); + File startUpload() throws IOException { + super.startUpload(); + try { + outputStream.flush(); + if (diskForce) { + fileoutputstream.getChannel().force(true); + } + + } finally { + outputStream.close(); + outputStream = null; + } + return bufferFile; } @Override @@ -937,19 +785,15 @@ public String toString() { + ", dataSize=" + dataSize() + ", limit=" + limit + '}'; } - /** - * Close the block. This will delete the block's buffer file if the - * block has not previously been closed. - */ - void closeBlock() { - LOG.debug("block[{}]: closeBlock()", getIndex()); - if (!closed.getAndSet(true)) { - if (!bufferFile.delete() && bufferFile.exists()) { - LOG.warn("delete({}) returned false", bufferFile.getAbsoluteFile()); - } - } else { - LOG.debug("block[{}]: skipping re-entrant closeBlock()", getIndex()); - } + @Override + void flush() throws IOException { + super.flush(); + outputStream.flush(); + } + + @Override + boolean hasCapacity(final long bytes) { + return dataSize() + bytes <= limit; } } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileConflictException.java similarity index 54% rename from hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java rename to hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileConflictException.java index 6b4212a..767804d 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/FileConflictException.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileConflictException.java @@ -5,8 +5,9 @@ /** * OBS file conflict exception. */ -class FileConflictException extends IOException { +class OBSFileConflictException extends IOException implements WithErrCode { private static final long serialVersionUID = -897856973823710492L; + private String errCode; /** * Constructs a FileConflictException with the specified detail @@ -14,9 +15,18 @@ class FileConflictException extends IOException { * {@link java.lang.Throwable#getMessage} * method of class java.lang.Throwable. * - * @param s the detail message. + * @param message the detail message. */ - FileConflictException(final String s) { - super(s); + OBSFileConflictException(final String message) { + super(message); + } + + public void setErrCode(String errCode) { + this.errCode = errCode; + } + + @Override + public String getErrCode() { + return this.errCode; } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java index 30dca8c..f0b84af 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileStatus.java @@ -22,6 +22,7 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; /** * File status for an OBS file. @@ -30,7 +31,10 @@ */ @InterfaceAudience.Private @InterfaceStability.Evolving -class OBSFileStatus extends FileStatus { +public class OBSFileStatus extends FileStatus { + + private String etag; + /** * Create a directory status. * @@ -83,4 +87,30 @@ class OBSFileStatus extends FileStatus { setOwner(owner); setGroup(owner); } + + /** + * A simple file with etag. + * + * @param length file length + * @param modificationTime mod time + * @param path path + * @param blockSize block size + * @param owner owner + * @param etag etag + */ + OBSFileStatus(final long length, final long modificationTime, final Path path, final long blockSize, + final String owner, final String etag) { + this(length, modificationTime, path, blockSize, owner); + this.etag = etag; + } + + public String getEtag() { + return this.etag; + } + + protected void setAccessControlAttr(final String owner, final String group, final FsPermission fsPermission) { + super.setOwner(owner); + super.setGroup(group); + super.setPermission(fsPermission); + } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java index eb7dea2..be8eea0 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFileSystem.java @@ -23,12 +23,25 @@ import com.obs.services.ObsClient; import com.obs.services.exception.ObsException; import com.obs.services.model.AccessControlList; +import com.obs.services.model.select.CsvInput; +import com.obs.services.model.select.CsvOutput; +import com.obs.services.model.select.ExpressionType; +import com.obs.services.model.select.FileHeaderInfo; +import com.obs.services.model.select.InputSerialization; +import com.obs.services.model.select.JsonInput; +import com.obs.services.model.select.JsonOutput; +import com.obs.services.model.select.JsonType; +import com.obs.services.model.select.OrcInput; +import com.obs.services.model.select.OutputSerialization; +import com.obs.services.model.select.ScanRange; +import com.obs.services.model.select.SelectObjectRequest; +import com.obs.services.model.select.SelectObjectResult; -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.CreateFlag; import org.apache.hadoop.fs.FSDataInputStream; @@ -43,15 +56,20 @@ import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathFilter; import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.obs.input.OBSInputStream; import org.apache.hadoop.fs.obs.input.InputPolicyFactory; import org.apache.hadoop.fs.obs.input.InputPolicys; -import org.apache.hadoop.fs.obs.input.OBSInputStream; +import org.apache.hadoop.fs.obs.input.ObsSelectInputStream; +import org.apache.hadoop.fs.obs.memartscc.CcGetShardParam; +import org.apache.hadoop.fs.obs.memartscc.MemArtsCCClient; +import org.apache.hadoop.fs.obs.memartscc.ObjectShard; import org.apache.hadoop.fs.obs.security.AccessType; import org.apache.hadoop.fs.obs.security.AuthorizeProvider; -import org.apache.hadoop.fs.obs.security.DelegationTokenCapability; import org.apache.hadoop.fs.obs.security.OBSAuthorizationException; +import org.apache.hadoop.fs.obs.security.ObsDelegationTokenManger; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.security.AccessControlException; +import org.apache.hadoop.security.Credentials; import org.apache.hadoop.security.UserGroupInformation; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Progressable; @@ -63,9 +81,16 @@ import java.io.IOException; import java.lang.reflect.Constructor; import java.lang.reflect.InvocationTargetException; +import java.net.InetAddress; import java.net.URI; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Date; import java.util.EnumSet; import java.util.HashMap; +import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.concurrent.LinkedBlockingQueue; import java.util.concurrent.ThreadPoolExecutor; @@ -87,13 +112,13 @@ //CHECKSTYLE:OFF @InterfaceAudience.Private @InterfaceStability.Evolving -public class OBSFileSystem extends FileSystem { +public class OBSFileSystem extends FileSystem implements OpenFileWithJobConf { //CHECKSTYLE:ON /** * Class logger. */ - public static final Logger LOG = LoggerFactory.getLogger(OBSFileSystem.class); + private static final Logger LOG = LoggerFactory.getLogger(OBSFileSystem.class); /** * Flag indicating if the filesystem instance is closed. @@ -119,11 +144,17 @@ public class OBSFileSystem extends FileSystem { private int invokeCountThreshold = 0; + private ObsDelegationTokenManger obsDelegationTokenManger; /** * OBS client instance. */ private ObsClient obs; + /** + * memartscc client instance. + */ + private MemArtsCCClient memArtsCCClient; + /** * Metrics consumer. */ @@ -248,12 +279,18 @@ public class OBSFileSystem extends FileSystem { /** * Flag indicating if fast delete is enabled. */ - private boolean enableTrash = false; + private boolean enableFastDelete = false; + + private String fastDeleteVersion; /** * Trash directory for fast delete. */ - private String trashDir; + private String fastDeleteDir; + + private String hdfsTrashVersion; + + private String hdfsTrashPrefix; /** * OBS redefined access control list. @@ -282,10 +319,23 @@ public class OBSFileSystem extends FileSystem { * currently being written by this client. Note that a file can only be * written by a single filesystem. */ - private Map filesBeingWritten = new HashMap<>(); + private final Map filesBeingWritten = new HashMap<>(); private boolean enableFileVisibilityAfterCreate = false; + private String readPolicy = OBSConstants.READAHEAD_POLICY_PRIMARY; + + private boolean localityEnabled; + + private TrafficStatisticsReporter statsReporter; + + private TrafficStatistics trafficStatistics; + + /** + * obs file system permissions mode. + */ + private String permissionsMode; + /** * Close all {@link FSDataOutputStream} opened by the owner {@link * OBSFileSystem}. @@ -347,6 +397,23 @@ boolean isFileBeingWritten(String file) { public void initialize(final URI name, final Configuration originalConf) throws IOException { uri = URI.create(name.getScheme() + "://" + name.getAuthority()); bucket = name.getAuthority(); + + // Delegation token only mode. + if (originalConf.getBoolean(OBSConstants.DELEGATION_TOKEN_ONLY, OBSConstants.DEFAULT_DELEGATION_TOKEN_ONLY) + && OBSCommonUtils.isStringEmpty(bucket)) { + LOG.debug("Delegation-token-only mode"); + obsDelegationTokenManger = OBSCommonUtils.initDelegationTokenManger(this, uri, originalConf); + if (originalConf.getTrimmed(OBSConstants.READAHEAD_POLICY, + OBSConstants.READAHEAD_POLICY_PRIMARY).equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) { + memArtsCCClient = new MemArtsCCClient(bucket, enablePosix); + boolean initRes = memArtsCCClient.initializeDtOnly(originalConf); + if (!initRes) { + LOG.warn("MemArtsCCClient dt-only initialize failed!"); + } + } + return; + } + // clone the configuration into one with propagated bucket options Configuration conf = OBSCommonUtils.propagateBucketOptions(originalConf, bucket); OBSCommonUtils.patchSecurityCredentialProviders(conf); @@ -367,6 +434,8 @@ public void initialize(final URI name, final Configuration originalConf) throws OBSConstants.OBS_CLIENT_FACTORY_IMPL, OBSConstants.DEFAULT_OBS_CLIENT_FACTORY_IMPL, OBSClientFactory.class); obs = ReflectionUtils.newInstance(obsClientFactoryClass, conf).createObsClient(name); + OBSCommonUtils.init(this, conf); + sse = new SseWrapper(conf); Class metricsConsumerClass = conf.getClass( @@ -382,8 +451,12 @@ public void initialize(final URI name, final Configuration originalConf) throws } } - OBSCommonUtils.verifyBucketExists(this); - enablePosix = OBSCommonUtils.getBucketFsStatus(obs, bucket); + try { + enablePosix = OBSCommonUtils.getBucketFsStatus(obs, bucket); + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.initialize, e); + throw e; + } maxKeys = OBSCommonUtils.intOption(conf, OBSConstants.MAX_PAGING_KEYS, OBSConstants.DEFAULT_MAX_PAGING_KEYS, 1); @@ -421,64 +494,96 @@ public void initialize(final URI name, final Configuration originalConf) throws LOG.debug("Using OBSBlockOutputStream with buffer = {}; block={};" + " queue limit={}", blockOutputBuffer, partSize, blockOutputActiveBlocks); - String readPolicy = conf.getTrimmed(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_PRIMARY); + readPolicy = conf.getTrimmed(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_PRIMARY); inputPolicyFactory = InputPolicys.createFactory(readPolicy); - enableTrash = conf.getBoolean(OBSConstants.TRASH_ENABLE, OBSConstants.DEFAULT_TRASH); - if (enableTrash) { + // initialize MemArtsCC + if (readPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) { + if (initMemArtsCC(conf, name)) { + // Traffic Report + initTrafficReport(conf); + } + } + + obsDelegationTokenManger = OBSCommonUtils.initDelegationTokenManger(this, uri, conf); + localityEnabled = conf.getBoolean(OBSConstants.MEMARTSCC_LOCALITY_ENABLE, OBSConstants.DEFAULT_MEMARTSCC_LOCALITY_ENABLE); + + hdfsTrashVersion = conf.get(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V1); + hdfsTrashPrefix = conf.get(OBSConstants.HDFS_TRASH_PREFIX, OBSConstants.DEFAULT_HDFS_TRASH_PREFIX); + enableFastDelete = conf.getBoolean(OBSConstants.FAST_DELETE_ENABLE, OBSConstants.DEFAULT_FAST_DELETE_ENABLE); + if (enableFastDelete) { if (!isFsBucket()) { String errorMsg = String.format("The bucket [%s] is not posix. not supported for " + "trash.", bucket); LOG.warn(errorMsg); - enableTrash = false; - trashDir = null; + enableFastDelete = false; + fastDeleteDir = null; } else { - trashDir = conf.get(OBSConstants.TRASH_DIR); - if (StringUtils.isEmpty(trashDir)) { + String fastDelete = "FastDelete"; + String defaultTrashDir = OBSCommonUtils.maybeAddTrailingSlash( + this.getTrashRoot(new Path(fastDelete)).toUri().getPath()) + fastDelete; + fastDeleteDir = conf.get(OBSConstants.FAST_DELETE_DIR, defaultTrashDir); + if (OBSCommonUtils.isStringEmpty(fastDeleteDir)) { String errorMsg = String.format( - "The trash feature(fs.obs.trash.enable) is " + "enabled, but the " - + "configuration(fs.obs.trash.dir [%s]) " + "is empty.", trashDir); + "The fast delete feature(fs.obs.trash.enable) is " + "enabled, but the " + + "configuration(fs.obs.trash.dir [%s]) " + "is empty.", fastDeleteDir); LOG.error(errorMsg); throw new ObsException(errorMsg); } - trashDir = OBSCommonUtils.maybeAddBeginningSlash(trashDir); - trashDir = OBSCommonUtils.maybeAddTrailingSlash(trashDir); + fastDeleteDir = OBSCommonUtils.maybeAddBeginningSlash(fastDeleteDir); + fastDeleteDir = OBSCommonUtils.maybeAddTrailingSlash(fastDeleteDir); + fastDeleteVersion = conf.get(OBSConstants.FAST_DELETE_VERSION, OBSConstants.FAST_DELETE_VERSION_V1); } } - OBSCommonUtils.setMaxTimeInMillisecondsToRetry(conf.getLong(OBSConstants.MAX_TIME_IN_MILLISECOND_TO_RETRY, - OBSConstants.DEFAULT_TIME_IN_MILLISECOND_TO_RETRY)); - enableCanonicalServiceName = - conf.getBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE, - OBSConstants.DEFAULT_GET_CANONICAL_SERVICE_NAME_ENABLE); + + OBSCommonUtils.setRetryTime(conf.getLong(OBSConstants.RETRY_MAXTIME, OBSConstants.DEFAULT_RETRY_MAXTIME), + conf.getLong(OBSConstants.RETRY_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_SLEEP_BASETIME), + conf.getLong(OBSConstants.RETRY_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_SLEEP_MAXTIME)); enableFileVisibilityAfterCreate = conf.getBoolean(OBSConstants.FILE_VISIBILITY_AFTER_CREATE_ENABLE, OBSConstants.DEFAULT_FILE_VISIBILITY_AFTER_CREATE_ENABLE); enableCanonicalServiceName = conf.getBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE, OBSConstants.DEFAULT_GET_CANONICAL_SERVICE_NAME_ENABLE); - + permissionsMode = conf.get(OBSConstants.PERMISSIONS_MODE, OBSConstants.DEFAULT_PERMISSIONS_MODE); this.authorizer = initAuthorizeProvider(conf); } catch (ObsException e) { + LOG.error("initializing OBSFilesystem fail", e); throw OBSCommonUtils.translateException("initializing ", new Path(name), e); } LOG.info("Finish initializing filesystem instance for uri: {}", uri); } + private void initTrafficReport(final Configuration conf) { + boolean reportEnable = conf.getBoolean(OBSConstants.MEMARTSCC_TRAFFIC_REPORT_ENABLE, OBSConstants.DEFAULT_MEMARTSCC_TRAFFIC_REPORT_ENABLE); + LOG.debug("get report enable from config:{}", reportEnable); + if (!reportEnable) { + return; + } + + trafficStatistics = new TrafficStatistics(); + long interval = getConf(). + getLong(OBSConstants.MEMARTSCC_TRAFFIC_REPORT_INTERVAL, OBSConstants.MEMARTSCC_TRAFFIC_REPORT_DEFAULT_INTERVAL); + LOG.debug("get report traffic interval from config: interval:{}", interval); + statsReporter = new TrafficStatisticsReporter(trafficStatistics, memArtsCCClient, interval); + statsReporter.startReport(); + } + private AuthorizeProvider initAuthorizeProvider(Configuration conf) throws IOException { - AuthorizeProvider authorizer = null; + AuthorizeProvider authorizeProvider = null; Class authClassName = conf.getClass(OBSConstants.AUTHORIZER_PROVIDER, null); if (authClassName != null) { try { LOG.info("authorize provider is " + authClassName.getName()); - authorizer = (AuthorizeProvider)authClassName.newInstance(); - authorizer.init(conf); + authorizeProvider = (AuthorizeProvider)authClassName.newInstance(); + authorizeProvider.init(conf); } catch (Exception e) { LOG.error(String.format("init %s failed", OBSConstants.AUTHORIZER_PROVIDER), e); throw new IOException(String.format("init %s failed", OBSConstants.AUTHORIZER_PROVIDER), e); } } - return authorizer; + return authorizeProvider; } private void checkPermission(Path path, AccessType accessType) throws IOException { @@ -497,7 +602,7 @@ private void checkPermission(Path path, AccessType accessType) throws IOExceptio long st = System.currentTimeMillis(); result = authorizer.isAuthorized(this.bucket, key, accessType); long et = System.currentTimeMillis(); - if (LOG.isDebugEnabled()){ + if (LOG.isDebugEnabled()) { LOG.debug("authorizing:[user: {}], [action: {}], " + "[bucket: {}], [path: {}], [result: {}], [cost: {}]", currentUser, @@ -530,6 +635,19 @@ private void checkPermission(Path path, AccessType accessType) throws IOExceptio } } + private boolean initMemArtsCC(final Configuration conf, final URI name) { + String obsBucketName = getBucket(); + memArtsCCClient = new MemArtsCCClient(obsBucketName, enablePosix); + if (!memArtsCCClient.initialize(name, conf)) { + LOG.warn("fallback to 'primary' read policy"); + // fall back to 'primary' read policy + inputPolicyFactory = InputPolicys.createFactory(OBSConstants.READAHEAD_POLICY_PRIMARY); + // do not set memArtsCCClient to null + return false; + } + return true; + } + private void initThreadPools(final Configuration conf) { long keepAliveTime = OBSCommonUtils.longOption(conf, OBSConstants.KEEPALIVE_TIME, OBSConstants.DEFAULT_KEEPALIVE_TIME, 0); @@ -715,6 +833,16 @@ public ObsClient getObsClient() { return obs; } + /** + * Return the MemArtsCC Client used by this filesystem. + * + * @return MemArtsCCClient + */ + @VisibleForTesting + public MemArtsCCClient getMemArtsCCClient() { + return memArtsCCClient; + } + /** * Return the read ahead range used by this filesystem. * @@ -757,6 +885,32 @@ protected URI canonicalizeUri(final URI rawUri) { return OBSLoginHelper.canonicalizeUri(rawUri, getDefaultPort()); } + /** + * Open an FSDataInputStream at the indicated Path with the given configuration. + * + * @param f + * @param jobConf + * @return + * @throws IOException + */ + @Override + public FSDataInputStream open(Path f, Configuration jobConf) throws IOException { + String sqlExpr = jobConf.get("mapreduce.job.input.file.option.fs.obs.select.sql"); + if (sqlExpr != null) { + String key = OBSCommonUtils.pathToKey(this, f); + String scanStart = jobConf.get("mapreduce.job.input.file.option.fs.obs.select.scan.start"); + String scanEnd = jobConf.get("mapreduce.job.input.file.option.fs.obs.select.scan.end"); + LOG.info("OBSFileSystem.open(): bucket: {}; key: {}; sql: {}; range: [{}, {}]%n", + bucket, key, + sqlExpr, + scanStart != null ? scanStart : "n/a", + scanEnd != null ? scanEnd : "n/a"); + + return select(f, sqlExpr, jobConf); + } + return open(f); + } + /** * Open an FSDataInputStream at the indicated Path. * @@ -768,43 +922,146 @@ protected URI canonicalizeUri(final URI rawUri) { @Override public FSDataInputStream open(final Path f, final int bufferSize) throws IOException { checkOpen(); - long startTime = System.currentTimeMillis(); - long endTime; LOG.debug("Opening '{}' for reading.", f); - final FileStatus fileStatus; + final OBSFileStatus fileStatus; try { - fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.OPEN, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, f); + } catch (OBSFileConflictException e) { throw new AccessControlException(e); } if (fileStatus.isDirectory()) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.OPEN, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw new FileNotFoundException("Can't open " + f + " because it is a directory"); } checkPermission(f, AccessType.READ); FSInputStream fsInputStream = inputPolicyFactory.create(this, bucket, OBSCommonUtils.pathToKey(this, f), - fileStatus.getLen(), statistics, boundedMultipartUploadThreadPool); - FSDataInputStream fsDataInputStream = new FSDataInputStream(fsInputStream); + fileStatus.getLen(), statistics, boundedMultipartUploadThreadPool, fileStatus); + return new FSDataInputStream(fsInputStream); + } + + private FSDataInputStream select(Path f, String sqlExpr, Configuration jobConf) throws IOException { + final FileStatus fileStatus = getFileStatus(f); + if (fileStatus.isDirectory()) { + throw new FileNotFoundException("Can't open " + f + " because it is a directory"); + } + + String key = OBSCommonUtils.pathToKey(this, f); + + final String obsSelectJobPath = "mapreduce.job.input.file.option.fs.obs.select."; + String fileFormat = jobConf.get(obsSelectJobPath + "format"); + if (fileFormat == null) { + throw new IllegalArgumentException("file format is missing"); + } + + if (!fileFormat.equals("orc") && + !fileFormat.equals("csv") && + !fileFormat.equals("json")) { + throw new IllegalArgumentException("invalid file format '" + fileFormat + "', it must be one of { 'csv', 'json', 'orc' }"); + } + + SelectObjectRequest selectRequest = new SelectObjectRequest() + .withExpression(sqlExpr) + .withBucketName(bucket) + .withKey(key) + .withExpressionType(ExpressionType.SQL); + + if (!fileFormat.equals("json")) { + // prepare input & output for CSV & ORC + CsvOutput output = new CsvOutput(); + String delimiter = jobConf.get(obsSelectJobPath + "output.csv.delimiter"); + if (delimiter != null) { + if (delimiter.length() != 1) { + throw new IllegalArgumentException("Invalid output delimiter " + delimiter); + } + + output.withFieldDelimiter(delimiter.charAt(0)); + } + + // prepare the request + if (fileFormat.equals("csv")) { + CsvInput input = new CsvInput(); + + delimiter = jobConf.get(obsSelectJobPath + "input.csv.delimiter"); + if (delimiter != null) { + if (delimiter.length() != 1) + throw new IllegalArgumentException("Invalid input delimiter " + delimiter); + + input.withFieldDelimiter(delimiter.charAt(0)); + } + + String headerInfo = jobConf.get(obsSelectJobPath + "input.csv.header"); + if (headerInfo != null) { + boolean found = false; + for (FileHeaderInfo enumEntry : FileHeaderInfo.values()) { + found = (enumEntry.toString().equals(headerInfo)); + if (found) { + input.withFileHeaderInfo(enumEntry); + break; + } + } + + if (!found) { + throw new IllegalArgumentException("Invalid header " + headerInfo); + } + } + + selectRequest.withInputSerialization( + new InputSerialization() + .withCsv(input) + ).withOutputSerialization( + new OutputSerialization() + .withCsv(output) + ); + + } else { + // it is ORC + selectRequest.withInputSerialization( + new InputSerialization() + .withOrc( + new OrcInput() + ) + ).withOutputSerialization( + new OutputSerialization() + .withCsv(output) + ); + } + } else { + // it is JSON + JsonType type = jobConf.get(obsSelectJobPath + "input.json.type").equals("lines") ? + JsonType.LINES : JsonType.DOCUMENT; + + selectRequest.withInputSerialization( + new InputSerialization() + .withJson( + new JsonInput() + .withType(type) + ) + ).withOutputSerialization( + new OutputSerialization() + .withJson( + new JsonOutput() + ) + ); + } - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.OPEN, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); + // Set Scan Range properties. + String scanStart = jobConf.get(obsSelectJobPath + "scan.start"); + String scanEnd = jobConf.get(obsSelectJobPath + "scan.end"); + if (scanStart != null && scanEnd != null) { + selectRequest.withScanRange( + new ScanRange() + .withStart(Long.parseLong(scanStart)) + .withEnd(Long.parseLong(scanEnd))); } - return fsDataInputStream; + + LOG.info("OBSFileSystem.select(): bucket: {}; key: {}; sql: {}; range: [{}, {}]\n", + bucket, key, sqlExpr, + scanStart != null ? scanStart : "n/a", + scanEnd != null ? scanEnd : "n/a"); + + SelectObjectResult selectResult = obs.selectObjectContent(selectRequest); + return new FSDataInputStream( + new ObsSelectInputStream(bucket, key, selectResult)); } /** @@ -830,45 +1087,21 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi checkOpen(); String key = OBSCommonUtils.pathToKey(this, f); final FileStatus status; - long startTime = System.currentTimeMillis(); - long endTime; boolean exist = true; try { // get the status or throw an exception try { - status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } - + status = OBSCommonUtils.getFileStatusWithRetry(this, f); + } catch (OBSFileConflictException e) { throw new ParentNotDirectoryException(e.getMessage()); } // if the thread reaches here, there is something at the path if (status.isDirectory()) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } // path references a directory: automatic error throw new FileAlreadyExistsException(f + " is a directory"); } if (!overwrite) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } // path references a file and overwrite is disabled throw new FileAlreadyExistsException(f + " already exists"); } @@ -895,13 +1128,6 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi filesBeingWritten.put(key, outputStream); } - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.OVERWRITE, BasicMetricsConsumer.MetricRecord.CREATE, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return outputStream; } @@ -952,8 +1178,6 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi final int bufferSize, final short replication, final long blkSize, final Progressable progress, final ChecksumOpt checksumOpt) throws IOException { checkOpen(); - long startTime = System.currentTimeMillis(); - long endTime; LOG.debug("create: Creating new file {}, flags:{}, isFsBucket:{}", f, flags, isFsBucket()); OBSCommonUtils.checkCreateFlag(flags); FSDataOutputStream outputStream; @@ -968,27 +1192,13 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi try { // get the status or throw an FNFE try { - status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + status = OBSCommonUtils.getFileStatusWithRetry(this, f); + } catch (OBSFileConflictException e) { throw new ParentNotDirectoryException(e.getMessage()); } // if the thread reaches here, there is something at the path if (status.isDirectory()) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } // path references a directory: automatic error throw new FileAlreadyExistsException(f + " is a directory"); } @@ -1010,26 +1220,10 @@ public FSDataOutputStream create(final Path f, final FsPermission permission, fi synchronized (filesBeingWritten) { filesBeingWritten.put(key, outputStream); } - - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return outputStream; } else { outputStream = create(f, permission, flags == null || flags.contains(CreateFlag.OVERWRITE), bufferSize, replication, blkSize, progress); - - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.FLAGS, BasicMetricsConsumer.MetricRecord.CREATE, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return outputStream; } } @@ -1053,28 +1247,13 @@ public FSDataOutputStream createNonRecursive(final Path path, final FsPermission final EnumSet flags, final int bufferSize, final short replication, final long blkSize, final Progressable progress) throws IOException { checkOpen(); - long startTime = System.currentTimeMillis(); - long endTime; OBSCommonUtils.checkCreateFlag(flags); if (path.getParent() != null && !this.exists(path.getParent())) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.CREATE_NR, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw new FileNotFoundException(path.toString() + " parent directory not exist."); } - FSDataOutputStream fsDataOutputStream = create(path, permission, flags.contains(CreateFlag.OVERWRITE), + return create(path, permission, flags.contains(CreateFlag.OVERWRITE), bufferSize, replication, blkSize, progress); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.CREATE_NR, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } - return fsDataOutputStream; } /** @@ -1089,8 +1268,6 @@ public FSDataOutputStream createNonRecursive(final Path path, final FsPermission public FSDataOutputStream append(final Path f, final int bufferSize, final Progressable progress) throws IOException { checkOpen(); - long startTime = System.currentTimeMillis(); - long endTime; if (!isFsBucket()) { throw new UnsupportedOperationException("non-posix bucket. Append is not supported " + "by OBSFileSystem"); } @@ -1100,26 +1277,14 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr // get the status or throw an FNFE FileStatus status; try { - status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.APPEND, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + status = OBSCommonUtils.getFileStatusWithRetry(this, f); + } catch (OBSFileConflictException e) { throw new AccessControlException(e); } long objectLen = status.getLen(); // if the thread reaches here, there is something at the path if (status.isDirectory()) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.APPEND, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } // path references a directory: automatic error throw new FileAlreadyExistsException(f + " is a directory"); } @@ -1137,35 +1302,9 @@ public FSDataOutputStream append(final Path f, final int bufferSize, final Progr synchronized (filesBeingWritten) { filesBeingWritten.put(key, outputStream); } - - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.APPEND, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return outputStream; } - /** - * Truncate the file in the indicated path to the indicated size. - *

    - *
  • Fails if path is a directory.
  • - *
  • Fails if path does not exist.
  • - *
  • Fails if path is not closed.
  • - *
  • Fails if new size is greater than current size.
  • - *
- * - * @param f The path to the file to be truncated - * @param newLength The size the file is to be truncated to - * @return true if the file has been truncated to the desired - * newLength and is immediately available to be reused for - * write operations such as append, or - * false if a background process of adjusting the length of - * the last block has been started, and clients should wait for it to - * complete before proceeding with further file updates. - * @throws IOException IO failure - */ @Override public boolean truncate(Path f, long newLength) throws IOException { checkOpen(); @@ -1179,8 +1318,8 @@ public boolean truncate(Path f, long newLength) throws IOException { FileStatus status; try { - status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); - } catch (FileConflictException e) { + status = OBSCommonUtils.getFileStatusWithRetry(this, f); + } catch (OBSFileConflictException e) { throw new AccessControlException(e); } @@ -1192,7 +1331,7 @@ public boolean truncate(Path f, long newLength) throws IOException { if (isFileBeingWritten(key)) { // AlreadyBeingCreatedException (on HDFS NameNode) is transformed // into IOException (on HDFS Client) - throw new AlreadyBeingCreatedException("Cannot truncate " + f + " that is being written."); + throw new OBSAlreadyBeingCreatedException("Cannot truncate " + f + " that is being written."); } // Truncate length check. @@ -1249,51 +1388,29 @@ public boolean rename(final Path src, final Path dst) throws IOException { checkPermission(src, AccessType.WRITE); checkPermission(dst,AccessType.WRITE); boolean success = OBSPosixBucketUtils.renameBasedOnPosix(this, src, dst); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.RENAME, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.rename, startTime); return success; } else { checkPermission(src, AccessType.WRITE); checkPermission(dst, AccessType.WRITE); boolean success = OBSObjectBucketUtils.renameBasedOnObject(this, src, dst); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.RENAME, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.rename, startTime); return success; } } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.RENAME, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e); throw OBSCommonUtils.translateException("rename(" + src + ", " + dst + ")", src, e); - } catch (RenameFailedException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.RENAME, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + } catch (OBSRenameFailedException e) { + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e); LOG.error(e.getMessage()); return e.getExitCode(); } catch (FileNotFoundException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.RENAME, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } - LOG.error(e.toString()); + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e); + LOG.error("file not found when rename(" + src + ", " + dst + ")"); return false; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.rename, e); + throw e; } finally { endTime = System.currentTimeMillis(); LOG.debug("Rename path {} to {} finished, thread:{}, " + "timeUsedInMilliSec:{}.", src, dst, threadId, @@ -1301,6 +1418,47 @@ public boolean rename(final Path src, final Path dst) throws IOException { } } + @Override + public Path getTrashRoot(Path path) { + if (OBSConstants.HDFS_TRASH_VERSION_V2.equals(hdfsTrashVersion)) { + return this.makeQualified(new Path(hdfsTrashPrefix, getUsername())); + } + + return super.getTrashRoot(path); + } + + @Override + public Collection getTrashRoots(boolean allUsers) { + if (OBSConstants.HDFS_TRASH_VERSION_V2.equals(hdfsTrashVersion)) { + Path trashPrefix = new Path(hdfsTrashPrefix); + List ret = new ArrayList<>(); + try { + if (!exists(trashPrefix)) { + return ret; + } + + if (allUsers) { + FileStatus[] candidates = listStatus(trashPrefix); + for (FileStatus fs : candidates) { + ret.add(fs); + } + return ret; + } + + Path userTrash = new Path(trashPrefix, getUsername()); + if (exists(userTrash)) { + ret.add(getFileStatus(userTrash)); + } + } catch (IOException e) { + LOG.warn("Cannot get all trash roots", e); + } + + return ret; + } + + return super.getTrashRoots(allUsers); + } + /** * Return maximum number of entries in one multi-object delete call. * @@ -1381,56 +1539,30 @@ boolean isEnableMultiObjectDelete() { public boolean delete(final Path f, final boolean recursive) throws IOException { checkOpen(); long startTime = System.currentTimeMillis(); - long endTime; try { - FileStatus status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); + FileStatus status = OBSCommonUtils.getFileStatusWithRetry(this, f); LOG.debug("delete: path {} - recursive {}", status.getPath(), recursive); if (enablePosix) { checkPermission(f,AccessType.WRITE); boolean success = OBSPosixBucketUtils.fsDelete(this, status, recursive); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.DELETE, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.delete, startTime); return success; } checkPermission(f,AccessType.WRITE); - boolean success = OBSObjectBucketUtils.objectDelete(this, status, recursive); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.DELETE, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } - return success; + return OBSObjectBucketUtils.objectDelete(this, status, recursive); } catch (FileNotFoundException e) { LOG.warn("Couldn't delete {} - does not exist", f); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.DELETE, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return false; - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.DELETE, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + } catch (OBSFileConflictException e) { + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.delete, e); throw new AccessControlException(e); } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.DELETE, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.delete, e); throw OBSCommonUtils.translateException("delete", f, e); + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.delete, e); + throw e; } } @@ -1439,8 +1571,8 @@ public boolean delete(final Path f, final boolean recursive) throws IOException * * @return the flag */ - boolean isEnableTrash() { - return enableTrash; + boolean isEnableFastDelete() { + return enableFastDelete; } /** @@ -1448,8 +1580,15 @@ boolean isEnableTrash() { * * @return the trash directory */ - String getTrashDir() { - return trashDir; + String getFastDeleteDir() { + if (fastDeleteVersion.equals(OBSConstants.FAST_DELETE_VERSION_V2)) { + SimpleDateFormat dateFmt = new SimpleDateFormat(OBSConstants.FAST_DELETE_VERSION_V2_CHECKPOINT_FORMAT); + String checkpointStr = dateFmt.format(new Date()); + String checkpointDir = String.format(Locale.ROOT, "%s%s/", + OBSCommonUtils.maybeAddTrailingSlash(fastDeleteDir), checkpointStr); + return checkpointDir; + } + return fastDeleteDir; } /** @@ -1469,28 +1608,31 @@ public FileStatus[] listStatus(final Path f) throws FileNotFoundException, IOExc long threadId = Thread.currentThread().getId(); long endTime; try { - FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f, false); + FileStatus[] statuses = OBSCommonUtils.listStatus(this, f, false); endTime = System.currentTimeMillis(); LOG.debug("List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f, threadId, endTime - startTime); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.NONRECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return statuses; } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.NONRECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS, - false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw OBSCommonUtils.translateException("listStatus", f, e); } } + @Override + public Path getHomeDirectory() { + return this.makeQualified(new Path("/user/" + getUsername())); + } + + static String getUsername() { + String user; + try { + user = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch(IOException ex) { + LOG.error("get user fail,fallback to system property user.name", ex); + user = System.getProperty("user.name"); + } + return user; + } + /** * This public interface is provided specially for Huawei MRS. List the * statuses of the files/directories in the given path if the path is a @@ -1510,24 +1652,11 @@ public FileStatus[] listStatus(final Path f, final boolean recursive) throws Fil long threadId = Thread.currentThread().getId(); long endTime; try { - FileStatus[] statuses = OBSCommonUtils.innerListStatus(this, f, recursive); + FileStatus[] statuses = OBSCommonUtils.listStatus(this, f, recursive); endTime = System.currentTimeMillis(); LOG.debug("List status for path:{}, thread:{}, timeUsedInMilliSec:{}", f, threadId, endTime - startTime); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.RECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return statuses; } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.RECURSIVE, BasicMetricsConsumer.MetricRecord.LIST_STATUS, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw OBSCommonUtils.translateException( "listStatus with recursive flag[" + (recursive ? "true] " : "false] "), f, e); } @@ -1592,28 +1721,70 @@ public boolean mkdirs(final Path path, final FsPermission permission) throws IOException, FileAlreadyExistsException { checkOpen(); checkPermission(path,AccessType.WRITE); - long startTime = System.currentTimeMillis(); - long endTime; try { - boolean success = OBSCommonUtils.innerMkdirs(this, path); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.MKDIRS, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } - return success; + return OBSCommonUtils.mkdirs(this, path); } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.MKDIRS, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw OBSCommonUtils.translateException("mkdirs", path, e); } } + @Override + public BlockLocation[] getFileBlockLocations(FileStatus file, long start, long len) throws IOException { + if (!readPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC) + || this.getMemArtsCCClient() == null || !localityEnabled) { + // only support for MemArtsCC + return super.getFileBlockLocations(file, start, len); + } + if (file == null || start < 0L || len < 0L) { + return super.getFileBlockLocations(file, start, len); + } else if (file.isDirectory()) { + return super.getFileBlockLocations(file, start, len); + } else if (file.getLen() <= start) { + return new BlockLocation[0]; + } else { + MemArtsCCClient ccClient = this.getMemArtsCCClient(); + CcGetShardParam shardParam = buildCcShardParam(file, start, len); + int result = ccClient.getObjectShardInfo(shardParam); + + if (result != OBSConstants.GET_SHARD_INFO_SUCCESS) { + LOG.error("Get memartscc shard info failed! ret code = {}", result); + return new BlockLocation[0]; + } + + ObjectShard[] objectShards = shardParam.getObjectShard(); + int numOfBlocks = shardParam.getValidShardNum(); + BlockLocation[] locations = new BlockLocation[numOfBlocks]; + for(int i = 0; i < numOfBlocks; ++i) { + long offset = objectShards[i].getStart(); + long length = objectShards[i].getEnd() - offset; + String[] hosts = objectShards[i].getHosts(); + for (int j = 0; j < hosts.length; j++) { + InetAddress addr = InetAddress.getByName(hosts[j]); + String hostName = addr.getHostName(); + hosts[j] = hostName; + } + locations[i] = new BlockLocation(hosts, hosts, offset, length); + } + return locations; + } + } + + private CcGetShardParam buildCcShardParam(FileStatus file, long start, long len) { + String obsBucketName = getBucket(); + long end = start + len; + String objKey = OBSCommonUtils.pathToKey(this, file.getPath()); + + int shardNum = (int) (len / this.blockSize + 1); + ObjectShard[] objectShards = new ObjectShard[shardNum]; + for (int i = 0; i < shardNum; i++) { + String[] hosts = new String[OBSConstants.MAX_DUPLICATION_NUM]; + objectShards[i] = new ObjectShard(hosts); + } + + return new CcGetShardParam(start, end, obsBucketName, enablePosix, objKey, + objectShards, shardNum, 0); + } + /** * Return a file status object that represents the path. * @@ -1626,26 +1797,16 @@ public boolean mkdirs(final Path path, final FsPermission permission) public FileStatus getFileStatus(final Path f) throws FileNotFoundException, IOException { checkOpen(); long startTime = System.currentTimeMillis(); - long endTime; try { - FileStatus fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.GET_FILE_STATUS, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + FileStatus fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, f); + OBSCommonUtils.setMetricsNormalInfo(this, OBSOperateAction.getFileStatus, startTime); return fileStatus; - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.GET_FILE_STATUS, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + } catch (OBSFileConflictException e) { + FileNotFoundException fileNotFoundException = new FileNotFoundException(e.getMessage()); + OBSCommonUtils.setMetricsAbnormalInfo(this, OBSOperateAction.getFileStatus, fileNotFoundException); // For super user, convert AccessControlException // to null on NameNode then to FileNotFoundException on Client - throw new FileNotFoundException(e.getMessage()); + throw fileNotFoundException; } } @@ -1658,13 +1819,10 @@ public FileStatus getFileStatus(final Path f) throws FileNotFoundException, IOEx */ @VisibleForTesting OBSFileStatus innerGetFileStatus(final Path f) throws IOException { - OBSFileStatus fileStatus; if (enablePosix) { - fileStatus = OBSPosixBucketUtils.innerFsGetObjectStatus(this, f); - return fileStatus; + return OBSPosixBucketUtils.innerFsGetObjectStatus(this, f); } - fileStatus = OBSObjectBucketUtils.innerGetObjectStatus(this, f); - return fileStatus; + return OBSObjectBucketUtils.innerGetObjectStatus(this, f); } /** @@ -1678,8 +1836,6 @@ OBSFileStatus innerGetFileStatus(final Path f) throws IOException { @Override public ContentSummary getContentSummary(final Path f) throws FileNotFoundException, IOException { checkOpen(); - long startTime = System.currentTimeMillis(); - long endTime; ContentSummary contentSummary; if (!obsContentSummaryEnable) { return super.getContentSummary(f); @@ -1687,14 +1843,8 @@ public ContentSummary getContentSummary(final Path f) throws FileNotFoundExcepti FileStatus status; try { - status = OBSCommonUtils.innerGetFileStatusWithRetry(this, f); - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + status = OBSCommonUtils.getFileStatusWithRetry(this, f); + } catch (OBSFileConflictException e) { throw new AccessControlException(e); } @@ -1706,34 +1856,41 @@ public ContentSummary getContentSummary(final Path f) throws FileNotFoundExcepti .directoryCount(0) .spaceConsumed(length) .build(); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, true, endTime - startTime); - - OBSCommonUtils.setMetricsInfo(this, record); - } return contentSummary; } // f is a directory if (enablePosix) { - contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummary(this, OBSCommonUtils.pathToKey(this, f)); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); + contentSummary = null; + if (this.getConf() + .get(OBSConstants.OBS_CONTENT_SUMMARY_VERSION, OBSConstants.OBS_CONTENT_SUMMARY_VERSION_V2) + .equals(OBSConstants.OBS_CONTENT_SUMMARY_VERSION_V2)) { + boolean fallback = false; + try { + contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(this, status); + if (contentSummary.getFileCount() + contentSummary.getDirectoryCount() + < OBSConstants.OBS_CONTENT_SUMMARY_FALLBACK_THRESHOLD) { + fallback = true; + } + } catch (OBSMethodNotAllowedException e) { + LOG.debug("bucket[{}] not support fsGetDirectoryContentSummaryV2, fallback to V1, path={}, cause {}" + , bucket, f.toString(), e.getMessage()); + fallback = true; + } catch (Exception e) { + LOG.warn("fsGetDirectoryContentSummaryV2 failed with exception, fallback to V1, path={}, cause {}" + , f.toString(), e.getMessage()); + fallback = true; + } + + if (!fallback) { + return contentSummary; + } + LOG.debug("fallback to getContentSummaryV1, path={}", f.toString()); } + contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummary(this, OBSCommonUtils.pathToKey(this, f)); return contentSummary; } else { contentSummary = OBSObjectBucketUtils.getDirectoryContentSummary(this, OBSCommonUtils.pathToKey(this, f)); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.GET_CONTENT_SUMMARY, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return contentSummary; } } @@ -1754,23 +1911,9 @@ public ContentSummary getContentSummary(final Path f) throws FileNotFoundExcepti public void copyFromLocalFile(final boolean delSrc, final boolean overwrite, final Path src, final Path dst) throws FileAlreadyExistsException, IOException { checkOpen(); - long startTime = System.currentTimeMillis(); - long endTime; try { super.copyFromLocalFile(delSrc, overwrite, src, dst); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.COPYFROMLOCAL, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.COPYFROMLOCAL, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw OBSCommonUtils.translateException("copyFromLocalFile(" + src + ", " + dst + ")", src, e); } } @@ -1785,16 +1928,11 @@ public void close() throws IOException { // already closed return; } - long startTime = System.currentTimeMillis(); closeAllFilesBeingWritten(); - closed = true; - long endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.FS, BasicMetricsConsumer.MetricRecord.CLOSE, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); + + if (statsReporter != null) { + statsReporter.shutdownReport(); } try { @@ -1802,37 +1940,44 @@ public void close() throws IOException { if (metricsConsumer != null) { metricsConsumer.close(); } - obs.close(); + if (obs != null) { + obs.close(); + } } finally { OBSCommonUtils.shutdownAll(boundedMultipartUploadThreadPool, boundedCopyThreadPool, boundedDeleteThreadPool, boundedCopyPartThreadPool, boundedListThreadPool); } + if (memArtsCCClient != null) { + memArtsCCClient.close(); + } + LOG.info("Finish closing filesystem instance for uri: {}", uri); } + + @Override public String getCanonicalServiceName() { - if (authorizer != null && authorizer instanceof DelegationTokenCapability) { - LOG.debug("getting CanonicalServiceName"); - return ((DelegationTokenCapability)authorizer).getCanonicalServiceName(); - } else if (enableCanonicalServiceName) { - // Does not support Token, only enable for HBase BulkLoad - return getScheme() + "://" + bucket; - } - return null; + return obsDelegationTokenManger != null + ? obsDelegationTokenManger.getCanonicalServiceName() + : getScheme() + "://" + bucket; + } + + @Override + public Token[] addDelegationTokens(String renewer, Credentials credentials) throws IOException { + LOG.info("add delegation tokens for renewer {}", renewer); + return obsDelegationTokenManger != null + ? obsDelegationTokenManger.addDelegationTokens(renewer, credentials) + : super.addDelegationTokens(renewer, credentials); } @Override public Token getDelegationToken(String renewer) throws IOException { - if (authorizer != null && authorizer instanceof DelegationTokenCapability) { - long st = System.currentTimeMillis(); - Token delegationToken = ((DelegationTokenCapability) authorizer).getDelegationToken(renewer); - long et = System.currentTimeMillis(); - LOG.debug("getDelegationToken:[renewer: {}], [cost: {}]", renewer, et - st); - return delegationToken; - } - return super.getDelegationToken(renewer); + LOG.info("get delegation tokens for renewer {}", renewer); + return obsDelegationTokenManger != null + ? obsDelegationTokenManger.getDelegationToken(renewer) + : super.getDelegationToken(renewer); } /** @@ -1949,8 +2094,6 @@ public RemoteIterator listFiles(final Path f, final boolean r throws FileNotFoundException, IOException { checkOpen(); checkPermission(f,AccessType.READ); - long startTime = System.currentTimeMillis(); - long endTime; RemoteIterator locatedFileStatus; Path path = OBSCommonUtils.qualify(this, f); @@ -1959,26 +2102,13 @@ public RemoteIterator listFiles(final Path f, final boolean r // lookup dir triggers existence check final FileStatus fileStatus; try { - fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, path); - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_FILES, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, path); + } catch (OBSFileConflictException e) { throw new AccessControlException(e); } if (fileStatus.isFile()) { locatedFileStatus = new OBSListing.SingleStatusRemoteIterator( OBSCommonUtils.toLocatedFileStatus(this, fileStatus)); - - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_FILES, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } // simple case: File LOG.debug("Path is a file"); return locatedFileStatus; @@ -1992,22 +2122,9 @@ public RemoteIterator listFiles(final Path f, final boolean r obsListing.createFileStatusListingIterator(path, OBSCommonUtils.createListObjectsRequest(this, key, delimiter), org.apache.hadoop.fs.obs.OBSListing.ACCEPT_ALL, new OBSListing.AcceptFilesOnly(path))); - - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_FILES, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return locatedFileStatus; } } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_FILES, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw OBSCommonUtils.translateException("listFiles", path, e); } } @@ -2049,21 +2166,13 @@ public RemoteIterator listLocatedStatus(final Path f, final P checkPermission(f,AccessType.READ); Path path = OBSCommonUtils.qualify(this, f); LOG.debug("listLocatedStatus({}, {}", path, filter); - long startTime = System.currentTimeMillis(); - long endTime; RemoteIterator locatedFileStatusRemoteList; try { // lookup dir triggers existence check final FileStatus fileStatus; try { - fileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(this, path); - } catch (FileConflictException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + fileStatus = OBSCommonUtils.getFileStatusWithRetry(this, path); + } catch (OBSFileConflictException e) { throw new AccessControlException(e); } @@ -2072,12 +2181,6 @@ public RemoteIterator listLocatedStatus(final Path f, final P LOG.debug("Path is a file"); locatedFileStatusRemoteList = new OBSListing.SingleStatusRemoteIterator( filter.accept(path) ? OBSCommonUtils.toLocatedFileStatus(this, fileStatus) : null); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } return locatedFileStatusRemoteList; } else { // directory: trigger a lookup @@ -2085,22 +2188,10 @@ public RemoteIterator listLocatedStatus(final Path f, final P locatedFileStatusRemoteList = obsListing.createLocatedFileStatusIterator( obsListing.createFileStatusListingIterator(path, OBSCommonUtils.createListObjectsRequest(this, key, "/"), filter, - new OBSListing.AcceptAllButSelfAndS3nDirs(path))); - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } + new OBSListing.AcceptAllButSelfAndOBSDirs(path))); return locatedFileStatusRemoteList; } } catch (ObsException e) { - endTime = System.currentTimeMillis(); - if (getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.LIST_LOCATED_STS, false, endTime - startTime); - OBSCommonUtils.setMetricsInfo(this, record); - } throw OBSCommonUtils.translateException("listLocatedStatus", path, e); } } @@ -2136,4 +2227,52 @@ public boolean getMetricSwitch() { int getInvokeCountThreshold() { return invokeCountThreshold; } + + public TrafficStatistics getTrafficStatistics() { + return trafficStatistics; + } + + /** + * Get disguise permission mode support stat. + * + * @return is disguise permission mode supported + */ + boolean supportDisguisePermissionsMode() { + return OBSConstants.PERMISSIONS_MODE_DISGUISE.equals(permissionsMode) && enablePosix; + } + + /** + * Set permission for given file. + * + * @param f the file to set permission info + * @param permission file's permission + * @throws IOException If an I/O error occurred + */ + @Override + public void setPermission(final Path f, final FsPermission permission) throws IOException { + checkOpen(); + checkPermission(f, AccessType.WRITE); + if (supportDisguisePermissionsMode()) { + LOG.debug("Set file {} permission to {}", f, permission); + OBSPosixBucketUtils.fsSetPermission(this, f, permission); + } + } + + /** + * Set owner and group for given file. + * + * @param f the file to set owner and group info + * @param username file's owner + * @param groupname file's group + * @throws IOException If an I/O error occurred + */ + @Override + public void setOwner(final Path f, final String username, final String groupname) throws IOException { + checkOpen(); + checkPermission(f, AccessType.WRITE); + if (supportDisguisePermissionsMode()) { + LOG.debug("Set file {} owner to {} and group to {}", f , username, groupname); + OBSPosixBucketUtils.fsSetOwner(this, f, username, groupname); + } + } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java index 39abc1b..10b804a 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSFsDFSListing.java @@ -51,7 +51,7 @@ static String fsDFSListNextBatch(final OBSFileSystem owner, final Stack resultQueue, fin continue; } - for (ObsObject extenedCommonPrefixes : oneLevelObjectListing.getExtenedCommonPrefixes()) { + for (ObsObject extenedCommonPrefixes : oneLevelObjectListing.getExtendCommonPrefixes()) { if (extenedCommonPrefixes.getObjectKey().equals(oneLevelListRequests.get(i).getPrefix())) { // skip prefix itself continue; @@ -285,9 +285,9 @@ static int fetchListResultLocally(final Queue resultQueue, final int while (!resultQueue.isEmpty() && resultNum < maxKeyNum) { ListEntity listEntity = resultQueue.poll(); if (listEntity.getType() == ListEntityType.LIST_TAIL) { - throw new RuntimeException("cannot put list tail (" + listEntity + ") into result queue"); + throw new IllegalStateException("cannot put list tail (" + listEntity + ") into result queue"); } else if (listEntity.getType() == ListEntityType.COMMON_PREFIX) { - throw new RuntimeException("cannot put common prefix (" + listEntity + ") into result queue"); + throw new IllegalStateException("cannot put common prefix (" + listEntity + ") into result queue"); } else { objectSummaries.add(listEntity.getObjectSummary()); increaseLevelStats(levelStatsList, listEntity.getLevel(), diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java index 069cd7f..d84c837 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSHDFSFileSystem.java @@ -112,7 +112,7 @@ private List> getMountList(final Configuration config, fina if (pathPrefix.endsWith(Path.SEPARATOR) && !pathPrefix.trim().equals(Path.SEPARATOR)) { pathPrefix = pathPrefix.substring(0, pathPrefix.length() - 1); } - mountList.add(new Pair(pathPrefix, si.getValue())); + mountList.add(new Pair<>(pathPrefix, si.getValue())); } } } @@ -264,9 +264,10 @@ private Path transferToWrappedPath(Path path, TransferedPath newPath) { public void initialize(URI theUri, Configuration conf) throws IOException { this.wrapperConf = new Configuration(conf); wrapperConf.set("fs.hdfs.impl", DistributedFileSystem.class.getName()); + wrapperConf.setBoolean("fs.hdfs.impl.disable.cache", true); super.initialize(theUri, conf); - underHDFS = (DistributedFileSystem) FileSystem.newInstance(theUri, wrapperConf); + underHDFS = (DistributedFileSystem) FileSystem.get(theUri, wrapperConf); final String authority = theUri.getAuthority(); mountMap = new HashMap<>(); @@ -531,9 +532,9 @@ public FileStatus[] listStatus(Path p) throws IOException { @Override public RemoteIterator listFiles(Path f, boolean recursive) - throws FileNotFoundException, IOException { + throws IOException { TransferedPath newPath = transferToNewPath(f); - return new WrappedRemoteIterator(newPath.getFS().listFiles(newPath.toPath(), recursive), + return new WrappedRemoteIterator<>(newPath.getFS().listFiles(newPath.toPath(), recursive), fileStatus -> { Path originPath = transferToWrappedPath(fileStatus.getPath(), newPath); fileStatus.setPath(originPath); @@ -544,7 +545,7 @@ public RemoteIterator listFiles(Path f, boolean recursive) @Override public RemoteIterator listLocatedStatus(Path p, final PathFilter filter) throws IOException { TransferedPath newPath = transferToNewPath(p); - return new WrappedRemoteIterator(newPath.getFS().listLocatedStatus(newPath.toPath()), + return new WrappedRemoteIterator<>(newPath.getFS().listLocatedStatus(newPath.toPath()), fileStatus -> { Path originPath = transferToWrappedPath(fileStatus.getPath(), newPath); fileStatus.setPath(originPath); @@ -555,7 +556,7 @@ public RemoteIterator listLocatedStatus(Path p, final PathFil @Override public RemoteIterator listStatusIterator(Path p) throws IOException { TransferedPath newPath = transferToNewPath(p); - return new WrappedRemoteIterator(newPath.getFS().listStatusIterator(newPath.toPath()), + return new WrappedRemoteIterator<>(newPath.getFS().listStatusIterator(newPath.toPath()), fileStatus -> { Path originPath = transferToWrappedPath(fileStatus.getPath(), newPath); fileStatus.setPath(originPath); @@ -617,7 +618,7 @@ public FsStatus getStatus(Path p) throws IOException { @Override public RemoteIterator listCorruptFileBlocks(Path path) throws IOException { TransferedPath newPath = transferToNewPath(path); - return new WrappedRemoteIterator(newPath.getFS().listCorruptFileBlocks(newPath.toPath()), + return new WrappedRemoteIterator<>(newPath.getFS().listCorruptFileBlocks(newPath.toPath()), p -> transferToWrappedPath(p, newPath)); } @@ -802,7 +803,7 @@ public Path getTrashRoot(Path path) { } } - private static class WrappedRemoteIterator implements RemoteIterator { + private static class WrappedRemoteIterator implements RemoteIterator { private final RemoteIterator origin; @@ -819,7 +820,7 @@ public boolean hasNext() throws IOException { } @Override - public Object next() throws IOException { + public T next() throws IOException { return convertFunc.apply(origin.next()); } } @@ -919,6 +920,8 @@ public Path toPath() { } static class UncheckException extends RuntimeException { + static final long serialVersionUID = 5746198432791324945L; + public UncheckException(IOException origin) { super(origin); } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java index 2d56d5f..c78f697 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIOException.java @@ -18,7 +18,6 @@ package org.apache.hadoop.fs.obs; -import com.google.common.base.Preconditions; import com.obs.services.exception.ObsException; import java.io.IOException; @@ -26,27 +25,25 @@ /** * IOException equivalent to {@link ObsException}. */ -public class OBSIOException extends IOException { +public class OBSIOException extends IOException implements WithErrCode { private static final long serialVersionUID = -1582681108285856259L; - - /** - * Peration message. - */ - private final String operation; + private String errCode; OBSIOException(final String operationMsg, final ObsException cause) { - super(cause); - Preconditions.checkArgument(operationMsg != null, "Null 'operation' argument"); - Preconditions.checkArgument(cause != null, "Null 'cause' argument"); - this.operation = operationMsg; + super(operationMsg,cause); } - public ObsException getCause() { + public synchronized ObsException getCause() { return (ObsException) super.getCause(); } + public void setErrCode(String errCode) { + this.errCode = errCode; + } + @Override - public String getMessage() { - return operation + ": " + getCause().getErrorMessage() + ", detailMessage: " + super.getMessage(); + public String getErrCode() { + return this.errCode; } + } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIllegalArgumentException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIllegalArgumentException.java new file mode 100644 index 0000000..e97b36c --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSIllegalArgumentException.java @@ -0,0 +1,27 @@ +package org.apache.hadoop.fs.obs; + +import java.io.IOException; + +/** + * description + * + * @since 2022-02-07 + */ +public class OBSIllegalArgumentException extends IOException implements WithErrCode { + + private static final long serialVersionUID = 2188013092663783231L; + private String errCode; + + OBSIllegalArgumentException(final String message) { + super(message); + } + + public void setErrCode(String errCode) { + this.errCode = errCode; + } + + @Override + public String getErrCode() { + return this.errCode; + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInvoker.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInvoker.java new file mode 100644 index 0000000..3933828 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSInvoker.java @@ -0,0 +1,115 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import com.obs.services.exception.ObsException; + +import org.apache.hadoop.io.retry.RetryPolicy; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InterruptedIOException; + +public class OBSInvoker { + private static final Logger LOG = LoggerFactory.getLogger(OBSInvoker.class); + + public static final RetryCallback LOG_EVENT = new RetryCallback() { + @Override + public void onFailure(String text, + IOException exception, + int retries, + boolean idempotent) { + LOG.debug("retry #{}, {}", retries, exception); + } + }; + + private final RetryPolicyWithMaxTime retryPolicy; + private final RetryCallback retryCallback; + private final OBSFileSystem fs; + + public OBSInvoker(OBSFileSystem fs, RetryPolicyWithMaxTime retryPolicy, RetryCallback callback) { + this.retryPolicy = retryPolicy; + this.retryCallback = callback; + this.fs = fs; + } + + public T retryByMaxTime(OBSOperateAction action, String path, OBSCallable operation, boolean idempotent) + throws IOException { + return retryByMaxTime(action, path, operation, idempotent, retryCallback); + } + + + public T retryByMaxTime(OBSOperateAction action, String path, OBSCallable operation, boolean idempotent, RetryCallback retrying) + throws IOException { + int retryCount = 0; + long startTime = System.currentTimeMillis(); + IOException translated = null; + RetryPolicy.RetryAction retryAction; + boolean shouldRetry; + do { + try { + return operation.call(); + } catch (IOException e) { + translated = e; + } catch (ObsException e) { + translated = OBSCommonUtils.translateException(action.toString(), path, e); + } + + OBSCommonUtils.putQosMetric(fs, action, translated); + + String text = action + " on " + path; + + try { + retryAction = retryPolicy.shouldRetryByMaxTime(startTime, translated, retryCount, 0, + idempotent); + shouldRetry = retryAction.action.equals( + RetryPolicy.RetryAction.RETRY.action); + if (shouldRetry) { + retryCount++; + retrying.onFailure(text, translated, retryCount, idempotent); + Thread.sleep(retryAction.delayMillis); + } + } catch (InterruptedException e) { + translated = new InterruptedIOException(text + ",interrupted in retry process"); + translated.initCause(e); + shouldRetry = false; + Thread.currentThread().interrupt(); + } catch (Exception e) { + translated = new IOException(text + ",unexpect exception in retry process", e); + shouldRetry = false; + } + } while (shouldRetry); + + if (retryCount != 0) { + LOG.error("retry {} times fail: {}", retryCount, translated.toString()); + } + throw translated; + } + + + @FunctionalInterface + public interface RetryCallback { + void onFailure( + String text, + IOException exception, + int retries, + boolean idempotent); + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java index 3b84a49..308d697 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSListing.java @@ -41,29 +41,20 @@ * OBS listing implementation. */ class OBSListing { - /** - * A Path filter which accepts all filenames. - */ static final PathFilter ACCEPT_ALL = new PathFilter() { @Override - public boolean accept(final Path file) { - return true; + public String toString() { + return "ACCEPT_ALL"; } @Override - public String toString() { - return "ACCEPT_ALL"; + public boolean accept(final Path file) { + return true; } }; - /** - * Class logger. - */ private static final Logger LOG = LoggerFactory.getLogger(OBSListing.class); - /** - * OBS File System instance. - */ private final OBSFileSystem owner; OBSListing(final OBSFileSystem ownerFS) { @@ -98,30 +89,27 @@ LocatedFileStatusIterator createLocatedFileStatusIterator(final RemoteIterator { requestNextBatch(); } - /** - * Report whether or not there is new data available. If there is data - * in the local filtered list, return true. Else: request more data util - * that condition is met, or there is no more remote listing data. - * - * @return true if a call to {@link #next()} will succeed. - * @throws IOException on any failure to request next batch - */ @Override public boolean hasNext() throws IOException { return statusBatchIterator.hasNext() || requestNextBatch(); @@ -380,21 +360,13 @@ public FileStatus next() throws IOException { * @throws IOException IO problems */ private boolean requestNextBatch() throws IOException { - // look for more object listing batches being available while (source.hasNext()) { - // if available, retrieve it and build the next status if (buildNextStatusBatch(source.next())) { - // this batch successfully generated entries matching - // the filters/acceptors; - // declare that the request was successful return true; } else { LOG.debug("All entries in batch were filtered...continuing"); } } - // if this code is reached, it means that all remaining - // object lists have been retrieved, and there are no new entries - // to return. return false; } @@ -403,8 +375,9 @@ private boolean requestNextBatch() throws IOException { * * @param objects the next object listing * @return true if this added any entries after filtering + * @throws IOException If an I/O error occurred */ - private boolean buildNextStatusBatch(final ObjectListing objects) { + private boolean buildNextStatusBatch(final ObjectListing objects) throws IOException { // counters for debug logs int added = 0; int ignored = 0; @@ -420,7 +393,7 @@ private boolean buildNextStatusBatch(final ObjectListing objects) { // Skip over keys that are ourselves and old OBS _$folder$ files if (acceptor.accept(keyPath, summary) && filter.accept(keyPath)) { FileStatus status = OBSCommonUtils.createFileStatus(keyPath, summary, - owner.getDefaultBlockSize(keyPath), owner.getShortUserName()); + owner.getDefaultBlockSize(keyPath), owner); LOG.debug("Adding: {}", status); stats.add(status); added++; @@ -431,31 +404,31 @@ private boolean buildNextStatusBatch(final ObjectListing objects) { } // prefixes: always directories - for (ObsObject prefix : objects.getExtenedCommonPrefixes()) { + for (ObsObject prefix : objects.getExtendCommonPrefixes()) { String key = prefix.getObjectKey(); Path keyPath = OBSCommonUtils.keyToQualifiedPath(owner, key); if (acceptor.accept(keyPath, key) && filter.accept(keyPath)) { long lastModified = prefix.getMetadata().getLastModified() == null ? System.currentTimeMillis() : OBSCommonUtils.dateToLong(prefix.getMetadata().getLastModified()); - FileStatus status = new OBSFileStatus(keyPath, lastModified, lastModified, + OBSFileStatus status = new OBSFileStatus(keyPath, lastModified, lastModified, owner.getShortUserName()); - LOG.debug("Adding directory: {}", status); + if (owner.supportDisguisePermissionsMode()) { + OBSCommonUtils.setAccessControlAttrForFileStatus(owner, status, + OBSCommonUtils.getObjectMetadata(owner, key)); + } added++; stats.add(status); } else { - LOG.debug("Ignoring directory: {}", keyPath); ignored++; } } - - // finish up batchSize = stats.size(); statusBatchIterator = stats.listIterator(); - boolean hasNext = statusBatchIterator.hasNext(); - LOG.debug("Added {} entries; ignored {}; hasNext={}; hasMoreObjects={}", added, ignored, hasNext, - objects.isTruncated()); - return hasNext; + boolean ret = statusBatchIterator.hasNext(); + LOG.debug("Added {}; ignored {}; hasNext={}; hasMoreObjects={}", + added, ignored, ret, objects.isTruncated()); + return ret; } /** @@ -531,25 +504,11 @@ class ObjectListingIterator implements RemoteIterator { this.objects = OBSCommonUtils.listObjects(owner, request); } - /** - * Declare that the iterator has data if it is either is the initial - * iteration or it is a later one and the last listing obtained was - * incomplete. - */ @Override public boolean hasNext() { return firstListing || objects.isTruncated(); } - /** - * Ask for the next listing. For the first invocation, this returns the - * initial set, with no remote IO. For later requests, OBS will be - * queried, hence the calls may block or fail. - * - * @return the next object listing. - * @throws IOException if a query made of OBS fails. - * @throws NoSuchElementException if there is no more data to list. - */ @Override public ObjectListing next() throws IOException { if (firstListing) { diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLocalDirAllocator.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLocalDirAllocator.java deleted file mode 100644 index a2c21be..0000000 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLocalDirAllocator.java +++ /dev/null @@ -1,601 +0,0 @@ -/** - * Licensed to the Apache Software Foundation (ASF) under one - * or more contributor license agreements. See the NOTICE file - * distributed with this work for additional information - * regarding copyright ownership. The ASF licenses this file - * to you under the Apache License, Version 2.0 (the - * "License"); you may not use this file except in compliance - * with the License. You may obtain a copy of the License at - *

- * http://www.apache.org/licenses/LICENSE-2.0 - *

- * Unless required by applicable law or agreed to in writing, software - * distributed under the License is distributed on an "AS IS" BASIS, - * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. - * See the License for the specific language governing permissions and - * limitations under the License. - */ - -package org.apache.hadoop.fs.obs; - -import org.apache.commons.logging.Log; -import org.apache.commons.logging.LogFactory; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.conf.Configuration; -import org.apache.hadoop.fs.DF; -import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.Path; -import org.apache.hadoop.util.DiskChecker; -import org.apache.hadoop.util.DiskChecker.DiskErrorException; -import org.apache.hadoop.util.StringUtils; - -import java.io.File; -import java.io.IOException; -import java.security.SecureRandom; -import java.util.ArrayList; -import java.util.Iterator; -import java.util.Map; -import java.util.NoSuchElementException; -import java.util.Random; -import java.util.TreeMap; -import java.util.concurrent.atomic.AtomicInteger; -import java.util.concurrent.atomic.AtomicReference; - -/** - * this class copy from hadoop-common module. - * in flink or other long run application,OBSAllocatorPerContext#createTmpFileForWrite method invoke - * File.deleteOnExit,and will cause JVM OOM. - *

- * JDK-4872014:The File.deleteOnExit API is flawed and causes JVM crashes - * on long running servers (or even short running if the API is used enough). See bug 4513817. - */ - -public class OBSLocalDirAllocator { - - //A Map from the config item names like "mapred.local.dir" - //to the instance of the AllocatorPerContext. This - //is a static object to make sure there exists exactly one instance per JVM - private static Map contexts = new TreeMap(); - - private String contextCfgItemName; - - /** - * Used when size of file to be allocated is unknown. - */ - public static final int SIZE_UNKNOWN = -1; - - /** - * Create an allocator object - * - * @param contextCfgItemName - */ - public OBSLocalDirAllocator(String contextCfgItemName) { - this.contextCfgItemName = contextCfgItemName; - } - - /** - * This method must be used to obtain the dir allocation context for a - * particular value of the context name. The context name must be an item - * defined in the Configuration object for which we want to control the - * dir allocations (e.g., mapred.local.dir). The method will - * create a context for that name if it doesn't already exist. - */ - private OBSAllocatorPerContext obtainContext(String contextCfgItemName) { - synchronized (contexts) { - OBSAllocatorPerContext l = contexts.get(contextCfgItemName); - if (l == null) { - contexts.put(contextCfgItemName, l = new OBSAllocatorPerContext(contextCfgItemName)); - } - return l; - } - } - - /** - * Get a path from the local FS. This method should be used if the size of - * the file is not known apriori. We go round-robin over the set of disks - * (via the configured dirs) and return the first complete path where - * we could create the parent directory of the passed path. - * - * @param pathStr the requested path (this will be created on the first - * available disk) - * @param conf the Configuration object - * @return the complete path to the file on a local disk - * @throws IOException - */ - public Path getLocalPathForWrite(String pathStr, Configuration conf) throws IOException { - return getLocalPathForWrite(pathStr, SIZE_UNKNOWN, conf); - } - - /** - * Get a path from the local FS. Pass size as - * SIZE_UNKNOWN if not known apriori. We - * round-robin over the set of disks (via the configured dirs) and return - * the first complete path which has enough space - * - * @param pathStr the requested path (this will be created on the first - * available disk) - * @param size the size of the file that is going to be written - * @param conf the Configuration object - * @return the complete path to the file on a local disk - * @throws IOException - */ - public Path getLocalPathForWrite(String pathStr, long size, Configuration conf) throws IOException { - return getLocalPathForWrite(pathStr, size, conf, true); - } - - /** - * Get a path from the local FS. Pass size as - * SIZE_UNKNOWN if not known apriori. We - * round-robin over the set of disks (via the configured dirs) and return - * the first complete path which has enough space - * - * @param pathStr the requested path (this will be created on the first - * available disk) - * @param size the size of the file that is going to be written - * @param conf the Configuration object - * @param checkWrite ensure that the path is writable - * @return the complete path to the file on a local disk - * @throws IOException - */ - public Path getLocalPathForWrite(String pathStr, long size, Configuration conf, boolean checkWrite) - throws IOException { - OBSAllocatorPerContext context = obtainContext(contextCfgItemName); - return context.getLocalPathForWrite(pathStr, size, conf, checkWrite); - } - - /** - * Get a path from the local FS for reading. We search through all the - * configured dirs for the file's existence and return the complete - * path to the file when we find one - * - * @param pathStr the requested file (this will be searched) - * @param conf the Configuration object - * @return the complete path to the file on a local disk - * @throws IOException - */ - public Path getLocalPathToRead(String pathStr, Configuration conf) throws IOException { - OBSAllocatorPerContext context = obtainContext(contextCfgItemName); - return context.getLocalPathToRead(pathStr, conf); - } - - /** - * Get all of the paths that currently exist in the working directories. - * - * @param pathStr the path underneath the roots - * @param conf the configuration to look up the roots in - * @return all of the paths that exist under any of the roots - * @throws IOException - */ - public Iterable getAllLocalPathsToRead(String pathStr, Configuration conf) throws IOException { - OBSAllocatorPerContext context; - synchronized (this) { - context = obtainContext(contextCfgItemName); - } - return context.getAllLocalPathsToRead(pathStr, conf); - } - - /** - * Creates a temporary file in the local FS. Pass size as -1 if not known - * apriori. We round-robin over the set of disks (via the configured dirs) - * and select the first complete path which has enough space. A file is - * created on this directory. The file is guaranteed to go away when the - * JVM exits. - * - * @param pathStr prefix for the temporary file - * @param size the size of the file that is going to be written - * @param conf the Configuration object - * @return a unique temporary file - * @throws IOException - */ - public File createTmpFileForWrite(String pathStr, long size, Configuration conf) throws IOException { - OBSAllocatorPerContext context = obtainContext(contextCfgItemName); - return context.createTmpFileForWrite(pathStr, size, conf); - } - - /** - * Method to check whether a context is valid - * - * @param contextCfgItemName - * @return true/false - */ - public static boolean isContextValid(String contextCfgItemName) { - synchronized (contexts) { - return contexts.containsKey(contextCfgItemName); - } - } - - /** - * Removes the context from the context config items - * - * @param contextCfgItemName - */ - @Deprecated - @InterfaceAudience.LimitedPrivate({"MapReduce"}) - public static void removeContext(String contextCfgItemName) { - synchronized (contexts) { - contexts.remove(contextCfgItemName); - } - } - - /** - * We search through all the configured dirs for the file's existence - * and return true when we find - * - * @param pathStr the requested file (this will be searched) - * @param conf the Configuration object - * @return true if files exist. false otherwise - * @throws IOException - */ - public boolean ifExists(String pathStr, Configuration conf) { - OBSAllocatorPerContext context = obtainContext(contextCfgItemName); - return context.ifExists(pathStr, conf); - } - - /** - * Get the current directory index for the given configuration item. - * - * @return the current directory index for the given configuration item. - */ - int getCurrentDirectoryIndex() { - OBSAllocatorPerContext context = obtainContext(contextCfgItemName); - return context.getCurrentDirectoryIndex(); - } - - private static class OBSAllocatorPerContext { - - private final Log log = LogFactory.getLog(OBSAllocatorPerContext.class); - - private Random dirIndexRandomizer = new Random(); - - private String contextCfgItemName; - - // NOTE: the context must be accessed via a local reference as it - // may be updated at any time to reference a different context - private AtomicReference currentContext; - - private static class Context { - private AtomicInteger dirNumLastAccessed = new AtomicInteger(0); - - private FileSystem localFS; - - private DF[] dirDF; - - private Path[] localDirs; - - private String savedLocalDirs; - - public int getAndIncrDirNumLastAccessed() { - return getAndIncrDirNumLastAccessed(1); - } - - public int getAndIncrDirNumLastAccessed(int delta) { - if (localDirs.length < 2 || delta == 0) { - return dirNumLastAccessed.get(); - } - int oldval; - int newval; - do { - oldval = dirNumLastAccessed.get(); - newval = (oldval + delta) % localDirs.length; - } while (!dirNumLastAccessed.compareAndSet(oldval, newval)); - return oldval; - } - } - - public OBSAllocatorPerContext(String contextCfgItemName) { - this.contextCfgItemName = contextCfgItemName; - this.currentContext = new AtomicReference(new Context()); - } - - /** - * This method gets called everytime before any read/write to make sure - * that any change to localDirs is reflected immediately. - */ - private Context confChanged(Configuration conf) throws IOException { - Context ctx = currentContext.get(); - String newLocalDirs = conf.get(contextCfgItemName); - if (null == newLocalDirs) { - throw new IOException(contextCfgItemName + " not configured"); - } - if (!newLocalDirs.equals(ctx.savedLocalDirs)) { - ctx = confChanged2(conf, newLocalDirs); - } - - return ctx; - } - - private Context confChanged2(Configuration conf, String newLocalDirs) throws IOException { - Context ctx = new Context(); - String[] dirStrings = StringUtils.getTrimmedStrings(newLocalDirs); - ctx.localFS = FileSystem.getLocal(conf); - int numDirs = dirStrings.length; - ArrayList dirs = new ArrayList(numDirs); - ArrayList dfList = new ArrayList(numDirs); - for (int i = 0; i < numDirs; i++) { - try { - // filter problematic directories - Path tmpDir = new Path(dirStrings[i]); - if (ctx.localFS.mkdirs(tmpDir) || ctx.localFS.exists(tmpDir)) { - try { - File tmpFile = tmpDir.isAbsolute() - ? new File(ctx.localFS.makeQualified(tmpDir).toUri()) - : new File(dirStrings[i]); - - DiskChecker.checkDir(tmpFile); - dirs.add(new Path(tmpFile.getPath())); - dfList.add(new DF(tmpFile, 30000)); - } catch (DiskErrorException de) { - log.warn(dirStrings[i] + " is not writable\n", de); - } - } else { - log.warn("Failed to create " + dirStrings[i]); - } - } catch (IOException ie) { - log.warn("Failed to create " + dirStrings[i] + ": " + ie.getMessage() + "\n", ie); - } //ignore - } - ctx.localDirs = dirs.toArray(new Path[dirs.size()]); - ctx.dirDF = dfList.toArray(new DF[dirs.size()]); - ctx.savedLocalDirs = newLocalDirs; - - if (dirs.size() > 0) { - // randomize the first disk picked in the round-robin selection - ctx.dirNumLastAccessed.set(dirIndexRandomizer.nextInt(dirs.size())); - } - - currentContext.set(ctx); - - return ctx; - } - - private Path createPath(Path dir, String path, boolean checkWrite) throws IOException { - Path file = new Path(dir, path); - if (checkWrite) { - //check whether we are able to create a directory here. If the disk - //happens to be RDONLY we will fail - try { - DiskChecker.checkDir(new File(file.getParent().toUri().getPath())); - return file; - } catch (DiskErrorException d) { - log.warn("Disk Error Exception: ", d); - return null; - } - } - return file; - } - - /** - * Get the current directory index. - * - * @return the current directory index. - */ - int getCurrentDirectoryIndex() { - return currentContext.get().dirNumLastAccessed.get(); - } - - /** - * Get a path from the local FS. If size is known, we go - * round-robin over the set of disks (via the configured dirs) and return - * the first complete path which has enough space. - *

- * If size is not known, use roulette selection -- pick directories - * with probability proportional to their available space. - */ - public Path getLocalPathForWrite(String pathStr, long size, Configuration conf, boolean checkWrite) - throws IOException { - Context ctx = confChanged(conf); - int numDirs = ctx.localDirs.length; - int numDirsSearched = 0; - //remove the leading slash from the path (to make sure that the uri - //resolution results in a valid path on the dir being checked) - if (pathStr.startsWith("/")) { - pathStr = pathStr.substring(1); - } - Path returnPath = null; - - if (size == SIZE_UNKNOWN) { //do roulette selection: pick dir with probability - //proportional to available size - long[] availableOnDisk = new long[ctx.dirDF.length]; - long totalAvailable = 0; - - //build the "roulette wheel" - for (int i = 0; i < ctx.dirDF.length; ++i) { - availableOnDisk[i] = ctx.dirDF[i].getAvailable(); - totalAvailable += availableOnDisk[i]; - } - - if (totalAvailable == 0) { - throw new DiskErrorException("No space available in any of the local directories."); - } - - // Keep rolling the wheel till we get a valid path - SecureRandom r = new SecureRandom(); - // Random r = new Random(); - while (numDirsSearched < numDirs && returnPath == null) { - long randomPosition = (r.nextLong() >>> 1) % totalAvailable; - int dir = 0; - while (randomPosition > availableOnDisk[dir]) { - randomPosition -= availableOnDisk[dir]; - dir++; - } - ctx.dirNumLastAccessed.set(dir); - returnPath = createPath(ctx.localDirs[dir], pathStr, checkWrite); - if (returnPath == null) { - totalAvailable -= availableOnDisk[dir]; - availableOnDisk[dir] = 0; // skip this disk - numDirsSearched++; - } - } - } else { - int dirNum = ctx.getAndIncrDirNumLastAccessed(); - while (numDirsSearched < numDirs) { - long capacity = ctx.dirDF[dirNum].getAvailable(); - if (capacity > size) { - returnPath = createPath(ctx.localDirs[dirNum], pathStr, checkWrite); - if (returnPath != null) { - ctx.getAndIncrDirNumLastAccessed(numDirsSearched); - break; - } - } - dirNum++; - dirNum = dirNum % numDirs; - numDirsSearched++; - } - } - if (returnPath != null) { - return returnPath; - } - - //no path found - throw new DiskErrorException("Could not find any valid local " + "directory for " + pathStr); - } - - /** - * Creates a file on the local FS. Pass size as - * {@link OBSLocalDirAllocator#SIZE_UNKNOWN} if not known apriori. We - * round-robin over the set of disks (via the configured dirs) and return - * a file on the first path which has enough space. The file is guaranteed - * to go away when the JVM exits. - */ - public File createTmpFileForWrite(String pathStr, long size, Configuration conf) throws IOException { - - // find an appropriate directory - Path path = getLocalPathForWrite(pathStr, size, conf, true); - File dir = new File(path.getParent().toUri().getPath()); - String prefix = path.getName(); - - // create a temp file on this directory - File result = File.createTempFile(prefix, null, dir); - - return result; - } - - /** - * Get a path from the local FS for reading. We search through all the - * configured dirs for the file's existence and return the complete - * path to the file when we find one - */ - public Path getLocalPathToRead(String pathStr, Configuration conf) throws IOException { - Context ctx = confChanged(conf); - int numDirs = ctx.localDirs.length; - int numDirsSearched = 0; - //remove the leading slash from the path (to make sure that the uri - //resolution results in a valid path on the dir being checked) - if (pathStr.startsWith("/")) { - pathStr = pathStr.substring(1); - } - while (numDirsSearched < numDirs) { - Path file = new Path(ctx.localDirs[numDirsSearched], pathStr); - if (ctx.localFS.exists(file)) { - return file; - } - numDirsSearched++; - } - - //no path found - throw new DiskErrorException( - "Could not find " + pathStr + " in any of" + " the configured local directories"); - } - - private static class PathIterator implements Iterator, Iterable { - private final FileSystem fs; - - private final String pathStr; - - private int i = 0; - - private final Path[] rootDirs; - - private Path next = null; - - private PathIterator(FileSystem fs, String pathStr, Path[] rootDirs) throws IOException { - this.fs = fs; - this.pathStr = pathStr; - this.rootDirs = rootDirs; - advance(); - } - - @Override - public boolean hasNext() { - return next != null; - } - - private void advance() throws IOException { - while (i < rootDirs.length) { - next = new Path(rootDirs[i++], pathStr); - if (fs.exists(next)) { - return; - } - } - next = null; - } - - @Override - public Path next() { - final Path result = next; - try { - advance(); - } catch (IOException ie) { - throw new RuntimeException("Can't check existence of " + next, ie); - } - if (result == null) { - throw new NoSuchElementException(); - } - return result; - } - - @Override - public void remove() { - throw new UnsupportedOperationException("read only iterator"); - } - - @Override - public Iterator iterator() { - return this; - } - } - - /** - * Get all of the paths that currently exist in the working directories. - * - * @param pathStr the path underneath the roots - * @param conf the configuration to look up the roots in - * @return all of the paths that exist under any of the roots - * @throws IOException - */ - Iterable getAllLocalPathsToRead(String pathStr, Configuration conf) throws IOException { - Context ctx = confChanged(conf); - if (pathStr.startsWith("/")) { - pathStr = pathStr.substring(1); - } - return new PathIterator(ctx.localFS, pathStr, ctx.localDirs); - } - - /** - * We search through all the configured dirs for the file's existence - * and return true when we find one - */ - public boolean ifExists(String pathStr, Configuration conf) { - Context ctx = currentContext.get(); - try { - int numDirs = ctx.localDirs.length; - int numDirsSearched = 0; - //remove the leading slash from the path (to make sure that the uri - //resolution results in a valid path on the dir being checked) - if (pathStr.startsWith("/")) { - pathStr = pathStr.substring(1); - } - while (numDirsSearched < numDirs) { - Path file = new Path(ctx.localDirs[numDirsSearched], pathStr); - if (ctx.localFS.exists(file)) { - return true; - } - numDirsSearched++; - } - } catch (IOException e) { - log.error(e); - } - return false; - } - } -} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java index f7d31a4..7e9a14a 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSLoginHelper.java @@ -18,9 +18,6 @@ package org.apache.hadoop.fs.obs; -import static org.apache.commons.lang.StringUtils.equalsIgnoreCase; - -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.Path; @@ -151,7 +148,7 @@ public static Login extractLoginDetails(final URI name) { } } catch (UnsupportedEncodingException e) { // this should never happen; translate it if it does. - throw new RuntimeException(e); + throw new IllegalArgumentException(e); } } @@ -216,13 +213,13 @@ public static void checkPath(final Configuration conf, final URI fsUri, final Pa URI thisUri = canonicalizeUri(fsUri, defaultPort); String thisScheme = thisUri.getScheme(); // hostname and scheme are not case sensitive in these checks - if (equalsIgnoreCase(thisScheme, thatScheme)) { // schemes match + if (OBSCommonUtils.stringEqualsIgnoreCase(thisScheme, thatScheme)) { // schemes match String thisHost = thisUri.getHost(); String thatHost = pathUri.getHost(); if (thatHost == null && // path's host is null thisHost != null) { // fs has a host URI defaultUri = FileSystem.getDefaultUri(conf); - if (equalsIgnoreCase(thisScheme, defaultUri.getScheme())) { + if (OBSCommonUtils.stringEqualsIgnoreCase(thisScheme, defaultUri.getScheme())) { pathUri = defaultUri; // schemes match, so use this uri instead } else { pathUri = null; // can't determine auth of the path @@ -232,7 +229,7 @@ public static void checkPath(final Configuration conf, final URI fsUri, final Pa // canonicalize uri before comparing with this fs pathUri = canonicalizeUri(pathUri, defaultPort); thatHost = pathUri.getHost(); - if (equalsIgnoreCase(thisHost, thatHost)) { + if (OBSCommonUtils.stringEqualsIgnoreCase(thisHost, thatHost)) { return; } } @@ -289,7 +286,7 @@ public static class Login { * @return true if the username is defined (not null, not empty). */ public boolean hasLogin() { - return StringUtils.isNotEmpty(user); + return OBSCommonUtils.isStringNotEmpty(user); } /** diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSMethodNotAllowedException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSMethodNotAllowedException.java new file mode 100644 index 0000000..b45c522 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSMethodNotAllowedException.java @@ -0,0 +1,22 @@ +package org.apache.hadoop.fs.obs; + +import java.io.IOException; + +public class OBSMethodNotAllowedException extends IOException implements WithErrCode { + + private static final long serialVersionUID = 2461327923217975442L; + private String errCode; + + OBSMethodNotAllowedException(final String message) { + super(message); + } + + public void setErrCode(String errCode) { + this.errCode = errCode; + } + + @Override + public String getErrCode() { + return this.errCode; + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java index e1cba55..3ea1e4c 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSObjectBucketUtils.java @@ -19,7 +19,6 @@ import com.obs.services.model.PartEtag; import com.obs.services.model.PutObjectRequest; -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.ParentNotDirectoryException; @@ -34,16 +33,17 @@ import java.io.InterruptedIOException; import java.util.ArrayList; import java.util.Collections; -import java.util.Comparator; import java.util.LinkedList; import java.util.List; +import java.util.Comparator; import java.util.Set; import java.util.TreeSet; +import java.util.Locale; import java.util.concurrent.ExecutionException; import java.util.concurrent.Future; /** - * Object bucket specific utils for {@link OBSFileSystem}. + * Object bucket specific utils */ final class OBSObjectBucketUtils { /** @@ -62,7 +62,7 @@ private OBSObjectBucketUtils() { * @param src path to be renamed * @param dst new path after rename * @return boolean - * @throws RenameFailedException if some criteria for a state changing + * @throws OBSRenameFailedException if some criteria for a state changing * rename was not met. This means work didn't * happen; it's not something which is * reported upstream to the FileSystem APIs, @@ -73,7 +73,7 @@ private OBSObjectBucketUtils() { * @throws ObsException on failures inside the OBS SDK */ static boolean renameBasedOnObject(final OBSFileSystem owner, final Path src, final Path dst) - throws RenameFailedException, FileNotFoundException, IOException, ObsException { + throws OBSRenameFailedException, FileNotFoundException, IOException, ObsException { String srcKey = OBSCommonUtils.pathToKey(owner, src); String dstKey = OBSCommonUtils.pathToKey(owner, dst); @@ -84,11 +84,11 @@ static boolean renameBasedOnObject(final OBSFileSystem owner, final Path src, fi // get the source file status; this raises a FNFE if there is no source // file. - FileStatus srcStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, src); + FileStatus srcStatus = OBSCommonUtils.getFileStatusWithRetry(owner, src); FileStatus dstStatus; try { - dstStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, dst); + dstStatus = OBSCommonUtils.getFileStatusWithRetry(owner, dst); // if there is no destination entry, an exception is raised. // hence this code sequence can assume that there is something // at the end of the path; the only detail being what it is and @@ -98,12 +98,12 @@ static boolean renameBasedOnObject(final OBSFileSystem owner, final Path src, fi String filename = srcKey.substring(OBSCommonUtils.pathToKey(owner, src.getParent()).length() + 1); newDstKey = newDstKey + filename; dstKey = newDstKey; - dstStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, OBSCommonUtils.keyToPath(dstKey)); + dstStatus = OBSCommonUtils.getFileStatusWithRetry(owner, OBSCommonUtils.keyToPath(dstKey)); if (dstStatus.isDirectory()) { - throw new RenameFailedException(src, dst, "new destination is an existed directory").withExitCode( + throw new OBSRenameFailedException(src, dst, "new destination is an existed directory").withExitCode( false); } else { - throw new RenameFailedException(src, dst, "new destination is an existed file").withExitCode(false); + throw new OBSRenameFailedException(src, dst, "new destination is an existed file").withExitCode(false); } } else { @@ -111,7 +111,7 @@ static boolean renameBasedOnObject(final OBSFileSystem owner, final Path src, fi LOG.warn("rename: src and dest refer to the same file or" + " directory: {}", dst); return true; } else { - throw new RenameFailedException(src, dst, "destination is an existed file").withExitCode(false); + throw new OBSRenameFailedException(src, dst, "destination is an existed file").withExitCode(false); } } } catch (FileNotFoundException e) { @@ -153,13 +153,13 @@ private static void checkDestinationParent(final OBSFileSystem owner, final Path Path parent = dst.getParent(); if (!OBSCommonUtils.pathToKey(owner, parent).isEmpty()) { try { - FileStatus dstParentStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, dst.getParent()); + FileStatus dstParentStatus = OBSCommonUtils.getFileStatusWithRetry(owner, dst.getParent()); if (!dstParentStatus.isDirectory()) { throw new ParentNotDirectoryException( "destination parent [" + dst.getParent() + "] is not a directory"); } } catch (FileNotFoundException e2) { - throw new RenameFailedException(src, dst, "destination has no parent "); + throw new OBSRenameFailedException(src, dst, "destination has no parent "); } } } @@ -403,11 +403,7 @@ static void createFakeDirectory(final OBSFileSystem owner, final String objectNa // Used to create an empty file that represents an empty directory static void createEmptyObject(final OBSFileSystem owner, final String objectName) throws IOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - IOException lastException = null; - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.createEmptyObject, objectName, () -> { InputStream im = null; try { im = new InputStream() { @@ -421,36 +417,13 @@ public int read() { owner.getObsClient().putObject(putObjectRequest); owner.getSchemeStatistics().incrementWriteOps(1); owner.getSchemeStatistics().incrementBytesWritten(putObjectRequest.getMetadata().getContentLength()); - return; - } catch (ObsException e) { - LOG.debug("create empty obj failed with [{}], " + "retry time [{}] - request id [{}] - " - + "error code [{}] - error message [{}]", e.getResponseCode(), retryTime, e.getErrorRequestId(), - e.getErrorCode(), e.getErrorMessage()); - - IOException ioException = OBSCommonUtils.translateException("innerCreateEmptyObject", objectName, e); - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - - lastException = ioException; - - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw ioException; - } - } } finally { if (im != null) { im.close(); } } - } - throw lastException; + return null; + }, true); } /** @@ -464,34 +437,12 @@ public int read() { * @throws IOException Other IO problems */ static void copyFile(final OBSFileSystem owner, final String srcKey, final String dstKey, final long size) - throws IOException, InterruptedIOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - innerCopyFile(owner, srcKey, dstKey, size); - return; - } catch (InterruptedIOException e) { - throw e; - } catch (OBSIOException e) { - String errMsg = String.format("Failed to copy file from %s to " + "%s with size %s, retry time %s", - srcKey, dstKey, size, retryTime); - LOG.debug(errMsg, e); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - LOG.error(errMsg, e); - throw e; - } - } - } - } - innerCopyFile(owner, srcKey, dstKey, size); + throws IOException { + String path = srcKey + " to " + dstKey; + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.copyFile, path, () -> { + innerCopyFile(owner, srcKey, dstKey, size); + return null; + }, true); } private static void innerCopyFile(final OBSFileSystem owner, final String srcKey, final String dstKey, @@ -572,8 +523,7 @@ static List getCopyFilePartEtags(final OBSFileSystem owner, final Stri owner.getObsClient() .abortMultipartUpload(new AbortMultipartUploadRequest(owner.getBucket(), dstKey, uploadId)); - throw OBSCommonUtils.extractException( - "Multi-part copy with id '" + uploadId + "' from " + srcKey + "to " + dstKey, dstKey, e); + throw OBSCommonUtils.extractException("copy part from " + srcKey + " to " + dstKey, dstKey, e); } // Make part numbers in ascending order @@ -634,7 +584,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path final Path path = OBSCommonUtils.qualify(owner, f); String key = OBSCommonUtils.pathToKey(owner, path); LOG.debug("Getting path status for {} ({})", path, key); - if (!StringUtils.isEmpty(key)) { + if (OBSCommonUtils.isStringNotEmpty(key)) { try { ObjectMetadata meta = getObjectMetadata(owner, key); @@ -644,7 +594,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path } else { LOG.debug("Found exact file: normal file"); return new OBSFileStatus(meta.getContentLength(), OBSCommonUtils.dateToLong(meta.getLastModified()), - path, owner.getDefaultBlockSize(path), owner.getShortUserName()); + path, owner.getDefaultBlockSize(path), owner.getShortUserName(), meta.getEtag()); } } catch (ObsException e) { if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) { @@ -665,7 +615,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path return new OBSFileStatus(meta.getContentLength(), OBSCommonUtils.dateToLong(meta.getLastModified()), path, owner.getDefaultBlockSize(path), - owner.getShortUserName()); + owner.getShortUserName(), meta.getEtag()); } } catch (ObsException e) { if (e.getResponseCode() != OBSCommonUtils.NOT_FOUND_CODE) { @@ -676,7 +626,7 @@ static OBSFileStatus innerGetObjectStatus(final OBSFileSystem owner, final Path } try { - boolean isEmpty = OBSCommonUtils.innerIsFolderEmpty(owner, key); + boolean isEmpty = OBSCommonUtils.isFolderEmpty(owner, key); LOG.debug("Is dir ({}) empty? {}", path, isEmpty); return new OBSFileStatus(path, owner.getShortUserName()); } catch (ObsException e) { @@ -727,7 +677,7 @@ static ContentSummary getDirectoryContentSummary(final OBSFileSystem owner, fina } summary[2] += directories.size(); LOG.debug( - String.format("file size [%d] - file count [%d] - directory count [%d] - " + "file path [%s]", summary[0], + String.format(Locale.ROOT,"file size [%d] - file count [%d] - directory count [%d] - " + "file path [%s]", summary[0], summary[1], summary[2], newKey)); return new ContentSummary.Builder().length(summary[0]) .fileCount(summary[1]) diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSOperateAction.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSOperateAction.java new file mode 100644 index 0000000..23432fd --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSOperateAction.java @@ -0,0 +1,32 @@ +package org.apache.hadoop.fs.obs; + +public enum OBSOperateAction { + getFileStatus, + rename, + delete, + getBucketFsStatus, + listObjects, + isFolderEmpty, + onReadFailure, + createEmptyObject, + createFolder, + truncate, + readOneByte, + readByteBuff, + readbytes, + readrandom, + readfully, + lazySeek, + putObject, + write, + uploadPart, + initMultiPartUpload, + completeMultipartUpload, + copyFile, + appendFile, + getContentSummaryFs, + listContentSummaryFs, + initialize, + getObjectMetadata, + setObjectMetadata +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java index 035070c..a79d1bd 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSPosixBucketUtils.java @@ -5,39 +5,68 @@ import com.obs.services.model.ListObjectsRequest; import com.obs.services.model.ObjectListing; import com.obs.services.model.ObsObject; +import com.obs.services.model.SetObjectMetadataRequest; +import com.obs.services.model.fs.ContentSummaryFsRequest; +import com.obs.services.model.fs.ContentSummaryFsResult; +import com.obs.services.model.fs.DirContentSummary; +import com.obs.services.model.fs.DirSummary; import com.obs.services.model.fs.GetAttributeRequest; import com.obs.services.model.fs.NewFolderRequest; import com.obs.services.model.fs.ObsFSAttribute; import com.obs.services.model.fs.RenameRequest; +import com.obs.services.model.fs.ListContentSummaryFsRequest; +import com.obs.services.model.fs.ListContentSummaryFsResult; +import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.FileStatus; import org.apache.hadoop.fs.ParentNotDirectoryException; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.PathIsNotEmptyDirectoryException; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.io.retry.RetryPolicy; import org.apache.hadoop.security.AccessControlException; +import com.google.common.collect.ImmutableMap; import org.slf4j.Logger; import org.slf4j.LoggerFactory; import java.io.FileNotFoundException; import java.io.IOException; +import java.io.InterruptedIOException; import java.text.SimpleDateFormat; import java.util.ArrayList; import java.util.Date; import java.util.List; +import java.util.Locale; +import java.util.Map; +import java.util.Objects; +import java.util.Optional; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingDeque; +import java.util.concurrent.TimeUnit; /** - * Posix bucket specific utils for {@link OBSFileSystem}. + * Posix bucket specific utils */ final class OBSPosixBucketUtils { - /** - * Class logger. - */ private static final Logger LOG = LoggerFactory.getLogger(OBSPosixBucketUtils.class); + private static RetryPolicyWithMaxTime retryPolicy; + private static RetryPolicyWithMaxTime obsRetryPolicy; + private OBSPosixBucketUtils() { } + public static void init(Configuration conf) { + retryPolicy = new OBSRetryPolicy.ExponentialBackoffRetryPolicy( + conf.getInt(OBSConstants.RETRY_LIMIT, OBSConstants.DEFAULT_RETRY_LIMIT), + conf.getLong(OBSConstants.RETRY_MAXTIME, OBSConstants.DEFAULT_RETRY_MAXTIME), + conf.getLong(OBSConstants.RETRY_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_SLEEP_BASETIME), + conf.getLong(OBSConstants.RETRY_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_SLEEP_MAXTIME)); + obsRetryPolicy = new OBSRetryPolicy(conf); + } + /** * Used to judge that an object is a file or folder. * @@ -64,7 +93,7 @@ static boolean fsIsFolder(final ObsFSAttribute attr) { * @param src source path to be renamed from * @param dst destination path to be renamed to * @return boolean - * @throws RenameFailedException if some criteria for a state changing + * @throws OBSRenameFailedException if some criteria for a state changing * rename was not met. This means work didn't * happen; it's not something which is * reported upstream to the FileSystem APIs, @@ -83,7 +112,7 @@ static boolean renameBasedOnPosix(final OBSFileSystem owner, final Path src, fin } try { - FileStatus dstStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, dstPath); + FileStatus dstStatus = OBSCommonUtils.getFileStatusWithRetry(owner, dstPath); if (dstStatus.isDirectory()) { String newDstString = OBSCommonUtils.maybeAddTrailingSlash(dstPath.toString()); String filename = srcKey.substring(OBSCommonUtils.pathToKey(owner, src.getParent()).length() + 1); @@ -108,7 +137,7 @@ static boolean renameBasedOnPosix(final OBSFileSystem owner, final Path src, fin // if destination does not exist, do not change the // destination key, and just do rename. LOG.debug("rename: dest [{}] does not exist", dstPath); - } catch (FileConflictException e) { + } catch (OBSFileConflictException e) { throw new ParentNotDirectoryException(e.getMessage()); } @@ -125,65 +154,70 @@ static boolean innerFsRenameWithRetry(final OBSFileSystem owner, final Path src, final String srcKey, final String dstKey) throws IOException { String newSrcKey = srcKey; String newDstKey = dstKey; - IOException lastException; - long delayMs; - int retryTime = 0; + int retries = 0; long startTime = System.currentTimeMillis(); - do { + while (true) { boolean isRegularDirPath = newSrcKey.endsWith("/") && newDstKey.endsWith("/"); try { - LOG.debug("rename: {}-st rename from [{}] to [{}] ...", retryTime, newSrcKey, newDstKey); + LOG.debug("rename: {}-st rename from [{}] to [{}] ...", retries, newSrcKey, newDstKey); innerFsRenameFile(owner, newSrcKey, newDstKey); return true; } catch (FileNotFoundException e) { if (owner.exists(dst)) { - LOG.debug("rename: successfully {}-st rename src [{}] " + "to dest [{}] with SDK retry", retryTime, - src, dst, e); + LOG.debug("file not found when rename. rename: successfully {}-st rename src [{}] " + "to dest [{}] with SDK retry", retries, + src, dst); return true; } else { - LOG.error("rename: failed {}-st rename src [{}] to dest [{}]", retryTime, src, dst, e); + LOG.error("file not found when rename. rename: failed {}-st rename src [{}] to dest [{}]", retries, src, dst); return false; } - } catch (IOException e) { - if (e instanceof AccessControlException && isRegularDirPath) { + } catch (AccessControlException e) { + if (isRegularDirPath) { throw e; } - try { - FileStatus srcFileStatus = OBSCommonUtils.innerGetFileStatusWithRetry(owner, src); + FileStatus srcFileStatus = OBSCommonUtils.getFileStatusWithRetry(owner, src); if (srcFileStatus.isDirectory()) { newSrcKey = OBSCommonUtils.maybeAddTrailingSlash(newSrcKey); newDstKey = OBSCommonUtils.maybeAddTrailingSlash(newDstKey); - } else if (e instanceof AccessControlException) { + continue; + } else { throw e; } - } catch (FileConflictException e1) { + } catch (OBSFileConflictException e1) { + throw new AccessControlException(e); + } + } catch (IOException e) { + try { + OBSCommonUtils.getFileStatusWithRetry(owner, src); + } catch (OBSFileConflictException e1) { throw new AccessControlException(e); } - - lastException = e; - LOG.warn("rename: failed {}-st rename src [{}] to dest [{}]", retryTime, src, dst, e); if (owner.exists(dst) && owner.exists(src)) { - LOG.warn("rename: failed {}-st rename src [{}] to " + "dest [{}] with SDK retry", retryTime, src, + LOG.warn("rename: failed {}-st rename src [{}] to " + "dest [{}] with SDK retry", retries, src, dst, e); return false; } - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw e; - } + OBSCommonUtils.putQosMetric(owner, OBSOperateAction.rename, e); + + RetryPolicy.RetryAction rc; + try { + rc = obsRetryPolicy.shouldRetryByMaxTime(startTime, e, retries++, 0, true); + } catch (Exception e1) { + throw new IOException("unexpected exception ", e1); + } + if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) { + throw e; + } + try { + LOG.warn("retry #{}, {}", retries, e); + Thread.sleep(rc.delayMillis); + } catch (InterruptedException e1) { + throw (IOException)new InterruptedIOException("interrupted").initCause(e1); } } - } while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY); - - LOG.error("rename: failed {}-st rename src [{}] to dest [{}]", retryTime, src, dst, lastException); - throw lastException; + } } static void innerFsRenameFile(final OBSFileSystem owner, final String srcKey, final String dstKey) @@ -258,7 +292,7 @@ static boolean fsDelete(final OBSFileSystem owner, final FileStatus status, fina LOG.debug("delete: Deleting fake empty directory {} - recursive {}", f, recursive); try { OBSCommonUtils.deleteObject(owner, key); - } catch (FileConflictException e) { + } catch (OBSFileConflictException e) { LOG.warn("delete emtryDir[{}] has conflict exception, " + "will retry.", key, e); trashFolderIfNeed(owner, key); @@ -283,39 +317,34 @@ private static void trashObjectIfNeed(final OBSFileSystem owner, final String ke } mkTrash(owner, key); - String destKeyWithNoSuffix = owner.getTrashDir() + key; + String destKeyWithNoSuffix = owner.getFastDeleteDir() + key; String destKey = destKeyWithNoSuffix; SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmssSSS"); if (owner.exists(new Path(destKey))) { destKey = destKeyWithNoSuffix + df.format(new Date()); } - // add timestamp when rename failed to avoid multi clients rename sources to the same target + + LOG.debug("Moved file : '" + key + "' to trash at: " + destKey); + int retries = 0; long startTime = System.currentTimeMillis(); - int retryTime = 0; - long delayMs; while (!fsRenameToNewObject(owner, key, destKey)) { - LOG.debug("Move file [{}] to [{}] failed, retryTime[{}].", key, - destKey, retryTime); - - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - if (System.currentTimeMillis() - startTime + delayMs - > OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - LOG.error("Failed rename file [{}] to [{}] after " - + "retryTime[{}].", key, destKey, retryTime); - throw new IOException("Failed to rename " + key + " to " + destKey); - } else { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - LOG.error("Failed rename file [{}] to [{}] after " - + "retryTime[{}].", key, destKey, retryTime); - throw new IOException("Failed to rename " + key + " to " + destKey); - } + RetryPolicy.RetryAction rc; + try { + rc = retryPolicy.shouldRetryByMaxTime(startTime, null, retries++, 0, true); + } catch (Exception e1) { + throw new IOException("unexpected exception ", e1); + } + if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) { + throw new IOException("failed to rename " + key + " to " + destKey); + } + try { + LOG.warn("retry #{}, fast delete {} to {}", retries, key, destKey); + Thread.sleep(rc.delayMillis); + destKey = destKeyWithNoSuffix + df.format(new Date()); + } catch (InterruptedException e1) { + throw (IOException)new InterruptedIOException("interrupted").initCause(e1); } - destKey = destKeyWithNoSuffix + df.format(new Date()); - retryTime++; } - LOG.debug("Moved file : '" + key + "' to trash at: " + destKey); } private static void trashFolderIfNeed(final OBSFileSystem owner, final String key) @@ -326,7 +355,7 @@ private static void trashFolderIfNeed(final OBSFileSystem owner, final String ke } mkTrash(owner, key); - StringBuilder sb = new StringBuilder(owner.getTrashDir()); + StringBuilder sb = new StringBuilder(owner.getFastDeleteDir()); SimpleDateFormat df = new SimpleDateFormat("-yyyyMMddHHmmssSSS"); int endIndex = key.endsWith("/") ? key.length() - 1 : key.length(); sb.append(key, 0, endIndex); @@ -339,64 +368,56 @@ private static void trashFolderIfNeed(final OBSFileSystem owner, final String ke String srcKey = OBSCommonUtils.maybeAddTrailingSlash(key); String dstKey = OBSCommonUtils.maybeAddTrailingSlash(destKey); - // add timestamp when rename failed to avoid multi clients rename sources to the same target + LOG.debug("Moved folder : '" + key + "' to trash: " + destKey); + int retries = 0; long startTime = System.currentTimeMillis(); - int retryTime = 0; - long delayMs; while (!fsRenameToNewObject(owner, srcKey, dstKey)) { - LOG.debug("Move folder [{}] to [{}] failed, retryTime[{}].", key, - destKey, retryTime); - - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - if (System.currentTimeMillis() - startTime + delayMs - > OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - LOG.error("Failed rename folder [{}] to [{}] after " - + "retryTime[{}].", key, destKey, retryTime); - throw new IOException("Failed to rename " + key + " to " + destKey); - } else { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - LOG.error("Failed rename folder [{}] to [{}] after " - + "retryTime[{}].", key, destKey, retryTime); - throw new IOException("Failed to rename " + key + " to " + destKey); - } + RetryPolicy.RetryAction rc; + try { + rc = retryPolicy.shouldRetryByMaxTime(startTime, null, retries++, 0, true); + } catch (Exception e1) { + throw new IOException("unexpected exception ", e1); + } + if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) { + throw new IOException("failed to rename " + key + " to " + destKey); + } + try { + LOG.warn("retry #{}, fast delete {} to {}", retries, key, destKey); + Thread.sleep(rc.delayMillis); + destKey = destKeyWithNoSuffix + df.format(new Date()); + dstKey = OBSCommonUtils.maybeAddTrailingSlash(destKey); + } catch (InterruptedException e1) { + throw (IOException)new InterruptedIOException("interrupted").initCause(e1); } - destKey = destKeyWithNoSuffix + df.format(new Date()); - dstKey = OBSCommonUtils.maybeAddTrailingSlash(destKey); - retryTime++; } - LOG.debug("Moved folder : '" + key + "' to trash at: " + destKey); } static void fsRecursivelyDeleteDirWithRetry(final OBSFileSystem owner, final String key, boolean deleteParent) throws IOException { + int retries = 0; long startTime = System.currentTimeMillis(); - long delayMs; - int retryTime = 0; - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { + while (true) { try { - long delNum = fsRecursivelyDeleteDir(owner, key, deleteParent); - LOG.debug("Recursively delete {} files/dirs when deleting {}", - delNum, key); + fsRecursivelyDeleteDir(owner, key, deleteParent); return; - } catch (FileConflictException e) { - LOG.warn("Recursively delete [{}] has conflict exception, " - + "retryTime[{}].", key, e); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw e; - } + } catch (OBSFileConflictException e) { + RetryPolicy.RetryAction rc; + try { + LOG.warn("retry #{}, {}", retries, e); + rc = retryPolicy.shouldRetryByMaxTime(startTime, e, retries++, 0, true); + } catch (Exception e1) { + throw new IOException("Unexpected exception ", e1); + } + if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) { + throw e; + } + try { + Thread.sleep(rc.delayMillis); + } catch (InterruptedException e1) { + throw (IOException)new InterruptedIOException("interrupted").initCause(e1); } } } - - fsRecursivelyDeleteDir(owner, key, deleteParent); } static long fsRecursivelyDeleteDir(final OBSFileSystem owner, final String parentKey, final boolean deleteParent) @@ -456,13 +477,13 @@ static long fsRecursivelyDeleteDir(final OBSFileSystem owner, final String paren private static boolean needToTrash(final OBSFileSystem owner, final String key) { String newKey = key; newKey = OBSCommonUtils.maybeDeleteBeginningSlash(newKey); - if (owner.isEnableTrash()) { - String trashPathKey = OBSCommonUtils.pathToKey(owner, new Path(owner.getTrashDir())); + if (owner.isEnableFastDelete()) { + String trashPathKey = OBSCommonUtils.pathToKey(owner, new Path(owner.getFastDeleteDir())); if (newKey.startsWith(trashPathKey)) { return false; } } - return owner.isEnableTrash(); + return owner.isEnableFastDelete(); } // Delete a sub dir. @@ -482,7 +503,7 @@ private static int fsRemoveSubdir(final OBSFileSystem owner, final String subdir private static void mkTrash(final OBSFileSystem owner, final String key) throws ObsException, IOException { String newKey = key; - StringBuilder sb = new StringBuilder(owner.getTrashDir()); + StringBuilder sb = new StringBuilder(owner.getFastDeleteDir()); newKey = OBSCommonUtils.maybeAddTrailingSlash(newKey); sb.append(newKey); sb.deleteCharAt(sb.length() - 1); @@ -500,39 +521,13 @@ static void fsCreateFolder(final OBSFileSystem owner, final String objectName) t final NewFolderRequest newFolderRequest = new NewFolderRequest(owner.getBucket(), newObjectName); newFolderRequest.setAcl(owner.getCannedACL()); long len = newFolderRequest.getObjectKey().length(); - IOException lastException = null; - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - owner.getObsClient().newFolder(newFolderRequest); - owner.getSchemeStatistics().incrementWriteOps(1); - owner.getSchemeStatistics().incrementBytesWritten(len); - return; - } catch (ObsException e) { - LOG.debug("Failed to create folder [{}], retry time [{}], " + "exception [{}]", newObjectName, - retryTime, e); + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.createFolder, objectName, () -> { + owner.getObsClient().newFolder(newFolderRequest); + return null; + },true); - IOException ioException = OBSCommonUtils.translateException("innerFsCreateFolder", newObjectName, e); - if (!(ioException instanceof OBSIOException)) { - throw ioException; - } - lastException = ioException; - - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw ioException; - } - } - } - } - throw lastException; + owner.getSchemeStatistics().incrementWriteOps(1); + owner.getSchemeStatistics().incrementBytesWritten(len); } // Used to get the status of a file or folder in a file-gateway bucket. @@ -549,16 +544,22 @@ static OBSFileStatus innerFsGetObjectStatus(final OBSFileSystem owner, final Pat try { final GetAttributeRequest getAttrRequest = new GetAttributeRequest(owner.getBucket(), key); ObsFSAttribute meta = owner.getObsClient().getAttribute(getAttrRequest); + owner.getSchemeStatistics().incrementReadOps(1); if (fsIsFolder(meta)) { LOG.debug("Found file (with /): fake directory"); - return new OBSFileStatus(path, OBSCommonUtils.dateToLong(meta.getLastModified()), + OBSFileStatus status = new OBSFileStatus(path, OBSCommonUtils.dateToLong(meta.getLastModified()), owner.getShortUserName()); - } else { - LOG.debug("Found file (with /): real file? should not happen: {}", key); - return new OBSFileStatus(meta.getContentLength(), OBSCommonUtils.dateToLong(meta.getLastModified()), - path, owner.getDefaultBlockSize(path), owner.getShortUserName()); + OBSCommonUtils.setAccessControlAttrForFileStatus(owner, status, meta.getAllMetadata()); + return status; } + + LOG.debug("Found file (with /): real file? should not happen: {}", key); + OBSFileStatus status = new OBSFileStatus(meta.getContentLength(), + OBSCommonUtils.dateToLong(meta.getLastModified()), path, owner.getDefaultBlockSize(path), + owner.getShortUserName(), meta.getEtag()); + OBSCommonUtils.setAccessControlAttrForFileStatus(owner, status, meta.getAllMetadata()); + return status; } catch (ObsException e) { throw OBSCommonUtils.translateException("getFileStatus", path, e); } @@ -601,7 +602,7 @@ static ContentSummary fsGetDirectoryContentSummary(final OBSFileSystem owner, fi objects = OBSCommonUtils.continueListObjects(owner, objects); } LOG.debug( - String.format("file size [%d] - file count [%d] - directory count [%d] - " + "file path [%s]", summary[0], + String.format(Locale.ROOT,"file size [%d] - file count [%d] - directory count [%d] - " + "file path [%s]", summary[0], summary[1], summary[2], newKey)); return new ContentSummary.Builder().length(summary[0]) .fileCount(summary[1]) @@ -612,41 +613,425 @@ static ContentSummary fsGetDirectoryContentSummary(final OBSFileSystem owner, fi static void innerFsTruncateWithRetry(final OBSFileSystem owner, final Path f, final long newLength) throws IOException { - long delayMs; - int retryTime = 0; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { + LOG.debug("truncate {} to newLength {}", f, newLength); + String key = OBSCommonUtils.pathToKey(owner, f); + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.truncate, key, () -> { + owner.getObsClient().truncateObject(owner.getBucket(), key, newLength); + return null; + }, true); + owner.getSchemeStatistics().incrementWriteOps(1); + } + + static ContentSummary fsGetDirectoryContentSummaryV2(final OBSFileSystem owner, final FileStatus fileStatus) + throws IOException { + FsDirContentSummaryCounter summaryCounter = new FsDirContentSummaryCounter(owner, fileStatus); + return summaryCounter.getContentSummary(); + } + + private static class FsDirContentSummaryCounter { + public static final int MULTI_LIST_CS_MAX_DIR = 10; // multiListCS allow up to 10 dir in one batch + + public static final int MIN_RETRY_TIME = 3; + + private final OBSFileSystem owner; + + private final LinkedBlockingDeque deque; + + private final Counter counter; + + private final FileStatus fileStatus; + + private final int parallelFactor; + + public FsDirContentSummaryCounter(OBSFileSystem owner, FileStatus fileStatus) { + this.owner = owner; + this.deque = new LinkedBlockingDeque<>(); + this.counter = new Counter(); + this.fileStatus = fileStatus; + this.parallelFactor = owner.getConf().getInt(OBSConstants.MULTILISTCS_PARALLEL_FACTOR, + OBSConstants.DEFAULT_MULTILISTCS_PARALLEL_FACTOR); + } + + /** + * get content summary of path, path should be a dir + * + * @return + */ + public ContentSummary getContentSummary() throws IOException { + if (!fileStatus.isDirectory()) { + throw new IllegalArgumentException("the input should be a dir"); + } + this.counter.increase(1, 0, 0); // count root itself + LOG.debug("counter increase (1, 0, 0) for root itself"); + + Path path = fileStatus.getPath(); + String key = OBSCommonUtils.pathToKey(owner, path); + if (path.isRoot()) { + key = "/"; // special case of 'root' + } + ContentSummaryFsRequest getCSReq = new ContentSummaryFsRequest(); + getCSReq.setBucketName(this.owner.getBucket()); + getCSReq.setDirName(key); + + // 1. getCS of root dir try { - innerFsTruncate(owner, f, newLength); + ContentSummaryFsResult getCSRes = OBSCommonUtils.getOBSInvoker().retryByMaxTime( + OBSOperateAction.getContentSummaryFs, key, () -> this.owner.getObsClient().getContentSummaryFs(getCSReq), + true); + DirSummary summary = getCSRes.getContentSummary(); + this.counter.increase(summary.getDirCount(), summary.getFileCount(), summary.getFileSize()); + LOG.debug("counter increase ({}, {}, {}) for [{}]", + summary.getDirCount(), summary.getFileCount(), summary.getFileSize(), key); + if (summary.getDirCount() != 0) { + enqueue(new Node(key, null, summary.getInode())); + } + } catch (ObsException e) { + if (e.getResponseCode() == OBSCommonUtils.NOT_ALLOWED_CODE) { + throw new UnsupportedOperationException("unsupported getContentSummaryFs"); + } + throw OBSCommonUtils.translateException("getContentSummaryFs", path, e); + } + + countExhaustive(); + + return this.counter.getContentSummary(); + } + + private void countExhaustive() throws IOException { + while (true) { + List> futures = new ArrayList<>(); + + // 2. try to get node for queue + try { + for (int i = 0; i < parallelFactor; i++) { + BatchNodes nodes = tryToGetBatchNodes(); + if (nodes.size() == 0) { // queue is empty, break out + break; + } else { + // 3. submit task + futures.add(submitTask(nodes)); + } + } + } catch (InterruptedException e) { + LOG.warn("getContentSummaryV2 for [{}] failed because get nodes from queue failed, {}", + fileStatus.getPath().toString(), e.getMessage()); + throw new IOException(String.format( + "getContentSummaryV2 for [%s] failed because get nodes from queue failed", + fileStatus.getPath().toString()), e); + } + + // 4. check finish + if (futures.size() == 0) { + if (deque.size() != 0) { + continue; + } + break; + } + + // 5. wait for batch task finished + waitBatchTaskFinish(futures); + } + } + + private BatchNodes tryToGetBatchNodes() throws InterruptedException, IOException { + BatchNodes ret = new BatchNodes(); + int size = this.deque.size(); + for (int i = 0; i < size; i++) { + if (ret.size() == MULTI_LIST_CS_MAX_DIR) { + break; + } + Node n = this.deque.pollFirst(1, TimeUnit.SECONDS); + if (n != null && verifyNodeRetryState(n)) { + ret.add(n); + } + } + return ret; + } + + // verify node should add to request + private boolean verifyNodeRetryState(Node n) throws IOException { + final int retryState = n.retryState(); + switch (retryState) { + case Node.RETRY_STATE_TRIGGER: + return true; + case Node.RETRY_STATE_DISCARD: + LOG.error("node[key={} marker={}] failed {} times, due to {}", + n.getPath(), n.getMarker(), n.getRetryNum(), n.getRetryMsg()); + throw new IllegalStateException(String.format(Locale.ROOT,"node[key=%s marker=%s] failed %d times, due to %s", + n.getPath(), n.getMarker(), n.getRetryNum(), n.getRetryMsg())); + case Node.RETRY_STATE_DELAY: + try { + Thread.sleep(100); + } catch (InterruptedException ie) { + LOG.error("failed to retry node[key={}, marker={}], retry time[{}], due to {}", + n.getPath(), n.getMarker(), n.getRetryNum(), n.getRetryMsg()); + throw new IllegalStateException(n.getRetryMsg()); + } + if (!deque.offerLast(n)) { // put retry node to the end of queue + LOG.warn("node [{}, {}, {}, {}] enqueue failed, may be queue is full", + n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum()); + throw new IllegalStateException( + String.format(Locale.ROOT,"node [%s, %d, %s, %d] enqueue failed, may be queue is full", + n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum())); + } + return false; + + default: + throw new IllegalStateException("unreachable code"); + } + } + + private Future submitTask(BatchNodes nodes) { + return owner.getBoundedListThreadPool().submit( + () -> { + List dirs = transformToDirLayer(nodes); + ListContentSummaryFsRequest req = new ListContentSummaryFsRequest(); + req.setBucketName(owner.getBucket()); + req.setMaxKeys(owner.getMaxKeys()); + req.setDirLayers(dirs); + ListContentSummaryFsResult res; + res = OBSCommonUtils.getOBSInvoker().retryByMaxTime( + OBSOperateAction.listContentSummaryFs, "", + () -> owner.getObsClient().listContentSummaryFs(req), + true); + handleListContentSummaryFsResult(res, nodes); + return ""; + } + ); + } + + private void enqueue(Node n) { + if (!deque.offerFirst(n)) { + LOG.warn("node [{}, {}, {}, {}] enqueue failed, may be queue is full", + n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum()); + throw new IllegalStateException( + String.format(Locale.ROOT,"node [%s, %d, %s, %d] enqueue failed, may be queue is full", + n.getPath(), n.getInode(), n.getMarker(), n.getRetryNum())); + } + } + + private List transformToDirLayer(BatchNodes nodes) { + List ret = new ArrayList<>(); + for (Node node : nodes) { + ListContentSummaryFsRequest.DirLayer dir = new ListContentSummaryFsRequest.DirLayer(); + dir.setKey(node.getPath()); + dir.setMarker(node.getMarker()); + dir.setInode(node.getInode()); + LOG.debug("transform node ({}, {}, {}, {})", + node.getPath(), node.getMarker(), node.getInode(), node.getRetryNum()); + ret.add(dir); + } + return ret; + } + + private void handleListContentSummaryFsResult(ListContentSummaryFsResult res, BatchNodes nodes) { + // handle errors + List errs = res.getErrorResults(); + if (errs != null) { + for (ListContentSummaryFsResult.ErrorResult err : errs) { + LOG.debug("listContentSummary return error contents: {}, {}, {}", err.getKey(), err.getErrorCode(), + err.getMessage()); + retryErrorResult(err, nodes); + } + } + + List dirs = res.getDirContentSummaries(); + if (dirs != null) { + for (DirContentSummary dir : dirs) { + if (dir.isTruncated()) { + // put self with nextMarker to the queue + enqueue(new Node(dir.getKey(), dir.getNextMarker(), dir.getInode())); + } + // handle subdirs + for (DirSummary subDir : dir.getSubDir()) { + counter.increase(subDir.getDirCount(), subDir.getFileCount(), subDir.getFileSize()); + LOG.debug("counter increase ({}, {}, {}) for [{}, {}]", + subDir.getDirCount(), subDir.getFileCount(), subDir.getFileSize(), + subDir.getName(), subDir.getInode()); + if (subDir.getDirCount() != 0) { // enqueue when dir has sub dir + enqueue(new Node(subDir.getName(), null, subDir.getInode())); + } + } + } + } + } + + private void retryErrorResult(ListContentSummaryFsResult.ErrorResult err, BatchNodes nodes) { + String statusCode = err.getStatusCode(); + if (statusCode == null) { + LOG.warn("statusCode is null, {}", err); return; - } catch (OBSIOException e) { - OBSFileSystem.LOG.debug( - "Failed to truncate [{}] to newLength" + " [{}], retry time [{}], exception [{}]", f, newLength, - retryTime, e); + } + // 408 409 429 5XX should retry + boolean needRetry = statusCode.equals("408") || statusCode.equals("409") || statusCode.equals("429"); + needRetry = needRetry || statusCode.startsWith("5"); // 5xx + + if (needRetry) { + for (final Node node : nodes) { + if (node.getPath().equals(err.getKey()) /*&& node.getMarker().equals(err.getMarker())*/) { + node.increaseRetryNum( + String.format("%s %s %s", err.getStatusCode(), err.getErrorCode(), err.getMessage())); + enqueue(node); + return; + } + } + throw new IllegalStateException(String.format("Unreachable code: could not find err[key=%s marker=] from nodes, err=%s", + err.getKey(), err.getMessage())); + } + } - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; + private void waitBatchTaskFinish(List> futures) throws IOException { + for (Future fRes : futures) { try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw e; + fRes.get(); + } catch (InterruptedException e) { + LOG.warn("Interrupted while listContentSummary, {}", e.getMessage()); + throw new InterruptedIOException(String.format("Interrupted while listContentSummary, %s", e)); + } catch (ExecutionException e) { + fRes.cancel(true); + throw OBSCommonUtils.extractException("ListContentSummary with exception", "", e); + } + } + } + + private static class BatchNodes extends ArrayList { + } + + private static class Node { + public static final int RETRY_STATE_DISCARD = 1; // exceed retry limits, discard + + public static final int RETRY_STATE_DELAY = 2; // not reach action moment + + public static final int RETRY_STATE_TRIGGER = 3; // reach the action moment, do request immediately + + private final String path; + + private final String marker; + + private final long inode; + + private int retryNum; + + private long previousRetry; + + private long firstProcessTime; + + private String retryMsg; + + public String getRetryMsg() { + return retryMsg; + } + + public long getInode() { + return inode; + } + + public int retryState() throws IOException { + if (retryNum == 0) { + return RETRY_STATE_TRIGGER; + } + RetryPolicy.RetryAction rc; + try { + rc = retryPolicy.shouldRetryByMaxTime(firstProcessTime, null, retryNum, 0, true); + } catch (Exception e) { + throw new IOException("unexpected exception", e); + } + + if (rc.action == RetryPolicy.RetryAction.RetryDecision.FAIL) { + return RETRY_STATE_DISCARD; + } + + if (rc.action == RetryPolicy.RetryAction.RetryDecision.RETRY) { + long delayMs = rc.delayMillis; + long now = System.currentTimeMillis(); + if (previousRetry + delayMs > now) { + return RETRY_STATE_DELAY; + } + } + return RETRY_STATE_TRIGGER; + } + + public int getRetryNum() { + return retryNum; + } + + public String getPath() { + return path; + } + + public String getMarker() { + return marker; + } + + public Node(String path, String marker, long inode) { + this.path = OBSCommonUtils.maybeAddTrailingSlash(path); + this.marker = marker; + this.retryNum = 0; + this.previousRetry = 0; + this.inode = inode; + } + + public void increaseRetryNum(String retryMsg) { + LOG.debug("node[{}, {}] increase retry, msg[{}]", path, marker, retryMsg); + this.retryNum++; + this.previousRetry = System.currentTimeMillis(); + if (retryNum == 1) { + firstProcessTime = previousRetry; } + this.retryMsg = retryMsg; + } + + } + + private static class Counter { + private volatile long dirNum; + + private volatile long fileNum; + + private volatile long size; + + synchronized void increase(long ndir, long nfile, long size) { + this.dirNum += ndir; + this.fileNum += nfile; + this.size += size; + } + + ContentSummary getContentSummary() { + return new ContentSummary.Builder().length(this.size) + .fileCount(this.fileNum) + .directoryCount(this.dirNum) + .spaceConsumed(this.size) + .build(); } } + } - innerFsTruncate(owner, f, newLength); + static void fsSetOwner(final OBSFileSystem owner, final Path f, final String username, final String groupname) + throws IOException { + Map userMeta = ImmutableMap.of("user", username, "group", groupname); + addObjectMetadata(owner, f, userMeta); } - private static void innerFsTruncate(final OBSFileSystem owner, final Path f, final long newLength) + static void fsSetPermission(final OBSFileSystem owner, final Path f, final FsPermission permission) throws IOException { - LOG.debug("truncate {} to newLength {}", f, newLength); + Map userMeta = ImmutableMap.of("permission", String.valueOf(permission.toShort())); + addObjectMetadata(owner, f, userMeta); + } - try { - String key = OBSCommonUtils.pathToKey(owner, f); - owner.getObsClient().truncateObject(owner.getBucket(), key, newLength); - owner.getSchemeStatistics().incrementWriteOps(1); - } catch (ObsException e) { - throw OBSCommonUtils.translateException("truncate", f, e); - } + private static void addObjectMetadata(final OBSFileSystem owner, final Path f, final Map metaToAdd) + throws IOException { + String key = OBSCommonUtils.pathToKey(owner, f); + SetObjectMetadataRequest req = new SetObjectMetadataRequest(owner.getBucket(), key); + req.addAllUserMetadata(metaToAdd); + + // query existed objMeta and add it to setMetaReq to avoid existed objMeta been overwritten. + Map objMeta = OBSCommonUtils.getObjectMetadata(owner, key); + objMeta.forEach((k, v) -> req.addUserMetadata(k, Optional.of(v).map(Objects::toString) + .orElse(null))); + + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.setObjectMetadata, key, () -> { + owner.getObsClient().setObjectMetadata(req); + return null; + }, true); } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/ObsClientFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSQosException.java similarity index 52% rename from hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/ObsClientFactory.java rename to hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSQosException.java index f0cbc3a..cbc5322 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/ObsClientFactory.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSQosException.java @@ -18,29 +18,17 @@ package org.apache.hadoop.fs.obs; -import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; -import org.apache.hadoop.classification.InterfaceAudience; -import org.apache.hadoop.classification.InterfaceStability; - -import java.io.IOException; -import java.net.URI; /** - * Factory for creating OBS client instance to be used by {@link - * OBSFileSystem}. + * IOException equivalent to {@link ObsException}. */ -@InterfaceAudience.Private -@InterfaceStability.Unstable -interface OBSClientFactory { - /** - * Creates a new {@link ObsClient} client. This method accepts the OBS file - * system URI both in raw input form and validated form as separate - * arguments, because both values may be useful in logging. - * - * @param name raw input OBS file system URI - * @return OBS client - * @throws IOException IO problem - */ - ObsClient createObsClient(URI name) throws IOException; +public class OBSQosException extends OBSIOException implements WithErrCode { + private static final long serialVersionUID = -1582681108285856259L; + + OBSQosException(final String operationMsg, final ObsException cause) { + super(operationMsg,cause); + } + } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRenameFailedException.java similarity index 81% rename from hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java rename to hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRenameFailedException.java index a67dbf0..14259dc 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RenameFailedException.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRenameFailedException.java @@ -25,13 +25,15 @@ * Exception to indicate a specific rename failure. The exit code defines the * value returned by {@link OBSFileSystem#rename(Path, Path)}. */ -class RenameFailedException extends PathIOException { +class OBSRenameFailedException extends PathIOException { + private static final long serialVersionUID = -1993835850714285642L; + /** * Exit code to be returned. */ private boolean exitCode = false; - RenameFailedException(final Path src, final Path optionalDest, final String error) { + OBSRenameFailedException(final Path src, final Path optionalDest, final String error) { super(src.toString(), error); setOperation("rename"); if (optionalDest != null) { @@ -43,13 +45,7 @@ public boolean getExitCode() { return exitCode; } - /** - * Set the exit code. - * - * @param code exit code to raise - * @return the exception - */ - public RenameFailedException withExitCode(final boolean code) { + public OBSRenameFailedException withExitCode(final boolean code) { this.exitCode = code; return this; } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRetryPolicy.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRetryPolicy.java new file mode 100644 index 0000000..692a519 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSRetryPolicy.java @@ -0,0 +1,210 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import java.io.EOFException; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.net.NoRouteToHostException; +import java.net.UnknownHostException; +import java.util.HashMap; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.security.AccessControlException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.io.retry.RetryPolicy; + + +public class OBSRetryPolicy implements RetryPolicyWithMaxTime { + + private RetryPolicyWithMaxTime defaultPolicy; + private Map, RetryPolicyWithMaxTime> exceptionToPolicyMap; + + private final RetryPolicyWithMaxTime throttleRetryPolicy; + + private final RetryPolicyWithMaxTime idempotencyIoRetryPolicy; + + private final RetryPolicyWithMaxTime failRetryPolicy; + + public OBSRetryPolicy(Configuration conf) { + failRetryPolicy = new NoRetryPolicy(); + + ExponentialBackoffRetryPolicy baseRetryPolicy = new ExponentialBackoffRetryPolicy( + conf.getInt(OBSConstants.RETRY_LIMIT, OBSConstants.DEFAULT_RETRY_LIMIT), + conf.getLong(OBSConstants.RETRY_MAXTIME, OBSConstants.DEFAULT_RETRY_MAXTIME), + conf.getLong(OBSConstants.RETRY_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_SLEEP_BASETIME), + conf.getLong(OBSConstants.RETRY_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_SLEEP_MAXTIME)); + idempotencyIoRetryPolicy = new IdempotencyIoRetryPolicy(baseRetryPolicy); + + throttleRetryPolicy = new ExponentialBackoffRetryPolicy( + conf.getInt(OBSConstants.RETRY_QOS_LIMIT, OBSConstants.DEFAULT_RETRY_QOS_LIMIT), + conf.getLong(OBSConstants.RETRY_QOS_MAXTIME, OBSConstants.DEFAULT_RETRY_QOS_MAXTIME), + conf.getLong(OBSConstants.RETRY_QOS_SLEEP_BASETIME, OBSConstants.DEFAULT_RETRY_QOS_SLEEP_BASETIME), + conf.getLong(OBSConstants.RETRY_QOS_SLEEP_MAXTIME, OBSConstants.DEFAULT_RETRY_QOS_SLEEP_MAXTIME)); + + exceptionToPolicyMap = createExceptionMap(); + defaultPolicy = idempotencyIoRetryPolicy; + } + + private Map, RetryPolicyWithMaxTime> createExceptionMap() { + Map, RetryPolicyWithMaxTime> policyMap = new HashMap<>(); + + policyMap.put(UnknownHostException.class, failRetryPolicy); + policyMap.put(NoRouteToHostException.class, failRetryPolicy); + policyMap.put(InterruptedIOException.class, failRetryPolicy); + policyMap.put(InterruptedException.class, failRetryPolicy); + + policyMap.put(AccessControlException.class, failRetryPolicy); + policyMap.put(FileNotFoundException.class, failRetryPolicy); + policyMap.put(OBSIllegalArgumentException.class, failRetryPolicy); + policyMap.put(OBSMethodNotAllowedException.class, failRetryPolicy); + policyMap.put(OBSFileConflictException.class, failRetryPolicy); + policyMap.put(EOFException.class, failRetryPolicy); + + policyMap.put(OBSQosException.class, throttleRetryPolicy); + policyMap.put(OBSIOException.class, idempotencyIoRetryPolicy); + return policyMap; + } + + @Override + public RetryAction shouldRetry(Exception exception, int retries, int failovers, boolean idempotent) + throws Exception { + RetryPolicy policy = exceptionToPolicyMap.get(exception.getClass()); + if (policy == null) { + policy = defaultPolicy; + } + return policy.shouldRetry(exception, retries, failovers, idempotent); + } + + @Override + public RetryAction shouldRetryByMaxTime(long startTime, Exception exception, int retries, int failovers, + boolean idempotent) throws Exception { + RetryPolicyWithMaxTime policy = exceptionToPolicyMap.get(exception.getClass()); + if (policy == null) { + policy = defaultPolicy; + } + return policy.shouldRetryByMaxTime(startTime, exception, retries, failovers, idempotent); + } + + @Override + public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers, boolean idempotent) + throws Exception { + RetryPolicyWithMaxTime policy = exceptionToPolicyMap.get(e.getClass()); + if (policy == null) { + policy = defaultPolicy; + } + return policy.shouldRetryByMaxTime(startTime, e, retries, failovers, idempotent); + } + + public static class IdempotencyIoRetryPolicy implements RetryPolicyWithMaxTime { + + private RetryPolicyWithMaxTime next; + + public IdempotencyIoRetryPolicy(RetryPolicyWithMaxTime retryPolicy) { + this.next = retryPolicy; + } + + @Override + public RetryAction shouldRetry(Exception e, int retries, int failovers, boolean idempotent) throws Exception { + return next.shouldRetry(e, retries, failovers, idempotent); + } + + @Override + public RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers, boolean idempotent) + throws Exception { + if (!(e instanceof IOException) || !idempotent) { + return RetryAction.FAIL; + } + return next.shouldRetryByMaxTime(startTime, e, retries, failovers, idempotent); + } + + @Override + public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers, boolean idempotent) + throws Exception { + return next.shouldRetryByMix(startTime, e, retries, failovers, idempotent); + } + } + + public static class ExponentialBackoffRetryPolicy implements RetryPolicyWithMaxTime { + private final int maxRetries; + private final long maxTime; + private final long sleepTime; + private final long sleepMaxTime; + + public ExponentialBackoffRetryPolicy(int maxRetries, long maxTime, long sleepTime, + long sleepMaxTime ) { + this.maxRetries = maxRetries; + this.maxTime = maxTime; + this.sleepTime = sleepTime; + this.sleepMaxTime = sleepMaxTime; + } + + private long calculateExponentialTime(int retries) { + long baseTime = Math.min(sleepTime * ((int) Math.pow(2.0D, (double) retries)), sleepMaxTime); + return (long)((double)baseTime * (ThreadLocalRandom.current().nextDouble() + 0.5D)); + } + + @Override + public RetryAction shouldRetry(Exception e, int retries, int failovers, boolean idempotent) { + return retries >= maxRetries ? + new RetryAction(RetryAction.RetryDecision.FAIL, 0 , "") : + new RetryAction(RetryAction.RetryDecision.RETRY, + TimeUnit.MILLISECONDS.toMillis(this.calculateExponentialTime(retries)), ""); + } + + @Override + public RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers, + boolean idempotent) { + return System.currentTimeMillis() - startTime > maxTime ? + new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "") : + new RetryAction(RetryAction.RetryDecision.RETRY, + TimeUnit.MILLISECONDS.toMillis(this.calculateExponentialTime(retries)), ""); + } + + @Override + public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers, + boolean idempotent) { + return retries >= maxRetries || (System.currentTimeMillis() - startTime > maxTime) ? + new RetryAction(RetryAction.RetryDecision.FAIL, 0 , "") : + new RetryAction(RetryAction.RetryDecision.RETRY, + TimeUnit.MILLISECONDS.toMillis(this.calculateExponentialTime(retries)), ""); + } + } + + public static class NoRetryPolicy implements RetryPolicyWithMaxTime { + @Override + public RetryAction shouldRetry(Exception e, int retries, int failovers, boolean idempotent) { + return new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "try once and fail."); + } + @Override + public RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers, + boolean idempotent) { + return new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "try once and fail."); + } + + @Override + public RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers, + boolean idempotent) { + return new RetryAction(RetryAction.RetryDecision.FAIL, 0L, "try once and fail."); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSSecurityProviderUtil.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSSecurityProviderUtil.java new file mode 100644 index 0000000..4713910 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSSecurityProviderUtil.java @@ -0,0 +1,114 @@ +package org.apache.hadoop.fs.obs; + +import com.obs.services.BasicObsCredentialsProvider; +import com.obs.services.IObsCredentialsProvider; + +import org.apache.hadoop.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.lang.reflect.Constructor; +import java.lang.reflect.InvocationTargetException; +import java.net.URI; +import java.util.Optional; + +public class OBSSecurityProviderUtil { + + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger(OBSSecurityProviderUtil.class); + + public static IObsCredentialsProvider createObsSecurityProvider(final Configuration conf, final URI name) + throws IOException { + Class credentialsProviderClass; + BasicSessionCredential credentialsProvider; + + try { + credentialsProviderClass = conf.getClass(OBSConstants.OBS_CREDENTIALS_PROVIDER, null); + } catch (RuntimeException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException("From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + c, c); + } + + if (credentialsProviderClass == null) { + return innerCreateObsSecurityProvider(conf, name); + } + + try { + Constructor cons = credentialsProviderClass.getDeclaredConstructor(URI.class, Configuration.class); + credentialsProvider = (BasicSessionCredential) cons.newInstance(name, conf); + } catch (NoSuchMethodException | SecurityException | IllegalAccessException | InstantiationException | InvocationTargetException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException("From option " + OBSConstants.OBS_CREDENTIALS_PROVIDER + ' ' + c, c); + } + + LOG.info("create ObsClient using credentialsProvider: {}", credentialsProviderClass.getName()); + String sessionToken = credentialsProvider.getSessionToken(); + String ak = credentialsProvider.getOBSAccessKeyId(); + String sk = credentialsProvider.getOBSSecretKey(); + if (sessionToken != null && sessionToken.length() != 0) { + return new BasicObsCredentialsProvider(ak, sk, sessionToken); + } else { + return new BasicObsCredentialsProvider(ak, sk); + } + } + + private static IObsCredentialsProvider innerCreateObsSecurityProvider(final Configuration conf, + final URI name) throws IOException { + OBSLoginHelper.Login creds = OBSCommonUtils.getOBSAccessKeys(name, conf); + + String ak = creds.getUser(); + String sk = creds.getPassword(); + String token = creds.getToken(); + + if (OBSCommonUtils.isStringNotEmpty(ak) || OBSCommonUtils.isStringNotEmpty(sk)) { + LOG.info("create ObsClient using aksk from configuration"); + return new BasicObsCredentialsProvider(ak, sk, token); + } + + Class securityProviderClass; + try { + securityProviderClass = conf.getClass(OBSConstants.OBS_SECURITY_PROVIDER, null); + LOG.info("From option {} get {}", OBSConstants.OBS_SECURITY_PROVIDER, securityProviderClass); + } catch (RuntimeException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException("From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, c); + } + + if (securityProviderClass == null) { + LOG.info("create ObsClient when securityProviderClass is null"); + return new BasicObsCredentialsProvider(ak, sk, token); + } + + LOG.info("create ObsClient using securityProvider {}", securityProviderClass.getName()); + IObsCredentialsProvider securityProvider; + try { + Optional> cons = tryGetConstructor(securityProviderClass, + new Class[] {URI.class, Configuration.class}); + + if (cons.isPresent()) { + securityProvider = (IObsCredentialsProvider) cons.get().newInstance(name, conf); + } else { + securityProvider = (IObsCredentialsProvider) securityProviderClass.getDeclaredConstructor() + .newInstance(); + } + + } catch (NoSuchMethodException | IllegalAccessException | InstantiationException | InvocationTargetException | RuntimeException e) { + Throwable c = e.getCause() != null ? e.getCause() : e; + throw new IOException("From option " + OBSConstants.OBS_SECURITY_PROVIDER + ' ' + c, c); + } + return securityProvider; + } + + private static Optional> tryGetConstructor(final Class mainClss, final Class[] args) { + try { + Constructor constructor = mainClss.getDeclaredConstructor(args); + return Optional.ofNullable(constructor); + } catch (NoSuchMethodException e) { + // ignore + return Optional.empty(); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java index fefab52..6d6b6eb 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OBSWriteOperationHelper.java @@ -35,6 +35,8 @@ import org.slf4j.LoggerFactory; import java.io.File; +import java.io.FileInputStream; +import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.util.ArrayList; @@ -62,7 +64,9 @@ class OBSWriteOperationHelper { /** * Class logger. */ - public static final Logger LOG = LoggerFactory.getLogger(OBSWriteOperationHelper.class); + private static final Logger LOG = LoggerFactory.getLogger(OBSWriteOperationHelper.class); + + static final String CONTENT_SHA256 = "x-obs-content-sha256"; /** * Part number of the multipart task. @@ -94,13 +98,19 @@ protected OBSWriteOperationHelper(final OBSFileSystem fs) { * Create a {@link PutObjectRequest} request. If {@code length} is set, the * metadata is configured with the size of the upload. * - * @param destKey key of object + * @param destKey key of object * @param inputStream source data - * @param length size, if known. Use -1 for not known + * @param length size, if known. Use -1 for not known + * @param type checksum type + * @param checksum checksum for source data * @return the request */ - PutObjectRequest newPutRequest(final String destKey, final InputStream inputStream, final long length) { - return OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), inputStream); + PutObjectRequest newPutRequest(final String destKey, final InputStream inputStream, final long length, + OBSDataBlocks.ChecksumType type, final String checksum) { + PutObjectRequest request = + OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), inputStream); + setPutObjectRequestChecksumInfo(request, type, checksum); + return request; } /** @@ -108,11 +118,31 @@ PutObjectRequest newPutRequest(final String destKey, final InputStream inputStre * * @param destKey object key for request * @param sourceFile source file + * @param type checksum type + * @param checksum checksum for source data * @return the request */ - PutObjectRequest newPutRequest(final String destKey, final File sourceFile) { + PutObjectRequest newPutRequest(final String destKey, final File sourceFile, OBSDataBlocks.ChecksumType type, + final String checksum) throws FileNotFoundException { int length = (int) sourceFile.length(); - return OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), sourceFile); + PutObjectRequest request = + OBSCommonUtils.newPutObjectRequest(owner, destKey, newObjectMetadata(length), sourceFile); + setPutObjectRequestChecksumInfo(request, type, checksum); + return request; + } + + private static void setPutObjectRequestChecksumInfo(PutObjectRequest request, OBSDataBlocks.ChecksumType type, + String checksum) { + switch (type) { + case MD5: + request.getMetadata().setContentMd5(checksum); + break; + case SHA256: + request.addUserHeaders(CONTENT_SHA256, checksum); + break; + default: + break; + } } /** @@ -152,11 +182,10 @@ String initiateMultiPartUpload(final String destKey) throws IOException { } else if (owner.getSse().isSseKmsEnable()) { initiateMPURequest.setSseKmsHeader(owner.getSse().getSseKmsHeader()); } - try { + return OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.initMultiPartUpload, + destKey, () -> { return obs.initiateMultipartUpload(initiateMPURequest).getUploadId(); - } catch (ObsException ace) { - throw OBSCommonUtils.translateException("Initiate MultiPartUpload", destKey, ace); - } + },true); } /** @@ -169,15 +198,18 @@ String initiateMultiPartUpload(final String destKey) throws IOException { * @throws ObsException on problems. */ CompleteMultipartUploadResult completeMultipartUpload(final String destKey, final String uploadId, - final List partETags) throws ObsException { + final List partETags) throws IOException { Preconditions.checkNotNull(uploadId); Preconditions.checkNotNull(partETags); Preconditions.checkArgument(!partETags.isEmpty(), "No partitions have been uploaded"); LOG.debug("Completing multipart upload {} with {} parts", uploadId, partETags.size()); // a copy of the list is required, so that the OBS SDK doesn't // attempt to sort an unmodifiable list. - return obs.completeMultipartUpload( - new CompleteMultipartUploadRequest(bucket, destKey, uploadId, new ArrayList<>(partETags))); + return OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.completeMultipartUpload, + destKey, () -> { + return obs.completeMultipartUpload( + new CompleteMultipartUploadRequest(bucket, destKey, uploadId, new ArrayList<>(partETags))); + },true); } /** @@ -195,47 +227,39 @@ void abortMultipartUpload(final String destKey, final String uploadId) throws Ob /** * Create request for uploading one part of a multipart task. * - * @param destKey destination object key - * @param uploadId upload id + * @param destKey destination object key + * @param uploadId upload id * @param partNumber part number - * @param size data size + * @param size data size * @param sourceFile source file to be uploaded + * @param type checksum type + * @param checksum checksum for source data * @return part upload request */ UploadPartRequest newUploadPartRequest(final String destKey, final String uploadId, final int partNumber, - final int size, final File sourceFile) { - Preconditions.checkNotNull(uploadId); - + final int size, final File sourceFile, OBSDataBlocks.ChecksumType type, final String checksum) + throws FileNotFoundException { Preconditions.checkArgument(sourceFile != null, "Data source"); - Preconditions.checkArgument(size > 0, "Invalid partition size %s", size); - Preconditions.checkArgument(partNumber > 0 && partNumber <= PART_NUMBER); - - LOG.debug("Creating part upload request for {} #{} size {}", uploadId, partNumber, size); - UploadPartRequest request = new UploadPartRequest(); - request.setUploadId(uploadId); - request.setBucketName(bucket); - request.setObjectKey(destKey); - request.setPartSize((long) size); - request.setPartNumber(partNumber); - request.setFile(sourceFile); - if (owner.getSse().isSseCEnable()) { - request.setSseCHeader(owner.getSse().getSseCHeader()); - } + UploadPartRequest request = + newUploadPartRequest(destKey, uploadId, partNumber, size, new FileInputStream(sourceFile), type, checksum); + request.setAutoClose(false); return request; } /** * Create request for uploading one part of a multipart task. * - * @param destKey destination object key - * @param uploadId upload id - * @param partNumber part number - * @param size data size + * @param destKey destination object key + * @param uploadId upload id + * @param partNumber part number + * @param size data size * @param uploadStream upload stream for the part + * @param type checksum type + * @param checksum checksum for source data * @return part upload request */ UploadPartRequest newUploadPartRequest(final String destKey, final String uploadId, final int partNumber, - final int size, final InputStream uploadStream) { + final int size, final InputStream uploadStream, final OBSDataBlocks.ChecksumType type, final String checksum) { Preconditions.checkNotNull(uploadId); Preconditions.checkArgument(uploadStream != null, "Data source"); @@ -253,6 +277,16 @@ UploadPartRequest newUploadPartRequest(final String destKey, final String upload if (owner.getSse().isSseCEnable()) { request.setSseCHeader(owner.getSse().getSseCHeader()); } + switch (type) { + case MD5: + request.setContentMd5(checksum); + break; + case SHA256: + request.addUserHeaders(CONTENT_SHA256, checksum); + break; + default: + break; + } return request; } @@ -268,10 +302,6 @@ public String toString(final String destKey) { * @throws IOException on problems */ PutObjectResult putObject(final PutObjectRequest putObjectRequest) throws IOException { - try { - return OBSCommonUtils.putObjectDirect(owner, putObjectRequest); - } catch (ObsException e) { - throw OBSCommonUtils.translateException("put", putObjectRequest.getObjectKey(), e); - } + return OBSCommonUtils.putObjectDirect(owner, putObjectRequest); } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OpenFileWithJobConf.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OpenFileWithJobConf.java new file mode 100644 index 0000000..b21551e --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/OpenFileWithJobConf.java @@ -0,0 +1,12 @@ +package org.apache.hadoop.fs.obs; + +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; + +public interface OpenFileWithJobConf { + FSDataInputStream open(Path f, Configuration jobConf) + throws IOException; +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java index 586a9a6..8024d46 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/Pair.java @@ -19,73 +19,24 @@ package org.apache.hadoop.fs.obs; public class Pair { - /** - * Key. - */ private final K key; - - /** - * Value. - */ private final V value; - - /** - * Create an entry representing a mapping from the specified key to the - * specified value. - * - * @param k Key (first element of the pair). - * @param v Value (second element of the pair). - */ public Pair(K k, V v) { key = k; value = v; } - - /** - * Get the key. - * - * @return the key (first element of the pair). - */ public K getKey() { return key; } - - /** - * Get the value. - * - * @return the value (second element of the pair). - */ public V getValue() { return value; } - - /** - * Get the first element of the pair. - * - * @return the first element of the pair. - * @since 3.1 - */ public K getFirst() { return key; } - - /** - * Get the second element of the pair. - * - * @return the second element of the pair. - * @since 3.1 - */ public V getSecond() { return value; } - - /** - * Compare the specified object with this entry for equality. - * - * @param o Object. - * @return {@code true} if the given object is also a map entry and - * the two entries represent the same mapping. - */ @Override public boolean equals(Object o) { if (this == o) { @@ -95,24 +46,15 @@ public boolean equals(Object o) { return false; } else { Pair oP = (Pair) o; - return (key == null ? oP.key == null : key.equals(oP.key)) && (value == null - ? oP.value == null - : value.equals(oP.value)); + return (key == null ? oP.key == null : key.equals(oP.key)) + && (value == null ? oP.value == null : value.equals(oP.value)); } } - - /** - * Compute a hash code. - * - * @return the hash code value. - */ @Override public int hashCode() { int result = key == null ? 0 : key.hashCode(); final int h = value == null ? 0 : value.hashCode(); - result = 37 * result + h ^ (h >>> 16); - - return result; + return 37 * result + h ^ (h >>> 16); } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RetryPolicyWithMaxTime.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RetryPolicyWithMaxTime.java new file mode 100644 index 0000000..96dc531 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/RetryPolicyWithMaxTime.java @@ -0,0 +1,17 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.io.retry.RetryPolicy; + +/** + * description + * + * @since 2022-05-10 + */ +public interface RetryPolicyWithMaxTime extends RetryPolicy { + RetryPolicy.RetryAction shouldRetryByMaxTime(long startTime, Exception e, int retries, int failovers, + boolean idempotent) + throws Exception; + RetryPolicy.RetryAction shouldRetryByMix(long startTime, Exception e, int retries, int failovers, + boolean idempotent) + throws Exception; +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java index 782d9a5..d745411 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/SemaphoredDelegatingExecutor.java @@ -86,41 +86,41 @@ protected ListeningExecutorService delegate() { return executorDelegatee; } + @NotNull + @Override + public ListenableFuture submit(@NotNull final Callable task) { + try { + queueingPermits.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return Futures.immediateFailedCheckedFuture(e); + } + return super.submit(new CallableWithPermitRelease<>(task)); + } + @NotNull @Override public List> invokeAll(@NotNull final Collection> tasks) { - throw new RuntimeException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @NotNull @Override public List> invokeAll(@NotNull final Collection> tasks, final long timeout, @NotNull final TimeUnit unit) { - throw new RuntimeException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @NotNull @Override public T invokeAny(@NotNull final Collection> tasks) { - throw new RuntimeException("Not implemented"); + throw new UnsupportedOperationException("Not implemented"); } @Override public T invokeAny(@NotNull final Collection> tasks, final long timeout, @NotNull final TimeUnit unit) { - throw new RuntimeException("Not implemented"); - } - - @NotNull - @Override - public ListenableFuture submit(@NotNull final Callable task) { - try { - queueingPermits.acquire(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return Futures.immediateFailedCheckedFuture(e); - } - return super.submit(new CallableWithPermitRelease<>(task)); + throw new UnsupportedOperationException("Not implemented"); } @NotNull @@ -135,16 +135,8 @@ public ListenableFuture submit(@NotNull final Runnable task, @NotNull fin return super.submit(new RunnableWithPermitRelease(task), result); } - @NotNull - @Override - public ListenableFuture submit(@NotNull final Runnable task) { - try { - queueingPermits.acquire(); - } catch (InterruptedException e) { - Thread.currentThread().interrupt(); - return Futures.immediateFailedCheckedFuture(e); - } - return super.submit(new RunnableWithPermitRelease(task)); + public int getAvailablePermits() { + return queueingPermits.availablePermits(); } @Override @@ -157,34 +149,26 @@ public void execute(@NotNull final Runnable command) { super.execute(new RunnableWithPermitRelease(command)); } - /** - * Get the number of permits available; guaranteed to be {@code 0 <= - * availablePermits <= size}. - * - * @return the number of permits available at the time of invocation. - */ - public int getAvailablePermits() { - return queueingPermits.availablePermits(); - } - - /** - * Get the number of threads waiting to acquire a permit. - * - * @return snapshot of the length of the queue of blocked threads. - */ public int getWaitingCount() { return queueingPermits.getQueueLength(); } - /** - * Total number of permits. - * - * @return the number of permits as set in the constructor - */ public int getPermitCount() { return permitCount; } + @NotNull + @Override + public ListenableFuture submit(@NotNull final Runnable task) { + try { + queueingPermits.acquire(); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + return Futures.immediateFailedCheckedFuture(e); + } + return super.submit(new RunnableWithPermitRelease(task)); + } + @Override public String toString() { return "SemaphoredDelegatingExecutor{" + "permitCount=" + getPermitCount() + ", available=" diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatistics.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatistics.java new file mode 100644 index 0000000..2135c03 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatistics.java @@ -0,0 +1,94 @@ +package org.apache.hadoop.fs.obs; + +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.atomic.AtomicLong; +import java.util.function.LongUnaryOperator; + +public class TrafficStatistics { + // originalTraffic: Q + private final AtomicLong originalTraffic = new AtomicLong(); + + /** + * ApplicationTraffic: Q`. + *

In fact Q` = Q1 + Q2. One should not operate the value of Q` directly. + *

The value of Q` shall only be determined as Q1 + Q2. + */ + private final AtomicLong applicationTraffic = new AtomicLong(); + + /** + * missTraffic: Q1 + */ + private final AtomicLong missTraffic = new AtomicLong(); + + /** + * hitTraffic: Q2 + */ + private final AtomicLong hitTraffic = new AtomicLong(); + + private static final Logger LOG = LoggerFactory.getLogger(TrafficStatistics.class); + + public enum TrafficType { + Q, + QDot, + Q1, + Q2 + } + + public void increase(long val, TrafficType type) { + if (val > 0) { + long now; + switch (type) { + case Q: + now = originalTraffic.updateAndGet(addWithinLimit(val)); + LOG.debug("originalTraffic(Q) added {}, now {}.", val, now); + break; + case Q1: + now = missTraffic.updateAndGet(addWithinLimit(val)); + LOG.debug("missTraffic(Q1) added {}, now {}.", val, now); + break; + case Q2: + now = hitTraffic.updateAndGet(addWithinLimit(val)); + LOG.debug("hitTraffic(Q2) added {}, now {}.", val, now); + break; + default: + LOG.error("Wrong type of TrafficType, val {}.", val); + } + } + } + + private LongUnaryOperator addWithinLimit(long val) { + return (x) -> { + if (x >= Long.MAX_VALUE) { + return 0; + } else { + return x + val; + } + }; + } + + public long getStatistics(TrafficType type) { + switch (type) { + case Q: + return originalTraffic.get(); + case QDot: + return missTraffic.get() + hitTraffic.get(); + case Q1: + return missTraffic.get(); + case Q2: + return hitTraffic.get(); + default: + LOG.error("Wrong type of TrafficType."); + return -1; + } + } + + public void clearStatistics() { + originalTraffic.getAndSet(0); + applicationTraffic.getAndSet(0); + missTraffic.getAndSet(0); + hitTraffic.getAndSet(0); + LOG.debug("Cleared all traffic statistics."); + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatisticsReporter.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatisticsReporter.java new file mode 100644 index 0000000..5fadeff --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/TrafficStatisticsReporter.java @@ -0,0 +1,95 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.fs.obs.memartscc.MemArtsCCClient; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.ScheduledExecutorService; +import java.util.concurrent.ScheduledFuture; +import java.util.concurrent.ScheduledThreadPoolExecutor; +import java.util.concurrent.TimeUnit; + +import static org.apache.hadoop.fs.obs.BlockingThreadPoolExecutorService.newDaemonThreadFactory; + +public class TrafficStatisticsReporter { + private static final Logger LOG = LoggerFactory.getLogger(TrafficStatisticsReporter.class); + + private final TrafficStatistics trafficStatistics; + + private final MemArtsCCClient memArtsCCClient; + + /** + * Time interval of the report + */ + private final long interval; + + /** + * MemArtsCC traffic report Thread pool & Schedule + */ + private ScheduledExecutorService reportPool; + + private ScheduledFuture reportSchedule; + + public TrafficStatisticsReporter(TrafficStatistics trafficStatistics, + MemArtsCCClient memArtsCCClient, long interval) { + this.trafficStatistics = trafficStatistics; + this.memArtsCCClient = memArtsCCClient; + this.interval = interval; + } + + public void startReport() { + initThreadPool(); + initReportSchedule(); + } + + private void initThreadPool() { + reportPool = new ScheduledThreadPoolExecutor(1, + newDaemonThreadFactory("obs-traffic-statistics-report")); + } + + private void initReportSchedule() { + reportSchedule = reportPool.scheduleAtFixedRate( + this::reportTraffic, interval, interval, TimeUnit.SECONDS); + } + + private void reportTraffic() { + if (memArtsCCClient == null || trafficStatistics == null) { + if (memArtsCCClient == null) { + LOG.debug("memArtsCCClient is null, statistics cannot be reported."); + } + if (trafficStatistics == null) { + LOG.debug("trafficStatistics is null, statistics cannot be reported."); + } + return; + } + + memArtsCCClient.reportReadStatistics(trafficStatistics); + LOG.debug("Statistics has been reported: Q:{} Q`:{} Q2:{} Q1:{}", + trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q), + trafficStatistics.getStatistics(TrafficStatistics.TrafficType.QDot), + trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q2), + trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q1)); + + trafficStatistics.clearStatistics(); + } + + public void shutdownReport() { + if (memArtsCCClient == null || trafficStatistics == null) { + return; + } + reportTraffic(); + // cancel report schedule + try { + reportSchedule.cancel(true); + if (reportSchedule.isCancelled()) { + LOG.debug("TrafficStatistics reportSchedule is canceled."); + } + reportPool.shutdownNow(); + if (reportPool.isShutdown()) { + LOG.debug("TrafficStatistics reportPool is shutdowned."); + } + } catch (Exception e) { + LOG.debug("Exception occurred when canceling scheduledFuture"); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/WithErrCode.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/WithErrCode.java new file mode 100644 index 0000000..e2f43e0 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/WithErrCode.java @@ -0,0 +1,5 @@ +package org.apache.hadoop.fs.obs; + +public interface WithErrCode { + String getErrCode(); +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java index 408d09c..a895027 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/BasicInputPolicyFactory.java @@ -6,6 +6,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.obs.OBSCommonUtils; import org.apache.hadoop.fs.obs.OBSConstants; +import org.apache.hadoop.fs.obs.OBSFileStatus; import org.apache.hadoop.fs.obs.OBSFileSystem; /** @@ -17,7 +18,7 @@ public class BasicInputPolicyFactory implements InputPolicyFactory { @Override public FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength, - FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool) { + FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus) { long readAheadRange = OBSCommonUtils.longBytesOption(obsFileSystem.getConf(), OBSConstants.READAHEAD_RANGE, OBSConstants.DEFAULT_READAHEAD_RANGE, 0); return new OBSInputStream(bucket, key, contentLength, obsFileSystem.getObsClient(), statistics, readAheadRange, diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java index 0bae43c..71d336e 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ExtendInputPolicyFactory.java @@ -6,6 +6,7 @@ import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.obs.OBSCommonUtils; import org.apache.hadoop.fs.obs.OBSConstants; +import org.apache.hadoop.fs.obs.OBSFileStatus; import org.apache.hadoop.fs.obs.OBSFileSystem; import org.apache.hadoop.fs.obs.SemaphoredDelegatingExecutor; @@ -28,7 +29,7 @@ public class ExtendInputPolicyFactory implements InputPolicyFactory { */ @Override public FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength, - FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool) { + FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus) { int maxReadAhead = OBSCommonUtils.intOption(obsFileSystem.getConf(), OBSConstants.READAHEAD_MAX_NUM, OBSConstants.DEFAULT_READAHEAD_MAX_NUM, 1); diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java index 374930c..ac6eca7 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicyFactory.java @@ -4,6 +4,7 @@ import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSFileStatus; import org.apache.hadoop.fs.obs.OBSFileSystem; /** @@ -13,5 +14,5 @@ */ public interface InputPolicyFactory { FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength, - FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool); + FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus); } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java index 1b5b73d..7fbbc4e 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/InputPolicys.java @@ -21,6 +21,8 @@ public static InputPolicyFactory createFactory(final String name) { return new BasicInputPolicyFactory(); case OBSConstants.READAHEAD_POLICY_ADVANCE: return new ExtendInputPolicyFactory(); + case OBSConstants.READAHEAD_POLICY_MEMARTSCC: + return new MemArtsCCInputPolicyFactory(); default: throw new IllegalArgumentException("Unsupported block buffer" + " \"" + name + '"'); } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/MemArtsCCInputPolicyFactory.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/MemArtsCCInputPolicyFactory.java new file mode 100644 index 0000000..736e6f4 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/MemArtsCCInputPolicyFactory.java @@ -0,0 +1,29 @@ +package org.apache.hadoop.fs.obs.input; + +import com.google.common.util.concurrent.ListeningExecutorService; + +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSCommonUtils; +import org.apache.hadoop.fs.obs.OBSConstants; +import org.apache.hadoop.fs.obs.OBSFileStatus; +import org.apache.hadoop.fs.obs.OBSFileSystem; + +/** + * 功能描述 + * + * @since 2021-05-19 + */ +public class MemArtsCCInputPolicyFactory implements InputPolicyFactory { + + @Override + public FSInputStream create(final OBSFileSystem obsFileSystem, String bucket, String key, Long contentLength, + FileSystem.Statistics statistics, ListeningExecutorService boundedThreadPool, OBSFileStatus fileStatus) { + long readAheadRange = OBSCommonUtils.longBytesOption(obsFileSystem.getConf(), OBSConstants.READAHEAD_RANGE, + OBSConstants.DEFAULT_READAHEAD_RANGE, 0); + long memartsccReadAheadRangeValue = OBSCommonUtils.longBytesOption(obsFileSystem.getConf(), + OBSConstants.MEMARTSCC_READAHEAD_RANGE, OBSConstants.DEFAULT_MEMARTSCC_READAHEAD_RANGE, 0); + return new OBSMemArtsCCInputStream(bucket, key, contentLength, statistics, readAheadRange, + memartsccReadAheadRangeValue, obsFileSystem, fileStatus); + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java index 73c4fae..be77f77 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSExtendInputStream.java @@ -28,7 +28,7 @@ * stream. */ public class OBSExtendInputStream extends FSInputStream implements CanSetReadahead, ByteBufferReadable { - public static final Logger LOG = LoggerFactory.getLogger(OBSExtendInputStream.class); + private static final Logger LOG = LoggerFactory.getLogger(OBSExtendInputStream.class); private OBSFileSystem fs; @@ -218,9 +218,7 @@ private void checkNotClosed() throws IOException { } private void validateReadArgs(byte[] buf, int off, int len) { - if (buf == null) { - throw new NullPointerException(); - } else if (off < 0 || len < 0 || len > buf.length - off) { + if (off < 0 || len < 0 || len > buf.length - off) { throw new IndexOutOfBoundsException(); } } diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java index c840653..a036401 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSInputStream.java @@ -6,7 +6,6 @@ import com.obs.services.model.GetObjectRequest; import com.sun.istack.NotNull; -import org.apache.commons.lang.StringUtils; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.ByteBufferReadable; @@ -14,11 +13,10 @@ import org.apache.hadoop.fs.FSExceptionMessages; import org.apache.hadoop.fs.FSInputStream; import org.apache.hadoop.fs.FileSystem; -import org.apache.hadoop.fs.obs.BasicMetricsConsumer; import org.apache.hadoop.fs.obs.OBSCommonUtils; import org.apache.hadoop.fs.obs.OBSConstants; import org.apache.hadoop.fs.obs.OBSFileSystem; -import org.apache.hadoop.fs.obs.OBSIOException; +import org.apache.hadoop.fs.obs.OBSOperateAction; import org.slf4j.Logger; import org.slf4j.LoggerFactory; @@ -51,7 +49,7 @@ public class OBSInputStream extends FSInputStream implements CanSetReadahead, By /** * Class logger. */ - public static final Logger LOG = LoggerFactory.getLogger(OBSInputStream.class); + private static final Logger LOG = LoggerFactory.getLogger(OBSInputStream.class); /** * The statistics for OBS file system. @@ -130,8 +128,8 @@ public class OBSInputStream extends FSInputStream implements CanSetReadahead, By OBSInputStream(final String bucketName, final String bucketKey, final long fileStatusLength, final ObsClient obsClient, final FileSystem.Statistics stats, final long readAheadRangeValue, final OBSFileSystem obsFileSystem) { - Preconditions.checkArgument(StringUtils.isNotEmpty(bucketName), "No Bucket"); - Preconditions.checkArgument(StringUtils.isNotEmpty(bucketKey), "No Key"); + Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(bucketName), "No Bucket"); + Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(bucketKey), "No Key"); Preconditions.checkArgument(fileStatusLength >= 0, "Negative content length"); this.bucket = bucketName; this.key = bucketKey; @@ -160,6 +158,11 @@ static long calculateRequestLimit(final long targetPos, final long length, final return Math.min(contentLength, length < 0 ? contentLength : targetPos + Math.max(readahead, length)); } + protected long calculateOBSTraffic(final long targetPos, final long length) { + long contentRangeEnd = Math.min(contentLength, length < 0 ? contentLength : targetPos + Math.max(readAheadRange, length)); + return contentRangeEnd - targetPos; + } + /** * Opens up the stream at specified target position and for given length. * @@ -168,7 +171,8 @@ static long calculateRequestLimit(final long targetPos, final long length, final * @param length length requested * @throws IOException on any failure to open the object */ - private synchronized void reopen(final String reason, final long targetPos, final long length) throws IOException { + protected synchronized void reopen(final String reason, final long targetPos, final long length) + throws IOException { long startTime = System.currentTimeMillis(); long threadId = Thread.currentThread().getId(); if (wrappedStream != null) { @@ -211,17 +215,21 @@ public synchronized long getPos() throws IOException { public synchronized void seek(final long targetPos) throws IOException { fs.checkOpen(); checkStreamOpen(); - - // Do not allow negative seek if (targetPos < 0) { - throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + targetPos); + EOFException eof = new EOFException(String.format("%s %s", FSExceptionMessages.NEGATIVE_SEEK, targetPos)); + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.lazySeek, eof); + throw eof; + } + + if (targetPos > contentLength) { + EOFException eof = new EOFException(String.format("%s %s", FSExceptionMessages.CANNOT_SEEK_PAST_EOF, targetPos)); + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.lazySeek, eof); + throw eof; } if (this.contentLength <= 0) { return; } - - // Lazy seek nextReadPos = targetPos; } @@ -250,22 +258,14 @@ private void seekInStream(final long targetPos) throws IOException { if (wrappedStream == null) { return; } - // compute how much more to skip long diff = targetPos - streamCurrentPos; if (diff > 0) { - // forward seek -this is where data can be skipped - int available = wrappedStream.available(); - // always seek at least as far as what is available long forwardSeekRange = Math.max(readAheadRange, available); - // work out how much is actually left in the stream - // then choose whichever comes first: the range or the EOF long remainingInCurrentRequest = remainingInCurrentRequest(); - long forwardSeekLimit = Math.min(remainingInCurrentRequest, forwardSeekRange); boolean skipForward = remainingInCurrentRequest > 0 && diff <= forwardSeekLimit; if (skipForward) { - // the forward seek range is within the limits LOG.debug("Forward seek on {}, of {} bytes", uri, diff); long skippedOnce = wrappedStream.skip(diff); while (diff > 0 && skippedOnce > 0) { @@ -276,22 +276,14 @@ private void seekInStream(final long targetPos) throws IOException { } if (streamCurrentPos == targetPos) { - // all is well return; } else { - // log a warning; continue to attempt to re-open LOG.info("Failed to seek on {} to {}. Current position {}", uri, targetPos, streamCurrentPos); } } } else if (diff == 0 && remainingInCurrentRequest() > 0) { - // targetPos == streamCurrentPos - // if there is data left in the stream, keep going return; } - - // if the code reaches here, the stream needs to be reopened. - // close the stream; if read the object will be opened at the - // new streamCurrentPos closeStream("seekInStream()", this.contentRangeFinish); streamCurrentPos = targetPos; } @@ -311,58 +303,19 @@ public boolean seekToNewSource(final long targetPos) throws IOException { * @throws IOException on any failure to lazy seek */ private void lazySeek(final long targetPos, final long len) throws IOException { - int retryTime = 0; - long delayMs; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.lazySeek, key, () -> { try { - // For lazy seek seekInStream(targetPos); } catch (IOException e) { if (wrappedStream != null) { closeStream("lazySeek() seekInStream has exception ", this.contentRangeFinish); } - - LOG.warn("IOException occurred in lazySeek, retry: {}", retryTime, e); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw e; - } - } else { - throw e; - } - - continue; } - - try { - // re-open at specific location if needed - if (wrappedStream == null) { - reopen("read from new offset", targetPos, len); - } - - return; - } catch (OBSIOException e) { - LOG.debug("IOException occurred in lazySeek, retry: {}", retryTime, e); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - throw e; - } - } else { - throw e; - } + if (wrappedStream == null) { + reopen("read from new offset", targetPos, len); } - } + return null; + },true); } /** @@ -377,15 +330,6 @@ private void incrementBytesRead(final long bytesRead) { } } - private void sleepInLock(long sleepTime) throws InterruptedException { - long start = System.currentTimeMillis(); - long now = start; - while (now - start < sleepTime) { - wait(start + sleepTime - now); - now = System.currentTimeMillis(); - } - } - @Override public synchronized int read() throws IOException { fs.checkOpen(); @@ -393,82 +337,55 @@ public synchronized int read() throws IOException { long startTime = System.currentTimeMillis(); long threadId = Thread.currentThread().getId(); long endTime; - boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength; + boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength; if (isTrue) { return -1; } - int byteRead = -1; try { lazySeek(nextReadPos, 1); } catch (EOFException e) { onReadFailure(e, 1); return -1; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readOneByte,e); + throw e; } - IOException exception = null; - int retryTime = 0; - long delayMs; - long retryStartTime = System.currentTimeMillis(); - do { - try { - byteRead = wrappedStream.read(); - exception = null; - break; - } catch (EOFException e) { - onReadFailure(e, 1); - return -1; - } catch (IOException e) { - exception = e; - onReadFailure(e, 1); - LOG.debug("read of [{}] failed, retry time[{}], due to exception[{}]", uri, retryTime, e); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - sleepInLock(delayMs); - } catch (InterruptedException ie) { - LOG.error("read of [{}] failed, retry time[{}], due to " + "exception[{}]", uri, retryTime, e); - throw e; - } + try { + int byteRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readOneByte, key, () -> { + int b; + try { + b = wrappedStream.read(); + } catch (EOFException e) { + onReadFailure(e, 1); + return -1; + } catch (IOException e) { + onReadFailure(e, 1); + throw e; } - } - } while (System.currentTimeMillis() - retryStartTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY); + return b; + }, true); - if (exception != null) { - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.BYTEBUF, BasicMetricsConsumer.MetricRecord.READ, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); + if (byteRead >= 0) { + streamCurrentPos++; + nextReadPos++; } - LOG.error("read of [{}] failed, retry time[{}], due to exception[{}]", uri, retryTime, exception); - throw exception; - } - - if (byteRead >= 0) { - streamCurrentPos++; - nextReadPos++; - } - if (byteRead >= 0) { - incrementBytesRead(1); - } + if (byteRead >= 0) { + incrementBytesRead(1); + } - endTime = System.currentTimeMillis(); - long position = byteRead >= 0 ? nextReadPos - 1 : nextReadPos; - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.ONEBYTE, BasicMetricsConsumer.MetricRecord.READ, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); + endTime = System.currentTimeMillis(); + long position = byteRead >= 0 ? nextReadPos - 1 : nextReadPos; + LOG.debug("read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, " + "thread:{}, timeUsedMilliSec:{}", + uri, contentLength, position, byteRead, threadId, endTime - startTime); + return byteRead; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readOneByte,e); + throw e; } - LOG.debug("read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, " + "thread:{}, timeUsedMilliSec:{}", - uri, contentLength, position, byteRead, threadId, endTime - startTime); - return byteRead; } /** @@ -481,27 +398,10 @@ public synchronized int read() throws IOException { */ private synchronized void onReadFailure(final IOException ioe, final int length) throws IOException { LOG.debug("Got exception while trying to read from stream {}" + " trying to recover: " + ioe, uri); - int retryTime = 0; - long delayMs; - long startTime = System.currentTimeMillis(); - while (System.currentTimeMillis() - startTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - reopen("failure recovery", streamCurrentPos, length); - return; - } catch (OBSIOException e) { - LOG.debug("OBSIOException occurred in reopen for failure recovery, " + "the {} retry time", retryTime, - e); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - try { - sleepInLock(delayMs); - } catch (InterruptedException ie) { - throw e; - } - } - } - - reopen("failure recovery", streamCurrentPos, length); + OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.onReadFailure, key, () -> { + reopen("failure recovery", streamCurrentPos, length); + return null; + },true); } @Override @@ -517,7 +417,6 @@ public synchronized int read(final ByteBuffer byteBuffer) throws IOException { if (len == 0) { return 0; } - byte[] buf = new byte[len]; boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength; if (isTrue) { @@ -530,76 +429,46 @@ public synchronized int read(final ByteBuffer byteBuffer) throws IOException { onReadFailure(e, len); // the end of the file has moved return -1; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readByteBuff,e); + throw e; } - int bytesRead = 0; - IOException exception = null; - int retryTime = 0; - long delayMs; - long startRetryTime = System.currentTimeMillis(); - do { - try { - bytesRead = tryToReadFromInputStream(wrappedStream, buf, 0, len); - if (bytesRead == -1) { - return -1; - } - exception = null; - break; - } catch (EOFException e) { - onReadFailure(e, len); - return -1; - } catch (IOException e) { - exception = e; - onReadFailure(e, len); - LOG.debug("read len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", len, uri, retryTime, - exception); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - sleepInLock(delayMs); - } catch (InterruptedException ie) { - LOG.error("read len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", len, uri, - retryTime, exception); - throw exception; + try { + int bytesRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readByteBuff, key, () -> { + int count; + try { + count = tryToReadFromInputStream(wrappedStream, buf, 0, len); + if (count == -1) { + return -1; } + } catch (EOFException e) { + onReadFailure(e, len); + return -1; + } catch (IOException e) { + onReadFailure(e, len); + throw e; } + return count; + }, true); + + if (bytesRead > 0) { + streamCurrentPos += bytesRead; + nextReadPos += bytesRead; + byteBuffer.put(buf, 0, bytesRead); } - } while (System.currentTimeMillis() - startRetryTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY); - if (exception != null) { + incrementBytesRead(bytesRead); + long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos; endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.BYTEBUF, BasicMetricsConsumer.MetricRecord.READ, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } - - LOG.error("read len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", len, uri, retryTime, - exception); - throw exception; - } - if (bytesRead > 0) { - streamCurrentPos += bytesRead; - nextReadPos += bytesRead; - byteBuffer.put(buf, 0, bytesRead); - } - incrementBytesRead(bytesRead); - long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos; - endTime = System.currentTimeMillis(); - - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.BYTEBUF, BasicMetricsConsumer.MetricRecord.READ, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); + LOG.debug("Read-ByteBuffer uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId, + endTime - startTime); + return bytesRead; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readByteBuff,e); + throw e; } - LOG.debug("Read-ByteBuffer uri:{}, contentLength:{}, destLen:{}, readLen:{}, " - + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId, - endTime - startTime); - return bytesRead; } private int tryToReadFromInputStream(final InputStream in, final byte[] buf, final int off, final int len) @@ -635,13 +504,10 @@ public synchronized int read(@NotNull final byte[] buf, final int off, final int checkStreamOpen(); long startTime = System.currentTimeMillis(); long threadId = Thread.currentThread().getId(); - long endTime; - validatePositionedReadArgs(nextReadPos, buf, off, len); if (len == 0) { return 0; } - boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength; if (isTrue) { return -1; @@ -653,81 +519,45 @@ public synchronized int read(@NotNull final byte[] buf, final int off, final int onReadFailure(e, len); // the end of the file has moved return -1; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readbytes,e); + throw e; } - int bytesRead = 0; - IOException exception = null; - int retryTime = 0; - long delayMs; - long startRetryTime = System.currentTimeMillis(); - do { - try { - bytesRead = tryToReadFromInputStream(wrappedStream, buf, off, len); - if (bytesRead == -1) { - return -1; - } - exception = null; - break; - } catch (EOFException e) { - onReadFailure(e, len); - return -1; - } catch (IOException e) { - exception = e; - onReadFailure(e, len); - LOG.debug("read offset[{}] len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", off, len, - uri, retryTime, exception); - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs - < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - sleepInLock(delayMs); - } catch (InterruptedException ie) { - LOG.error("read offset[{}] len[{}] of [{}] failed, " + "retry time[{}], due to exception[{}]", - off, len, uri, retryTime, exception); - throw exception; + try { + int bytesRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readbytes, key, () -> { + int count; + try { + count = tryToReadFromInputStream(wrappedStream, buf, off, len); + if (count == -1) { + return -1; } + } catch (EOFException e) { + onReadFailure(e, len); + return -1; + } catch (IOException e) { + onReadFailure(e, len); + throw e; } - } - } while (System.currentTimeMillis() - startRetryTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY); - - long costTime; + return count; + }, true); - if (exception != null) { - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.SEQ, BasicMetricsConsumer.MetricRecord.READ, false, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); + if (bytesRead > 0) { + streamCurrentPos += bytesRead; + nextReadPos += bytesRead; } - LOG.error("read offset[{}] len[{}] of [{}] failed, retry time[{}], " + "due to exception[{}]", off, len, - uri, retryTime, exception); - throw exception; - } - - if (bytesRead > 0) { - streamCurrentPos += bytesRead; - nextReadPos += bytesRead; - } - incrementBytesRead(bytesRead); - - endTime = System.currentTimeMillis(); - costTime = endTime - startTime; - readMetric(costTime); + incrementBytesRead(bytesRead); - long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos; - LOG.debug("Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, " - + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId, - endTime - startTime); - return bytesRead; - } + long endTime = System.currentTimeMillis(); - private void readMetric(long costTime) { - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.SEQ, BasicMetricsConsumer.MetricRecord.READ, true, costTime); - OBSCommonUtils.setMetricsInfo(fs, record); + long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos; + LOG.debug("Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId, + endTime - startTime); + return bytesRead; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readbytes,e); + throw e; } } @@ -753,7 +583,6 @@ private void checkStreamOpen() throws IOException { */ @Override public synchronized void close() throws IOException { - long startTime = System.currentTimeMillis(); if (!closed) { fs.checkOpen(); // close or abort the stream @@ -762,14 +591,6 @@ public synchronized void close() throws IOException { super.close(); closed = true; } - - long endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.INPUT, BasicMetricsConsumer.MetricRecord.CLOSE, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } } /** @@ -783,7 +604,7 @@ public synchronized void close() throws IOException { * @param length length of the stream * @throws IOException on any failure to close stream */ - private synchronized void closeStream(final String reason, final long length) throws IOException { + protected synchronized void closeStream(final String reason, final long length) throws IOException { if (wrappedStream != null) { try { wrappedStream.close(); @@ -804,7 +625,7 @@ public synchronized int available() throws IOException { fs.checkOpen(); checkStreamOpen(); - long remaining = remainingInFile(); + long remaining = remainingInStream(); if (remaining > Integer.MAX_VALUE) { return Integer.MAX_VALUE; } @@ -818,7 +639,7 @@ public synchronized int available() throws IOException { */ @InterfaceAudience.Private @InterfaceStability.Unstable - private synchronized long remainingInFile() { + private synchronized long remainingInStream() { return this.contentLength - this.streamCurrentPos; } @@ -869,40 +690,40 @@ public String toString() { @Override public void readFully(final long position, final byte[] buffer, final int offset, final int length) throws IOException { - fs.checkOpen(); - checkStreamOpen(); - long startTime = System.currentTimeMillis(); - long threadId = Thread.currentThread().getId(); + try { + fs.checkOpen(); + checkStreamOpen(); + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); - validatePositionedReadArgs(position, buffer, offset, length); - if (length == 0) { - return; - } - int nread = 0; - synchronized (this) { - long oldPos = getPos(); - try { - seek(position); - while (nread < length) { - int nbytes = read(buffer, offset + nread, length - nread); - if (nbytes < 0) { - throw new EOFException(FSExceptionMessages.EOF_IN_READ_FULLY); + validatePositionedReadArgs(position, buffer, offset, length); + if (length == 0) { + return; + } + int bytesRead = 0; + synchronized (this) { + long oldPos = getPos(); + try { + seek(position); + while (bytesRead < length) { + int bRead = read(buffer, offset + bytesRead, length - bytesRead); + if (bRead < 0) { + throw new EOFException(FSExceptionMessages.EOF_IN_READ_FULLY); + } + bytesRead += bRead; } - nread += nbytes; + } finally { + seekQuietly(oldPos); } - } finally { - seekQuietly(oldPos); } + long endTime = System.currentTimeMillis(); + LOG.debug("ReadFully uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, length, + bytesRead, position, threadId, endTime - startTime); + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readfully,e); + throw e; } - long endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord(null, - BasicMetricsConsumer.MetricRecord.READFULLY, true, endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } - LOG.debug("ReadFully uri:{}, contentLength:{}, destLen:{}, readLen:{}, " - + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, length, nread, position, threadId, - endTime - startTime); } /** @@ -920,121 +741,59 @@ public int read(final long position, final byte[] buffer, final int offset, fina fs.checkOpen(); checkStreamOpen(); int len = length; - long startTime = System.currentTimeMillis(); - long endTime; - int readSize; - validatePositionedReadArgs(position, buffer, offset, len); - if (position < 0 || position >= contentLength) { - return -1; - } - if ((position + len) > contentLength) { - len = (int) (contentLength - position); - } + try { + validatePositionedReadArgs(position, buffer, offset, len); + if (position < 0 || position >= contentLength) { + return -1; + } + if ((position + len) > contentLength) { + len = (int) (contentLength - position); + } - if (fs.isReadTransformEnabled()) { - readSize = super.read(position, buffer, offset, len); - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.RANDOM, BasicMetricsConsumer.MetricRecord.READ, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); + if (fs.isReadTransformEnabled()) { + readSize = super.read(position, buffer, offset, len); + return readSize; } + readSize = randomReadWithNewInputStream(position, buffer, offset, len); return readSize; + } catch (IOException e) { + OBSCommonUtils.setMetricsAbnormalInfo(fs, OBSOperateAction.readrandom,e); + throw e; } - readSize = randomReadWithNewInputStream(position, buffer, offset, len); - endTime = System.currentTimeMillis(); - if (fs.getMetricSwitch()) { - BasicMetricsConsumer.MetricRecord record = new BasicMetricsConsumer.MetricRecord( - BasicMetricsConsumer.MetricRecord.RANDOM, BasicMetricsConsumer.MetricRecord.READ, true, - endTime - startTime); - OBSCommonUtils.setMetricsInfo(fs, record); - } - return readSize; } private int randomReadWithNewInputStream(final long position, final byte[] buffer, final int offset, final int length) throws IOException { long startTime = System.currentTimeMillis(); long threadId = Thread.currentThread().getId(); - int bytesRead = 0; - InputStream inputStream = null; - IOException exception = null; - GetObjectRequest request = new GetObjectRequest(bucket, key); - request.setRangeStart(position); - request.setRangeEnd(position + length); - if (fs.getSse().isSseCEnable()) { - request.setSseCHeader(fs.getSse().getSseCHeader()); - } - int retryTime = 0; - long delayMs; - long startRetryTime = System.currentTimeMillis(); - do { - exception = null; + int bytesRead = OBSCommonUtils.getOBSInvoker().retryByMaxTime(OBSOperateAction.readrandom, key, () -> { + int count; + InputStream inputStream = null; try { - inputStream = client.getObject(request).getObjectContent(); - } catch (ObsException e) { - exception = OBSCommonUtils.translateException("Read at position " + position, uri, e); - - LOG.debug( - "read position[{}] destLen[{}] offset[{}] readLen[{}] " + "of [{}] failed, retry time[{}], due to " - + "exception[{}]", position, length, offset, bytesRead, uri, retryTime, exception); - - if (!(exception instanceof OBSIOException)) { - throw exception; + GetObjectRequest request = new GetObjectRequest(bucket, key); + request.setRangeStart(position); + request.setRangeEnd(position + length); + if (fs.getSse().isSseCEnable()) { + request.setSseCHeader(fs.getSse().getSseCHeader()); } - } - - if (exception == null) { - try { - bytesRead = tryToReadFromInputStream(inputStream, buffer, offset, length); - if (bytesRead == -1) { - return -1; - } - - exception = null; - break; - } catch (EOFException e) { - onReadFailure(e, length); + inputStream = client.getObject(request).getObjectContent(); + count = tryToReadFromInputStream(inputStream, buffer, offset, length); + if (count == -1) { return -1; - } catch (IOException e) { - exception = e; - - LOG.debug("read position[{}] destLen[{}] offset[{}] readLen[{}] " - + "of [{}] failed, retry time[{}], due to " + "exception[{}]", position, length, offset, - bytesRead, uri, retryTime, exception); - } finally { - if (inputStream != null) { - inputStream.close(); - } } - } - - delayMs = OBSCommonUtils.getSleepTimeInMs(retryTime); - retryTime++; - if (System.currentTimeMillis() - startTime + delayMs < OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY) { - try { - Thread.sleep(delayMs); - } catch (InterruptedException ie) { - LOG.error( - "read position[{}] destLen[{}] offset[{}] " + "readLen[{}] of [{}] failed, retry time[{}], " - + "due to exception[{}]", position, length, offset, bytesRead, uri, retryTime, exception); - throw exception; + } catch (EOFException e) { + return -1; + } finally { + if (inputStream != null) { + inputStream.close(); } } - } while (System.currentTimeMillis() - startRetryTime <= OBSCommonUtils.MAX_TIME_IN_MILLISECONDS_TO_RETRY); - - if (inputStream == null || exception != null) { - IOException e = new IOException( - "read failed of " + uri + ", inputStream is " + (inputStream == null ? "null" : "not null"), exception); - LOG.error( - "read position[{}] destLen[{}] offset[{}] len[{}] failed, " + "retry time[{}], due to exception[{}]", - position, length, offset, bytesRead, retryTime, exception); - throw e; - } + return count; + }, true); + long endTime = System.currentTimeMillis(); LOG.debug("Read-4args uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, length, bytesRead, position, threadId, diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsCCInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsCCInputStream.java new file mode 100644 index 0000000..7d637b5 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsCCInputStream.java @@ -0,0 +1,631 @@ +package org.apache.hadoop.fs.obs.input; + +import com.google.common.base.Preconditions; + +import org.apache.hadoop.fs.ByteBufferReadable; +import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSCommonUtils; +import org.apache.hadoop.fs.obs.OBSConstants; +import org.apache.hadoop.fs.obs.OBSFileStatus; +import org.apache.hadoop.fs.obs.OBSFileSystem; +import org.apache.hadoop.fs.obs.TrafficStatistics; +import org.apache.hadoop.fs.obs.memartscc.MemArtsCCClient; +import org.apache.hadoop.fs.obs.memartscc.MemArtsCCInputStream; +import org.apache.hadoop.fs.obs.memartscc.MemArtsCCInputStreamBase; +import org.apache.hadoop.fs.obs.memartscc.MemArtsCCInputStreamWithPool; +import org.apache.hadoop.fs.obs.memartscc.OBSInputStreamSimulator; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; +import java.util.concurrent.atomic.AtomicLong; + +/** + * 功能描述 + * + * @since 2021-05-19 + */ +public class OBSMemArtsCCInputStream extends FSInputStream implements CanSetReadahead, ByteBufferReadable { + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger(OBSMemArtsCCInputStream.class); + + /** + * The State of this inputStream: + * New: newly created + * ORead: read from OBS directly + * MRead: read from MemArtsCC + * Close: closed + */ + State state; + + /** + * The statistics for OBS file system. + */ + private final FileSystem.Statistics statistics; + + /** + * MemArtsCC client. + */ + private final MemArtsCCClient memArtsCCClient; + + /** + * Bucket name. + */ + private final String bucket; + + /** + * Object key. + */ + private final String key; + + /** + * Content length. + */ + private final long contentLength; + + /** + * Object uri. + */ + private final String uri; + + /** + * Obs file system instance. + */ + private OBSFileSystem fs; + + /** + * Closed bit. Volatile so reads are non-blocking. Updates must be in a + * synchronized block to guarantee an atomic check and set + */ + private volatile boolean closed = false; + + /** + * This is the actual position within the object, used by lazy seek to + * decide whether to seek on the next read or not. + */ + private long nextReadPos = 0; + + /** + * InputStream used for read from MemArtsCC + */ + private MemArtsCCInputStreamBase ccStream; + + /** + * MemArtsCC ccRead fail, or return CacheMiss, use this input stream to escape + */ + OBSMemArtsPartnerInputStream partnerInputStream; + + private int bufSize; + + private byte[] tailBuf; + + private Counter oCounter; + + private Counter mCounter; + + private TrafficStatistics trafficStatistics; + + private final OBSInputStreamSimulator oisSimulator; + + public OBSMemArtsCCInputStream(final String bucketName, final String objectKey, final long fileStatusLength, + final FileSystem.Statistics stats, final long readAheadRangeValue, final long memartsccReadAheadRangeValue, + final OBSFileSystem obsFileSystem, OBSFileStatus fileStatus) { + Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(bucketName), "No Bucket"); + Preconditions.checkArgument(OBSCommonUtils.isStringNotEmpty(objectKey), "No Key"); + Preconditions.checkArgument(fileStatusLength >= 0, "Negative content length"); + this.bucket = bucketName; + this.key = objectKey; + this.contentLength = fileStatusLength; + this.memArtsCCClient = obsFileSystem.getMemArtsCCClient(); + this.statistics = stats; + this.uri = "obs://" + this.bucket + "/" + this.key; + this.fs = obsFileSystem; + this.partnerInputStream = new OBSMemArtsPartnerInputStream(bucketName, objectKey, fileStatusLength, + obsFileSystem.getObsClient(), stats, readAheadRangeValue, obsFileSystem, this); + + // the initial state of the stream + this.state = State.MREAD; + + this.bufSize = obsFileSystem.getConf() + .getInt(OBSConstants.MEMARTSCC_BUFFER_SIZE, OBSConstants.DEFAULT_MEMARTSCC_BUFFER_SIZE); + int directBufferSize = obsFileSystem.getConf() + .getInt(OBSConstants.MEMARTSCC_DIRECTBUFFER_SIZE, OBSConstants.DEFAULT_MEMARTSCC_DIRECTBUFFER_SIZE); + String inputSteamType = obsFileSystem.getConf() + .get(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE, OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_POOL); + if (inputSteamType.equals(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_BIND)) { + this.ccStream = new MemArtsCCInputStream( + memArtsCCClient, objectKey, fileStatus, memartsccReadAheadRangeValue, stats, bufSize, directBufferSize); + } else if (inputSteamType.equals(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_POOL)) { + int borrowTimeout = obsFileSystem.getConf() + .getInt(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_TIMEOUT, + OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_BORROW_DEFAULT_TIMEOUT); + this.ccStream = new MemArtsCCInputStreamWithPool( + memArtsCCClient, objectKey, fileStatus, memartsccReadAheadRangeValue, stats, borrowTimeout); + } else { + throw new IllegalArgumentException("invalid input stream type:" + inputSteamType); + } + this.oCounter = new Counter(); + this.mCounter = new Counter(); + + oisSimulator = new OBSInputStreamSimulator(fileStatusLength, readAheadRangeValue); + initTrafficReport(obsFileSystem); + + LOG.debug("create OBSMemArtsCCInputStream[{}] for file {}", this.hashCode(), objectKey); + } + + private void initTrafficReport(OBSFileSystem obsFileSystem) { + // traffic statistics report + trafficStatistics = obsFileSystem.getTrafficStatistics(); + partnerInputStream.setTrafficStaticsClass(trafficStatistics); + ccStream.setTrafficStaticsClass(trafficStatistics); + } + + public synchronized int available() throws IOException { + fs.checkOpen(); + checkStreamOpen(); + + long remaining = this.contentLength - this.nextReadPos; + if (remaining > Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } + + return (int) remaining; + } + + /** + * {@inheritDoc} + * + *

This updates the statistics on read operations started and whether or + * not the read operation "completed", that is: returned the exact number of + * bytes requested. + * + * @throws IOException if there are other problems + */ + @Override + public synchronized int read(final byte[] buf, final int off, final int len) throws IOException { + LOG.debug("read(buf,off,len), offset[{}], len[{}] ", off, len); + fs.checkOpen(); + checkStreamOpen(); + + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + long endTime; + + validatePositionedReadArgs(nextReadPos, buf, off, len); + if (len == 0) { + return 0; + } + + boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength; + if (isTrue) { + return -1; + } + + // when the position of read is in the tail of the file + int bytesRead = 0; + int posInTail = posInTailBuf(nextReadPos); + if (posInTail != -1 && tailBuf != null) { + int copyLen = (int)(contentLength - nextReadPos); + copyLen = Math.min(copyLen, Math.min(len, buf.length - off)); + System.arraycopy(tailBuf, posInTail, buf, off, copyLen); + nextReadPos += copyLen; + this.partnerInputStream.seek(nextReadPos); + this.ccStream.seek(nextReadPos); + oisSimulator.seek(nextReadPos); + incrementBytesRead(copyLen); + return copyLen; + } + + bytesRead = readInState(buf, off, len, false); + + if (bytesRead > 0) { + nextReadPos += bytesRead; + } + + endTime = System.currentTimeMillis(); + long position = bytesRead >= 0 ? nextReadPos - 1 : nextReadPos; + LOG.debug("Read-3args uri:{}, contentLength:{}, destLen:{}, readLen:{}, " + + "position:{}, thread:{}, timeUsedMilliSec:{}", uri, contentLength, len, bytesRead, position, threadId, + endTime - startTime); + return bytesRead; + } + + private int readInputStream(final InputStream is, final byte[] buf, final int off, + final int len, boolean oneByteRead, final Counter counter, final State state) + throws IOException { + + int ret; + int readLen; + long start; + long end; + long pos = this.nextReadPos; + + start = System.nanoTime(); + if (oneByteRead) { + ret = is.read(); + readLen = 1; + } else { + ret = is.read(buf, off, len); + readLen = ret; + } + end = System.nanoTime(); + counter.increase(end - start, readLen); + LOG.debug("{} {}({},{},{})", this.hashCode(), state, pos, readLen, end - start); + return ret; + } + + private int readInState(final byte[] buf, final int off, final int len, boolean oneByteRead) throws IOException { + switch (this.state) { + case NEW: + this.partnerInputStream.reopen("open first connection", this.nextReadPos, len); + + // simulate reopen without passing through MemArts + long readBytes = oisSimulator.reopen(this.nextReadPos, len); + increaseSimTraffic(readBytes); + + stateTransitionToORead(); + // continue to oread + case OREAD: + try { + // lazy seek + this.partnerInputStream.seek(this.nextReadPos); + oisSimulator.seek(nextReadPos); + + // read + int ret = readInputStream(this.partnerInputStream, buf, off, len, oneByteRead, oCounter, State.OREAD); + + // simulate the read without passing through MemArts + long obsReadBytes = oisSimulator.read(oneByteRead ? 1 : len); + increaseSimTraffic(obsReadBytes); + + return ret; + } catch (OBSMemArtsPartnerInputStream.OReadToMReadTransitionException e) { + if (this.state != State.OREAD) { + throw new IllegalStateException("state must be oread"); + } + stateTransitionToMRead(); + + /** + * based on the implementation of OBSInputStream, + * when OBSInputStream trigger reopen(), + * it will not read any data from stream, + * thus, we should not update the {nextReadPos} + */ + // continue to mread + } + case MREAD: + try { + int ret = tryToReadFromCCStream(buf, off, len, oneByteRead); + + // simulate the read passing through MemArts + long obsReadBytes = oisSimulator.read(oneByteRead ? 1 : len); + increaseSimTraffic(obsReadBytes); + + return ret; + } catch (EOFException e) { + return -1; + } catch (IOException e) { + LOG.error("tryToReadFromCCStream offset[{}] len[{}] of [{}] failed, due to exception[{}]", + off, len, uri, e); + throw e; + } + + default: + throw new IllegalStateException("unreachable code"); + } + } + + @Override + public synchronized int read() throws IOException { + fs.checkOpen(); + checkStreamOpen(); + long startTime = System.currentTimeMillis(); + long threadId = Thread.currentThread().getId(); + long endTime; + boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength; + + if (isTrue) { + return -1; + } + + int byteRead = 0; + // tail case + int posInTail = posInTailBuf(nextReadPos); + if (posInTail != -1 && tailBuf != null) { + byteRead = tailBuf[posInTail] & 0xFF; + nextReadPos++; + this.partnerInputStream.seek(nextReadPos); + oisSimulator.seek(nextReadPos); + this.ccStream.seek(nextReadPos); + incrementBytesRead(1); + return byteRead; + } + + byteRead = readInState(null, 0, 0, true); + + if (byteRead >= 0) { + nextReadPos++; + } + + endTime = System.currentTimeMillis(); + long position = byteRead >= 0 ? nextReadPos - 1 : nextReadPos; + LOG.debug("read-0arg uri:{}, contentLength:{}, position:{}, readValue:{}, " + "thread:{}, timeUsedMilliSec:{}", + uri, contentLength, position, byteRead, threadId, endTime - startTime); + + return byteRead; + } + + @Override + public synchronized int read(ByteBuffer byteBuffer) throws IOException { + int len = byteBuffer.remaining(); + if (len == 0) { + return 0; + } + + byte[] buf = new byte[len]; + int size = read(buf, 0, len); + if (size != -1) { + byteBuffer.put(buf, 0, size); + } + + return size; + } + + private int posInTailBuf(long pos) { + if (pos < 0) { + return -1; + } + if (pos < contentLength - bufSize) { + return -1; + } + long tailBufHead = contentLength - bufSize; + if (tailBufHead < 0) tailBufHead = 0; + long idx = pos - tailBufHead; + if (idx < 0 || idx >= bufSize) { + LOG.warn("nextReadPos is in invalid state, pos = {}, contentLength = {}, bufSize = {}", pos, contentLength, bufSize); + return -1; + } + return (int)idx; + } + + @Override + public synchronized void seek(final long targetPos) throws IOException { + LOG.debug("seek(targetPos), targetPos [{}] ", targetPos); + + fs.checkOpen(); + checkStreamOpen(); + + // Do not allow negative seek + if (targetPos < 0) throw new EOFException(FSExceptionMessages.NEGATIVE_SEEK + " " + targetPos); + if (targetPos > contentLength) throw new EOFException("Cannot seek after EOF"); + + if (this.contentLength <= 0) { + return; + } + + // tail case + if (posInTailBuf(targetPos) != -1 && tailBuf == null) { + byte[] tmpBuf = new byte[bufSize]; + long tailBufHead = contentLength - bufSize; + if (tailBufHead < 0) tailBufHead = 0; + // seek to tailBufHead + nextReadPos = tailBufHead; + this.partnerInputStream.seek(tailBufHead); + oisSimulator.seek(tailBufHead); + this.ccStream.seek(tailBufHead); + int off = 0; + int bytesRead = 0; + do { + bytesRead = read(tmpBuf, off, bufSize - off); + if (bytesRead == -1) { + break; + } + off += bytesRead; + } while (off < bufSize); + tailBuf = tmpBuf; + } + + // Lazy seek + nextReadPos = targetPos; + this.partnerInputStream.seek(targetPos); + oisSimulator.seek(targetPos); + this.ccStream.seek(targetPos); + } + + @Override + public synchronized long getPos() throws IOException { + fs.checkOpen(); + checkStreamOpen(); + return nextReadPos < 0 ? 0 : nextReadPos; + } + + @Override + public boolean seekToNewSource(long targetPos) throws IOException { + return false; + } + + @Override + public synchronized void setReadahead(Long readahead) throws IOException { + fs.checkOpen(); + checkStreamOpen(); + this.ccStream.setReadahead(readahead); + } + + /** + * Close the stream. This triggers publishing of the stream statistics back + * to the filesystem statistics. This operation is synchronized, so that + * only one thread can attempt to close the connection; all later/blocked + * calls are no-ops. + * + * @throws IOException on any problem + */ + @Override + public synchronized void close() throws IOException { + if (!closed) { + fs.checkOpen(); + // this is actually a no-op + super.close(); + if (partnerInputStream != null) { + partnerInputStream.close(); + partnerInputStream = null; + } + if (ccStream != null) { + ccStream.close(); + ccStream = null; + } + closed = true; + oisSimulator.close(); + } + LOG.debug("{} SUMMARY: OREAD{}, MREAD{}", this.hashCode(), oCounter, mCounter); + } + + /** + * Verify that the input stream is open. Non blocking; this gives the last + * state of the volatile {@link #closed} field. + * + * @throws IOException if the connection is closed. + */ + private void checkStreamOpen() throws IOException { + if (closed) { + throw new IOException(uri + ": " + FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + /** + * Increment the bytes read counter if there is a stats instance and the + * number of bytes read is more than zero. + * + * @param bytesRead number of bytes read + */ + private void incrementBytesRead(final long bytesRead) { + if (statistics != null && bytesRead > 0) { + statistics.incrementBytesRead(bytesRead); + } + } + + private int tryToReadFromCCStream(final byte[] buf, final int off, final int len, boolean oneByteRead) throws IOException { + int bytesRead; + try { + bytesRead = readInputStream(this.ccStream, buf, off, len, oneByteRead, mCounter, State.MREAD); + return bytesRead; + } catch (IOException e) { + // escape IOException and CacheMiss + LOG.debug("{} mread escape, caused by {}", this.hashCode(), e.getMessage()); + } + + /** + * escape: + * 1. partnerInputStream lazy seek to {nextReadPos} + * 2. do one read() from partnerInputStream (ORead), + * 3. transfer to ORead state + */ + if (this.getState() != State.MREAD) { + throw new IllegalStateException("cachemiss reopen must in state mread"); + } + + // 1. lazy seek to the right position + this.partnerInputStream.seek(this.nextReadPos); + oisSimulator.seek(nextReadPos); + + // 2. do once OBSInputStream.read + // in MRead state, if this read() trigger reopen(), + // it will handled by partnerInputStream quietly. + try { + bytesRead = readInputStream(this.partnerInputStream, buf, off, len, oneByteRead, oCounter, State.OREAD); + } catch (OBSMemArtsPartnerInputStream.OReadToMReadTransitionException e) { + // we should not cache transfer signal in this place, + LOG.error("catch unexpected reopen signal, {}", e.getMessage()); + throw new IllegalStateException("catch unexpected reopen signal", e); + } + + // 3. into ORead state, cache signal + this.stateTransitionToORead(); + return bytesRead; + } + + public State getState() { + return this.state; + } + + private void stateTransitionToMRead() throws IOException { + if (this.state != State.OREAD) { + throw new IllegalStateException("cannot transit state from " + this.state.toString() + " to mread"); + } + this.state = State.MREAD; + this.ccStream.seek(nextReadPos); + oisSimulator.seek(nextReadPos); + } + + private void stateTransitionToORead() { + if (this.state != State.MREAD && this.state != State.NEW) { + throw new IllegalStateException("cannot transit state from " + this.state.toString() + " to oread"); + } + this.state = State.OREAD; + } + + public enum State { + /** + * new created input stream + */ + NEW("NEW"), + + /** + * using partnerInputStream(OBSInputStream) to do actual read() + */ + OREAD("OREAD"), + + /** + * using memarts to do actual read() + */ + MREAD("MREAD"), + + /** + * closed state + */ + CLOSED("CLOSED"); + + private String state; + + State(String s) { + this.state = s; + } + + public String toString() { + return this.state; + } + } + + private static class Counter { // because read is synchronized this count need not lock + + private volatile AtomicLong num = new AtomicLong(); + + private volatile AtomicLong totalTime = new AtomicLong(); + + private volatile AtomicLong totalLen = new AtomicLong(); + + public void increase(long time, long len) { + totalTime.getAndAdd(time); + totalLen.getAndAdd(len); + num.getAndIncrement(); + } + + @Override + public String toString() { + return String.format("[count=%s,time=%s,size=%s]", num, totalTime, totalLen); + } + } + + private void increaseSimTraffic(long value) { + if (trafficStatistics != null) { + trafficStatistics.increase(value, TrafficStatistics.TrafficType.Q); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsPartnerInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsPartnerInputStream.java new file mode 100644 index 0000000..347d840 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/OBSMemArtsPartnerInputStream.java @@ -0,0 +1,95 @@ +package org.apache.hadoop.fs.obs.input; + +import com.obs.services.ObsClient; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSFileSystem; +import org.apache.hadoop.fs.obs.TrafficStatistics; + +import java.io.IOException; + +public class OBSMemArtsPartnerInputStream extends OBSInputStream { + private OBSMemArtsCCInputStream mistream; + + private TrafficStatistics trafficStatistics; + + OBSMemArtsPartnerInputStream( + String bucketName, + String bucketKey, + long fileStatusLength, + ObsClient obsClient, + FileSystem.Statistics stats, + long readAheadRangeValue, + OBSFileSystem obsFileSystem, + OBSMemArtsCCInputStream mistream) { + super(bucketName, bucketKey, fileStatusLength, obsClient, stats, readAheadRangeValue, obsFileSystem); + this.mistream = mistream; + } + + public void setTrafficStaticsClass(TrafficStatistics instance) { + trafficStatistics = instance; + } + + private void increaseMissTraffic(long value) { + if (trafficStatistics != null) { + trafficStatistics.increase(value, TrafficStatistics.TrafficType.Q1); + } + } + + @Override + protected synchronized void reopen(final String reason, final long targetPos, final long length) throws IOException { + /** + * reopen() in New state should call super reopen directly + */ + if (this.mistream.getState() == OBSMemArtsCCInputStream.State.NEW) { + super.reopen(reason, targetPos, length); + long readFromOBS = calculateOBSTraffic(targetPos, length); + increaseMissTraffic(readFromOBS); + } + + /** + * oread reopen + */ + if (this.mistream.getState() == OBSMemArtsCCInputStream.State.OREAD) { + /** + * based on the implementation of OBSInputStream + * when reopen() occurred,byteRead must be 0, + * thus, the caller need not to consider the + * intermediate state of reading a fractional of data. + * Then just close the wrapper stream and throw the state transfer signal. + */ + closeStream(reason, length); + throw new OReadToMReadTransitionException("oread reopen(), transit to mread, origin reason: " + reason, targetPos, length); + } + + /** + * escape from MemArtsCC, reopen the OBSInputStream + */ + if (this.mistream.getState() == OBSMemArtsCCInputStream.State.MREAD) { + super.reopen(reason, targetPos, length); + long readFromOBS = calculateOBSTraffic(targetPos, length); + increaseMissTraffic(readFromOBS); + } + } + + public static class OReadToMReadTransitionException extends RuntimeException { + static final long serialVersionUID = 5364319876219655679L; + + public long getTargetPos() { + return targetPos; + } + + public long getLen() { + return len; + } + + private long targetPos; + private long len; + + public OReadToMReadTransitionException(String msg, long targetPos, long len) { + super(msg); + this.len = len; + this.targetPos = targetPos; + } + + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ObsSelectInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ObsSelectInputStream.java new file mode 100644 index 0000000..8c9133a --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ObsSelectInputStream.java @@ -0,0 +1,299 @@ +package org.apache.hadoop.fs.obs.input; + +import java.io.EOFException; +import java.io.IOException; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicLong; + +import javax.annotation.Nullable; + +import com.google.common.base.Preconditions; +import com.obs.services.model.select.SelectEventVisitor; +import com.obs.services.model.select.SelectInputStream; +import com.obs.services.model.select.SelectObjectResult; + +import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.PathIOException; +import org.apache.hadoop.fs.obs.OBSCommonUtils; +import org.apache.hadoop.fs.obs.OBSConstants; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +public class ObsSelectInputStream + extends FSInputStream implements CanSetReadahead { + private static final Logger LOG = LoggerFactory.getLogger(ObsSelectInputStream.class); + + private AtomicLong pos = new AtomicLong(0); + private final AtomicBoolean closed = new AtomicBoolean(false); + private final AtomicBoolean completedSuccessfully = new AtomicBoolean(false); + + private final SelectInputStream recordsInputStream; + + private final String bucket; + private final String key; + private final String uri; + + private long readahead; + + public ObsSelectInputStream( + final String bucket, + final String key, + final SelectObjectResult selectResult) { + this.recordsInputStream = selectResult.getInputStream( + new SelectEventVisitor() { + @Override + public void visitEndEvent() { + completedSuccessfully.set(true); + } + }); + + this.bucket = bucket; + this.key = key; + this.uri = String.format("obs://%s/%s", this.bucket, this.key); + } + + @Override + public void close() + throws IOException { + if (!closed.getAndSet(true)) { + try { + // set up for aborts. + // if we know the available amount > readahead. Abort. + // + boolean shouldAbort = recordsInputStream.available() > readahead; + if (!shouldAbort) { + // read our readahead range worth of data + long skipped = recordsInputStream.skip(readahead); + shouldAbort = recordsInputStream.read() >= 0; + } + // now, either there is data left or not. + if (shouldAbort) { + recordsInputStream.abort(); + } + } catch (IOException e) { + LOG.debug("While closing stream", e); + } finally { + OBSCommonUtils.closeAll(recordsInputStream); + super.close(); + } + } + } + + /** + * Verify that the input stream is open. Non blocking; this gives + * the last state of the atomic {@link #closed} field. + * + * @throws PathIOException if the connection is closed. + */ + private void checkNotClosed() + throws IOException { + if (closed.get()) { + throw new PathIOException(uri, FSExceptionMessages.STREAM_IS_CLOSED); + } + } + + @Override + public int available() + throws IOException { + checkNotClosed(); + return recordsInputStream.available(); + } + + @Override + public long skip(long n) + throws IOException { + checkNotClosed(); + long skipped = recordsInputStream.skip(n); + pos.addAndGet(skipped); + return skipped; + } + + @Override + public long getPos() { + return pos.get(); + } + + /** + * from a possibly null Long value, return a valid + * readahead. + * + * @param readahead new readahead + * @return a natural number. + * @throws IllegalArgumentException if the range is invalid. + */ + public static long validateReadahead(@Nullable Long readahead) { + if (readahead == null) { + return OBSConstants.DEFAULT_READAHEAD_RANGE; + } else { + Preconditions.checkArgument(readahead >= 0, "Negative readahead value" /* E_NEGATIVE_READAHEAD_VALUE */); + return readahead; + } + } + + @Override + public void setReadahead(Long readahead) { + this.readahead = validateReadahead(readahead); + } + + public long getReadahead() { + return readahead; + } + + @Override + public int read() + throws IOException { + checkNotClosed(); + int byteRead; + try { + byteRead = recordsInputStream.read(); + } catch (EOFException e) { + // this could be one of: end of file, some IO failure + if (completedSuccessfully.get()) { + // read was successful + return -1; + } else { + // the stream closed prematurely + LOG.info("Reading of OBS Select data from {} failed before all results " + + " were generated.", uri); + throw new PathIOException(uri, + "Read of OBS Select data did not complete"); + } + } + + if (byteRead >= 0) { + incrementBytesRead(1); + } + return byteRead; + } + + @Override + public int read(byte[] buf, int off, int len) + throws IOException { + checkNotClosed(); + validatePositionedReadArgs(pos.get(), buf, off, len); + if (len == 0) { + return 0; + } + + int bytesRead; + try { + bytesRead = recordsInputStream.read(buf, off, len); + } catch (EOFException e) { + // the base implementation swallows EOFs. + return -1; + } + + incrementBytesRead(bytesRead); + return bytesRead; + } + + @Override + public void seek(long newPos) + throws IOException { + long current = getPos(); + long distance = newPos - current; + if (distance < 0) { + throw unsupported("seek() backwards from " + current + " to " + newPos); + } + if (distance == 0) { + LOG.debug("ignoring seek to current position."); + } else { + // the complicated one: Forward seeking. Useful for split files. + LOG.debug("Forward seek by reading {} bytes", distance); + long bytesSkipped = 0L; + // read byte-by-byte, hoping that buffering will compensate for this. + // doing it this way ensures that the seek stops at exactly the right + // place. skip(len) can return a smaller value, at which point + // it's not clear what to do. + while (distance > 0) { + int r = read(); + if (r == -1) { + // reached an EOF too early + throw new EOFException("Seek to " + newPos + + " reached End of File at offset " + getPos()); + } + distance--; + bytesSkipped++; + } + } + } + + /** + * Build an exception to raise when an operation is not supported here. + * + * @param action action which is unsupported. + * @return an exception to throw. + */ + protected PathIOException unsupported(final String action) { + return new PathIOException(this.uri, action + " not supported"); + } + + @Override + public boolean seekToNewSource(long targetPos) + throws IOException { + return false; + } + + @Override + public boolean markSupported() { + return false; + } + + @Override + public synchronized void mark(int readlimit) { + } + + @Override + public synchronized void reset() + throws IOException { + throw unsupported("Mark"); + } + + public void abort() { + if (!closed.get()) { + LOG.debug("Aborting"); + recordsInputStream.abort(); + } + } + + /** + * Read at a specific position. + * Reads at a position earlier than the current {@link #getPos()} position + * will fail with a {@link PathIOException}. See {@link #seek(long)}. + * Unlike the base implementation And the requirements of the filesystem + * specification, this updates the stream position as returned in + * {@link #getPos()}. + * + * @param position offset in the stream. + * @param buffer buffer to read in to. + * @param offset offset within the buffer + * @param length amount of data to read. + * @return the result. + * @throws PathIOException Backwards seek attempted. + * @throws EOFException attempt to seek past the end of the stream. + * @throws IOException IO failure while seeking in the stream or reading + * data. + */ + @Override + public int read(long position, byte[] buffer, int offset, int length) + throws IOException { + // maybe seek forwards to the position. + seek(position); + return read(buffer, offset, length); + } + + /** + * Increment the bytes read counter if there is a stats instance + * and the number of bytes read is more than zero. + * This also updates the {@link #pos} marker by the same value. + * + * @param bytesRead number of bytes read + */ + private void incrementBytesRead(long bytesRead) { + if (bytesRead > 0) { + pos.addAndGet(bytesRead); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java index 46c0106..f4ff1de 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/input/ReadAheadTask.java @@ -22,7 +22,7 @@ * we can accelerate the sequential read. */ public class ReadAheadTask implements Runnable { - public final Logger log = LoggerFactory.getLogger(ReadAheadTask.class); + private static final Logger log = LoggerFactory.getLogger(ReadAheadTask.class); private String bucketName; diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/CcGetShardParam.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/CcGetShardParam.java new file mode 100644 index 0000000..e13cb33 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/CcGetShardParam.java @@ -0,0 +1,65 @@ +package org.apache.hadoop.fs.obs.memartscc; + +/** + * 功能描述 + * + * @since 2021-05-24 + */ +public class CcGetShardParam { + long start; + + long end; + + String bucketName; + + boolean enablePosix; + + public String objectKey; + + ObjectShard[] ObjectShard; + + int allocShardNum; + + int validShardNum; + + public CcGetShardParam(long start, long end, String bucketName, boolean enablePosix, String objectKey, + ObjectShard[] ObjectShard, int allocShardNum, int validShardNum) { + this.start = start; + this.end = end; + this.bucketName = bucketName; + this.enablePosix = enablePosix; + this.objectKey = objectKey; + this.ObjectShard = ObjectShard; + this.allocShardNum = allocShardNum; + this.validShardNum = validShardNum; + } + + public int getValidShardNum() { + return validShardNum; + } + + public ObjectShard[] getObjectShard() { + return ObjectShard; + } + + public long getStart() { + return start; + } + + public long getEnd() { + return end; + } + + public String getBucketName() { + return bucketName; + } + + public boolean isEnablePosix() { + return enablePosix; + } + + public int getAllocShardNum() { + return allocShardNum; + } + +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ICache.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ICache.java new file mode 100644 index 0000000..19ac2e1 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ICache.java @@ -0,0 +1,78 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import java.io.IOException; +import java.nio.ByteBuffer; + +public interface ICache { + int init(String config, String otherInfo) throws IOException; + + int read(ReadParam readParam, ByteBuffer buf, byte[] dt) throws IOException; + + int getObjectShardInfo(CcGetShardParam ccGetShardParam); + + byte[] getDT(); + + // originalTraffic: Q + // applicationTraffic: Q' + // hitTraffic: Q2 + // missTraffic: Q1 + void reportReadStatistics(long originalTraffic, long applicationTraffic, long hitTraffic, long missTraffic); + + void close(); + + class ReadParam { + public long offset; + public long length; + public long prefetchStart; + public long prefetchEnd; + public boolean isPrefetch; + public boolean isConsistencyCheck; + public boolean isFileLayout; // 特性暂未启用,默认false + public BucketContext bucketCtx; + public ObjectAttr objectAttr; + + public ReadParam(long offset, long length, long prefetchStart, long prefetchEnd, + boolean isPrefetch, boolean isConsistencyCheck, boolean isFileLayout, + BucketContext bucketCtx, ObjectAttr objectAttr) { + this.offset = offset; + this.length = length; + this.prefetchStart = prefetchStart; + this.prefetchEnd = prefetchEnd; + this.isPrefetch = isPrefetch; + this.isConsistencyCheck = isConsistencyCheck; + this.isFileLayout = isFileLayout; + this.bucketCtx = bucketCtx; + this.objectAttr = objectAttr; + } + } + + class BucketContext { + public String ak; + public String sk; + public String token; + public String endpoint; + public String bucketName; + public boolean enablePosix; + + public BucketContext(String ak, String sk, String token, String endpoint, String bucketName, boolean enablePosix) { + this.ak = ak; + this.sk = sk; + this.token = token; + this.endpoint = endpoint; + this.bucketName = bucketName; + this.enablePosix = enablePosix; + } + } + + class ObjectAttr { + public String name; + public String etag; + public long mtime; + + public ObjectAttr(String name, String etag, long mtime) { + this.name = name; + this.etag = etag; + this.mtime = mtime; + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCClient.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCClient.java new file mode 100644 index 0000000..6a3681d --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCClient.java @@ -0,0 +1,440 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import com.obs.services.IObsCredentialsProvider; +import com.obs.services.internal.security.LimitedTimeSecurityKey; +import com.obs.services.model.ISecurityKey; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.obs.OBSConstants; +import org.apache.hadoop.fs.obs.OBSSecurityProviderUtil; +import org.apache.hadoop.fs.obs.TrafficStatistics; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.security.token.TokenIdentifier; +import org.codehaus.jettison.json.JSONObject; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; +import java.util.HashMap; +import java.util.Map; +import java.nio.ByteBuffer; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicBoolean; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.concurrent.atomic.AtomicReference; + +import sun.nio.ch.DirectBuffer; + +/** + * 功能描述 + * + * @since 2021-05-18 + */ +public class MemArtsCCClient { + + private static final String MRS_AZ_ENV_VARNAME = "AZ"; + + private static final String SECURITY_ENABLE = "security_enable"; + + private static final Logger LOG = LoggerFactory.getLogger(MemArtsCCClient.class); + + private static final boolean IS_FILE_LAYOUT = false; + + private IObsCredentialsProvider securityProvider; + + private String endpoint; + + private final String bucket; + + private final boolean enablePosix; + + private volatile boolean closed; + + private boolean initialized; + + private int akskRefreshInterval; + + AtomicReference iSecurityKey = new AtomicReference<>(); + + private ICache cache; + + private Configuration conf; + + private final AtomicReference password; + + public MemArtsCCClient(String bucket, boolean enablePosix) { + this.bucket = bucket; + this.enablePosix = enablePosix; + this.password = new AtomicReference<>(); + } + + public boolean initialize(URI name, Configuration conf) { + if (initialized) { + LOG.warn("MemArtsCCClient have been initialized more than once"); + return false; + } + + if (needEscapePyspark(conf)) { + return false; + } + + this.conf = conf; + + if (!createCacheClass(conf)) { + return false; + } + + setEndpoint(conf); + + akskRefreshInterval = conf.getInt( + OBSConstants.MEMARTSCC_AKSK_REFRESH_INTERVAL, + OBSConstants.DEFAULT_MEMARTSCC_AKSK_REFRESH_INTERVAL); + try { + securityProvider = OBSSecurityProviderUtil.createObsSecurityProvider(conf, name); + } catch (IOException e) { + LOG.warn("create security provider failed, {}", e.getMessage()); + return false; + } + setSecurityKey(securityProvider.getSecurityKey()); + + if (!initCacheClass(conf)) { + return false; + } + + startRefreshJob(); + initBufferPool(conf); + initialized = true; + return true; + } + + /** + * Initialize MemArtsCCClient in the delegation token only scenario, + * i.e. where DELEGATION_TOKEN_ONLY is set to true + */ + public boolean initializeDtOnly(final Configuration conf) { + if (initialized) { + LOG.warn("MemArtsCCClient hav e been initialized more than once"); + return false; + } + + this.conf = conf; + + if (needEscapePyspark(conf)) { + return false; + } + + if (!createCacheClass(conf)) { + return false; + } + + if (!initCacheClass(conf)) { + return false; + } + + initialized = true; + return true; + } + + private void initBufferPool(Configuration conf) { + int maxNum = conf.getInt(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_POOL_MAX_SIZE, + OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_POOL_DEFAULT_MAX_SIZE); + int bufferSize = conf.getInt(OBSConstants.MEMARTSCC_DIRECTBUFFER_SIZE, + OBSConstants.DEFAULT_MEMARTSCC_DIRECTBUFFER_SIZE); + bufferPool.initialize(maxNum, bufferSize); + } + + private void setEndpoint(Configuration conf) { + String ep = conf.getTrimmed(OBSConstants.ENDPOINT); + if (ep.startsWith("http://")) { + ep = ep.substring("http://".length()); + } + if (ep.startsWith("https://")) { + ep = ep.substring("https://".length()); + } + this.endpoint = ep; + } + + private boolean initCacheClass(Configuration conf) { + try { + String config = filterCCConfig(conf); + String otherInfo = collectOtherInfo(conf); + int result = init(config, otherInfo); + if (result == 0) { + LOG.debug("memArtsCCClient.ccInit OK!"); + } else { + LOG.warn("memArtsCC init failed, ccInit ret code = {}, will trying to fallback", result); + return false; + } + } catch (Throwable e) { + LOG.warn("memArtsCC init exception, will trying to fallback, caused by {}", e.getMessage()); + return false; + } + return true; + } + + private boolean createCacheClass(Configuration conf) { + try { + Class cacheClass = conf.getClass(OBSConstants.MEMARTSCC_CACHE_IMPL, + null, ICache.class); + if (cacheClass == null) { + LOG.warn("get null ICache instance"); + return false; + } + this.cache = cacheClass.newInstance(); + } catch (InstantiationException | IllegalAccessException | RuntimeException e) { + LOG.warn("get instance of ICache failed", e); + return false; + } + return true; + } + + private boolean needEscapePyspark(Configuration conf) { + boolean isPySpark = conf.getBoolean("spark.yarn.isPython", false); + if (isPySpark) { + boolean pySparkOptimizedConf = conf.getBoolean(OBSConstants.MEMARTSCC_PYSPARK_OPTIMIZED, + OBSConstants.DEFAULT_MEMARTSCC_PYSPARK_OPTIMIZED); + if (!pySparkOptimizedConf) { + LOG.error("disable pyspark optimize from config"); + } + boolean pySparkOptimizedProp = Boolean.parseBoolean(System.getProperty(OBSConstants.MEMARTSCC_PYSPARK_OPTIMIZED, + String.valueOf(OBSConstants.DEFAULT_MEMARTSCC_PYSPARK_OPTIMIZED))); + if (!pySparkOptimizedProp) { + LOG.error("disable pyspark optimize from properties"); + } + boolean pySparkOptimized = pySparkOptimizedConf && pySparkOptimizedProp; + if (!pySparkOptimized) { + LOG.error("escape in pyspark"); + return true; + } + } + return false; + } + + private String filterCCConfig(Configuration conf) { + String prefix = conf.get(OBSConstants.CACHE_CONFIG_PREFIX, OBSConstants.DEFAULT_CACHE_CONFIG_PREFIX); + if (!prefix.endsWith(".")) { + prefix = prefix + "."; + } + Map confMap = conf.getPropsWithPrefix(prefix); + String az = System.getenv(MRS_AZ_ENV_VARNAME); + if (az != null && !az.equals("")) { + confMap.put("zk_root_node", "/memartscc/" + az); + } + confMap.put(SECURITY_ENABLE, Boolean.toString(UserGroupInformation.isSecurityEnabled())); + JSONObject jsonObject = new JSONObject(confMap); + String jsonStr = jsonObject.toString(); + LOG.info("memArtsCC config json: {}", jsonStr); + return jsonStr; + } + + private byte[] getDtFromUgi() throws IOException { + byte[] dt = this.password.get(); + if (dt != null) { + return dt; + } + Credentials credentials = UserGroupInformation.getLoginUser().getCredentials(); + Text serviceName = new Text(MemArtsCCDelegationTokenProvider.getCanonicalName(conf)); + Token token = credentials.getToken(serviceName); + if (token == null) { + return null; + } + dt = token.getPassword(); + this.password.set(dt); + return dt; + } + + private String collectOtherInfo(Configuration conf) { + Map otherInfo = new HashMap<>(); + boolean locality = conf.getBoolean(OBSConstants.MEMARTSCC_LOCALITY_ENABLE, OBSConstants.DEFAULT_MEMARTSCC_LOCALITY_ENABLE); + otherInfo.put("locality_switch", locality); + otherInfo.put("client_type", "obsa"); + JSONObject jsonObject = new JSONObject(otherInfo); + String jsonStr = jsonObject.toString(); + LOG.info("memArtsCC other info json: {}", jsonStr); + return jsonStr; + } + + private void startRefreshJob() { + Thread refreshThread = new Thread(() -> { + while(true) { + try { + while (true) { + if (closed) { + this.cache.close(); + return; + } + long sleepInSec = akskRefreshInterval; + ISecurityKey securityKey = securityProvider.getSecurityKey(); + if (securityKey instanceof LimitedTimeSecurityKey) { + LimitedTimeSecurityKey lsk = (LimitedTimeSecurityKey) securityKey; + long expireAt = lsk.getExpiryDate().getTime(); + long now = LimitedTimeSecurityKey.getUtcTime().getTime(); + long keyAgeMill = expireAt - now; + if (keyAgeMill < akskRefreshInterval * 1000L) { + sleepInSec = keyAgeMill / 10000; + if (sleepInSec < 1) { + sleepInSec = 1; + } + LOG.warn("Refresh MemArtsCC AK/SK interval reset to {} sec, " + + "please check fs.obs.memartscc.aksk.refresh.interval", sleepInSec); + } + } + setSecurityKey(securityKey); + try { + Thread.sleep(sleepInSec * 1000L); + } catch (InterruptedException e) { + LOG.warn("Refresh ak sk interrupted", e); + } + } + } catch (Exception e) { + LOG.warn("Refresh ak sk job failed, will trying to restart.", e); + try { + Thread.sleep(1000); + } catch (InterruptedException interruptedException) { + LOG.warn("Refresh ak sk interrupted", e); + } + } + } + }); + refreshThread.setDaemon(true); + refreshThread.start(); + } + + private void setSecurityKey(ISecurityKey key) { + if (key == null) { + LOG.warn("SecurityKey cannot be set, securityKey is null"); + return; + } + this.iSecurityKey.set(key); + } + + public void close() { + this.closed = true; + } + + public int init(String config, String otherInfo) throws IOException { + return this.cache.init(config, otherInfo); + } + + public int read(boolean isPrefetch, long prefetchStart, long prefetchEnd, ByteBuffer buf, long offset, + long len, String objectKey, long modifyTime, String etag, boolean isConsistencyCheck) throws IOException { + if(!initialized) { + throw new IOException("MemArtsCCClient read before initializing."); + } + + ICache.BucketContext bucketContext = getBucketContext(); + + ICache.ObjectAttr objectAttr = new ICache.ObjectAttr(objectKey, etag, modifyTime); + + ICache.ReadParam readParam = new ICache.ReadParam(offset, len, prefetchStart, prefetchEnd, + isPrefetch, isConsistencyCheck, IS_FILE_LAYOUT, bucketContext, objectAttr); + + byte[] dt = getDtFromUgi(); + + return this.cache.read(readParam, buf, dt); + } + + private ICache.BucketContext getBucketContext() { + String ak = iSecurityKey.get() == null ? "" : iSecurityKey.get().getAccessKey(); + String sk = iSecurityKey.get() == null ? "" : iSecurityKey.get().getSecretKey(); + String securityToken = iSecurityKey.get() == null ? "" : iSecurityKey.get().getSecurityToken(); + + return new ICache.BucketContext(ak, sk, securityToken, endpoint, bucket, enablePosix); + } + + public int getObjectShardInfo(CcGetShardParam ccGetShardParam) { + return this.cache.getObjectShardInfo(ccGetShardParam); + } + + public void reportReadStatistics(TrafficStatistics trafficStatistics) { + if (!initialized || this.cache == null) { + LOG.debug("MemArtsCCClient is not initialized, statistics cannot be reported."); + return; + } + + this.cache.reportReadStatistics(trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q), + trafficStatistics.getStatistics(TrafficStatistics.TrafficType.QDot), + trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q2), + trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q1)); + } + + public byte[] getDT() { + return this.cache.getDT(); + } + + public static final int CCREAD_RETCODE_CACHEMISS = -100; + + public static class ByteBufferPool { + private final AtomicBoolean initialized = new AtomicBoolean(false); + + private final LinkedBlockingQueue pool = new LinkedBlockingQueue<>(); + + private int maxNum; + + private final AtomicInteger createdNum = new AtomicInteger(0); + + private int bufferSize; + + public void initialize(int maxNum, int bufferSize) { + if (!initialized.compareAndSet(false, true)) { + return; + } + this.maxNum = maxNum; + this.bufferSize = bufferSize; + } + + public ByteBuffer borrowBuffer(int timeout) throws InterruptedException, IOException { + // 1.从队列中非阻塞获取 + ByteBuffer buffer = pool.poll(); + // 2. 若队列为空,尝试创建 + if (buffer == null) { + buffer = creatBuffer(); + } + // 3. 创建失败则等待 + if (buffer == null) { + if (timeout < 0) { + buffer = pool.take(); + } else if (timeout > 0) { + buffer = pool.poll(timeout, TimeUnit.MILLISECONDS); + } + // 等于0不等待,直接报错 + } + // 4. 等待不到则报错 + if (buffer == null) { + throw new IOException("ByteBuffer pool exhausted"); + } + return buffer; + } + + public void returnBuffer(ByteBuffer buffer) { + if (buffer == null) { + return; + } + if (!pool.offer(buffer)) { + destroyBuffer(buffer); + } + } + + private ByteBuffer creatBuffer() { + final long newCreateCount = createdNum.incrementAndGet(); + if (newCreateCount > maxNum) { + createdNum.decrementAndGet(); + return null; + } + return ByteBuffer.allocateDirect(bufferSize); + } + + private void destroyBuffer(ByteBuffer buffer) { + if (buffer instanceof DirectBuffer) { + ((DirectBuffer)buffer).cleaner().clean(); + } + } + } + + public static final ByteBufferPool bufferPool = new ByteBufferPool(); +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenIdentifier.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenIdentifier.java new file mode 100644 index 0000000..13f285e --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenIdentifier.java @@ -0,0 +1,109 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.TokenIdentifier; + +import java.io.DataInput; +import java.io.DataOutput; +import java.io.IOException; + +public class MemArtsCCDelegationTokenIdentifier extends TokenIdentifier { + public static final Text MEMARTSCC_DELEGATION_KIND = + new Text("MEMARTSCC_DELEGATION_TOKEN"); + + private Text owner; + private Text renewer; + private Text realUser; + + public MemArtsCCDelegationTokenIdentifier() { + owner = new Text(); + renewer = new Text(); + realUser = new Text(); + } + + + public MemArtsCCDelegationTokenIdentifier(Text owner, Text renewer, Text realUser) { + setOwner(owner); + setRenewer(renewer); + setRealUser(realUser); + } + + public MemArtsCCDelegationTokenIdentifier(Text renewer) { + this(null, renewer, null); + } + + @Override + public Text getKind() { + return MEMARTSCC_DELEGATION_KIND; + } + + @Override + public UserGroupInformation getUser() { + if(owner == null) { + return null; + } + if(owner.toString().isEmpty()) { + return null; + } + final UserGroupInformation realUgi; + final UserGroupInformation ugi; + if ((realUser == null) || (realUser.toString().isEmpty()) + || realUser.equals(owner)) { + realUgi = UserGroupInformation.createRemoteUser(owner.toString()); + ugi = realUgi; + } else { + realUgi = UserGroupInformation.createRemoteUser(realUser.toString()); + ugi = UserGroupInformation.createProxyUser(owner.toString(), realUgi); + } + realUgi.setAuthenticationMethod(UserGroupInformation.AuthenticationMethod.TOKEN); + return ugi; + } + + @Override + public void write(DataOutput dataOutput) throws IOException { + owner.write(dataOutput); + renewer.write(dataOutput); + realUser.write(dataOutput); + } + + @Override + public void readFields(DataInput dataInput) throws IOException { + owner.readFields(dataInput, Text.DEFAULT_MAX_LEN); + renewer.readFields(dataInput, Text.DEFAULT_MAX_LEN); + realUser.readFields(dataInput, Text.DEFAULT_MAX_LEN); + } + + public Text getOwner() { + return owner; + } + + private void setOwner(Text owner) { + if (owner == null) { + this.owner = new Text(); + } else { + this.owner = owner; + } + } + + public Text getRenewer() { + return renewer; + } + + private void setRenewer(Text renewer) { + this.renewer = renewer; + } + + public Text getRealUser() { + return realUser; + } + + private void setRealUser(Text realUser) { + if (realUser == null) { + this.realUser = new Text(); + } else { + this.realUser = realUser; + } + } + +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenProvider.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenProvider.java new file mode 100644 index 0000000..009ac46 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCDelegationTokenProvider.java @@ -0,0 +1,64 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSFileSystem; +import org.apache.hadoop.fs.obs.security.DelegationTokenProvider; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.token.Token; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.net.URI; + +public class MemArtsCCDelegationTokenProvider implements DelegationTokenProvider { + private static final Logger logger = LoggerFactory.getLogger(MemArtsCCDelegationTokenProvider.class); + + public static final String ServiceNameKey = "fs.obs.memartscc.service.name"; + + public static final String DefaultServiceName = "memartscc"; + + public static String getCanonicalName(Configuration conf) { + return conf.get(ServiceNameKey, DefaultServiceName); + } + + private String canonicalServiceName; + + private Configuration conf; + + private URI uri; + + private OBSFileSystem fs; + + @Override + public void initialize(FileSystem fs, URI uri, Configuration conf) { + this.canonicalServiceName = getCanonicalName(conf); + this.uri = uri; + this.conf = conf; + if (fs instanceof OBSFileSystem) { + this.fs = (OBSFileSystem) fs; + } else { + throw new IllegalArgumentException("fs only support OBSFileSystem"); + } + } + + @Override + public Token getDelegationToken(String renewer) { + logger.info("get delegation token for renewer {}", renewer); + try { + MemArtsCCDelegationTokenIdentifier tokenIdentifier = new MemArtsCCDelegationTokenIdentifier( + new Text(renewer)); + byte[] password = fs.getMemArtsCCClient().getDT(); + return new Token(tokenIdentifier.getBytes(), password, + MemArtsCCDelegationTokenIdentifier.MEMARTSCC_DELEGATION_KIND, new Text(this.canonicalServiceName)); + } catch (Exception e) { + logger.warn("get dt from memartscc failed", e); + } + return null; + } + + @Override + public String getCanonicalServiceName() { + return canonicalServiceName; + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStream.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStream.java new file mode 100644 index 0000000..ccd95c6 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStream.java @@ -0,0 +1,219 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSFileStatus; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; + +import sun.nio.ch.DirectBuffer; + +public class MemArtsCCInputStream extends MemArtsCCInputStreamBase { + // private static final Logger LOG = LoggerFactory.getLogger(MemArtsCCInputStream.class); + + private final FileSystem.Statistics statistics; + + private boolean closed; + + private MemArtsCCClient ccClient; + + private final String objKey; + + private long prefetchRange; + + private final int bufSize; + + private long nextReadPos; + + private final long contentLength; + + private final String etag; + + private final long mtime; + + private ByteBuffer buffer; + + private int directBufferSize; + + private long bufferStartPos; + + public MemArtsCCInputStream(MemArtsCCClient ccClient, String objKey, OBSFileStatus fileStatus, + long prefetchRange, final FileSystem.Statistics stats, int bufSize, int directBufferSize) { + this.directBufferSize = directBufferSize; + this.buffer = ByteBuffer.allocateDirect(directBufferSize); + this.ccClient = ccClient; + this.objKey = objKey; + this.prefetchRange = prefetchRange; + this.statistics = stats; + this.bufSize = bufSize; + this.contentLength = fileStatus.getLen(); + this.etag = fileStatus.getEtag(); + this.mtime = fileStatus.getModificationTime(); + this.closed = false; + buffer.position(0); + buffer.limit(0); + } + + private long getPrefetchEnd(long pos, int len) { + long prefetchEnd = pos + Math.max(prefetchRange, len); + return Math.min(prefetchEnd, contentLength); + } + + private int readInBuffer(byte[] buf, int off, int len, boolean oneByteRead) throws IOException { + if (ensureData(oneByteRead ? 1 : len) < 0) { + return -1; + } + // buffer must have data + if (oneByteRead) { + return buffer.get() & 0xFF; + } + int readLen = Math.min(len, buffer.remaining()); + buffer.get(buf, off, readLen); + return readLen; + } + + private void ensurePos() { + int bufPos = (int)(nextReadPos - bufferStartPos); + if (buffer.remaining() == 0 || bufPos < 0 || bufPos >= buffer.limit()) { + buffer.position(0); + buffer.limit(0); + } else { + buffer.position(bufPos); + } + } + + private int ensureData(int len) throws IOException { + ensurePos(); + int remaining = buffer.remaining(); + if (remaining > 0) { + return remaining; + } + // try to fill + int readLen = len < bufSize ? bufSize : Math.min(len, directBufferSize); + readLen = nextReadPos + readLen > contentLength ? (int) (contentLength - nextReadPos) : readLen; + buffer.position(0); + buffer.limit(0); + int bytesRead = ccClient.read( + true, + nextReadPos, + getPrefetchEnd(nextReadPos, len), + buffer, + nextReadPos, + readLen, + objKey, + mtime, + etag, + true + ); + if (bytesRead == MemArtsCCClient.CCREAD_RETCODE_CACHEMISS) { + throw new IOException("cache miss"); + } + if (bytesRead > 0) { + buffer.position(0); + buffer.limit(bytesRead); + bufferStartPos = nextReadPos; + } + increaseHitTrafficTraffic(readLen); + incrementBytesRead(readLen); + return bytesRead; + } + + @Override + public int read() throws IOException { + checkClosed(); + if (nextReadPos >= contentLength) { + return -1; + } + int ret = readInBuffer(null, 0, 0, true); + if (ret >= 0) { + nextReadPos++; + } + return ret; + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + checkClosed(); + if (nextReadPos >= contentLength) { + return -1; + } + if (nextReadPos + len > contentLength) { + len = (int) (contentLength - nextReadPos); + } + int bytesRead = readInBuffer(buf, off, len, false); + if (bytesRead > 0) { + nextReadPos += bytesRead; + } + return bytesRead; + } + + @Override + public void seek(long l) throws IOException { + checkClosed(); + if (l < 0) { + throw new EOFException("Cannot seek to negative offset"); + } + if (l > contentLength) { + throw new EOFException("seek pos " + l + " is larger than contentLength" + contentLength); + } + this.nextReadPos = l; + } + + @Override + public long skip(long n) throws IOException { + checkClosed(); + ensurePos(); + if (nextReadPos + n > contentLength) { + n = contentLength - nextReadPos; + } + if (n > buffer.remaining()) { + // clear the buffer + buffer.limit(0); + buffer.position(0); + } + nextReadPos += n; + return n; + } + + @Override + public long getPos() throws IOException { + checkClosed(); + return nextReadPos; + } + + @Override + public boolean seekToNewSource(long l) throws IOException { + checkClosed(); + return false; + } + + private void checkClosed() throws IOException { + if (closed) { + throw new IOException("MemArtsCCInputStream already closed"); + } + } + + @Override + public void close() throws IOException { + closed = true; + ccClient = null; + nextReadPos = 0; + ((DirectBuffer)buffer).cleaner().clean(); + buffer = null; + } + + @Override + public void setReadahead(Long aLong) throws IOException, UnsupportedOperationException { + if (aLong <= 0) { + return; + } + this.prefetchRange = aLong; + } + + private void incrementBytesRead(final long bytesRead) { + if (statistics != null && bytesRead > 0) { + statistics.incrementBytesRead(bytesRead); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamBase.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamBase.java new file mode 100644 index 0000000..bb7f7df --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamBase.java @@ -0,0 +1,21 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import org.apache.hadoop.fs.CanSetReadahead; +import org.apache.hadoop.fs.Seekable; +import org.apache.hadoop.fs.obs.TrafficStatistics; + +import java.io.InputStream; + +public abstract class MemArtsCCInputStreamBase extends InputStream implements Seekable, CanSetReadahead { + protected TrafficStatistics trafficStatistics; + + public void setTrafficStaticsClass(TrafficStatistics instance) { + trafficStatistics = instance; + } + + protected void increaseHitTrafficTraffic(long value) { + if (trafficStatistics != null) { + trafficStatistics.increase(value, TrafficStatistics.TrafficType.Q2); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamWithPool.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamWithPool.java new file mode 100644 index 0000000..0b068d8 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/MemArtsCCInputStreamWithPool.java @@ -0,0 +1,201 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSFileStatus; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; + +/** + * 为了规避CBG spark组件inputstream 泄露,引入规避措施:bufferPool, 每次进行ccRead之前申请buffer,从ccRead读取数据后再还回pool + * 代价:性能变差 + * 1. 频繁借还buffer + * 2. 原先inputStream中的buffer具有缓冲数据的作用,当前因为要read完后立即还回去,所以缓冲失效。 + */ +public class MemArtsCCInputStreamWithPool extends MemArtsCCInputStreamBase { + private static final Logger LOG = LoggerFactory.getLogger(MemArtsCCInputStreamWithPool.class); + + private final FileSystem.Statistics statistics; + + private boolean closed; + + private MemArtsCCClient ccClient; + + private final String objKey; + + private long prefetchRange; + + // private final int bufSize; + + private long nextReadPos; + + private final long contentLength; + + private final String etag; + + private final long mtime; + + private final int borrowTimeout; + + public MemArtsCCInputStreamWithPool(MemArtsCCClient ccClient, String objKey, OBSFileStatus fileStatus, + long prefetchRange, final FileSystem.Statistics stats, int borrowTimeout) { + this.ccClient = ccClient; + this.objKey = objKey; + this.prefetchRange = prefetchRange; + this.statistics = stats; + this.contentLength = fileStatus.getLen(); + this.etag = fileStatus.getEtag(); + this.mtime = fileStatus.getModificationTime(); + this.closed = false; + this.borrowTimeout = borrowTimeout; + } + + private long getPrefetchEnd(long pos, int len) { + long prefetchEnd = pos + Math.max(prefetchRange, len); + return Math.min(prefetchEnd, contentLength); + } + + private int readInBuffer(byte[] buf, int off, int len, boolean oneByteRead) throws IOException { + ByteBuffer buffer = null; + int readLen = oneByteRead ? 1 : Math.min(len, buf.length - off); + try { + buffer = MemArtsCCClient.bufferPool.borrowBuffer(borrowTimeout); + int bytesRead = fillData(buffer, readLen); + if (bytesRead < 0) { + return -1; + } + if (oneByteRead) { + return buffer.get() & 0xFF; + } + readLen = Math.min(len, buffer.remaining()); + buffer.get(buf, off, readLen); + return readLen; + } catch (InterruptedException e) { + throw new IOException("borrow buffer interrupted", e); + } finally { + MemArtsCCClient.bufferPool.returnBuffer(buffer); + } + } + + private int fillData(ByteBuffer buffer, int len) throws IOException { + buffer.position(0); + buffer.limit(0); + int readLen = Math.min(len, buffer.capacity()); + readLen = nextReadPos + readLen > contentLength ? (int) (contentLength - nextReadPos) : readLen; + int bytesRead = ccClient.read( + true, + nextReadPos, + getPrefetchEnd(nextReadPos, len), + buffer, + nextReadPos, + readLen, + objKey, + mtime, + etag, + true + ); + if (bytesRead == MemArtsCCClient.CCREAD_RETCODE_CACHEMISS) { + throw new IOException("cache miss"); + } + if (bytesRead > 0) { + buffer.position(0); + buffer.limit(bytesRead); + } + increaseHitTrafficTraffic(readLen); + incrementBytesRead(readLen); + return bytesRead; + } + + @Override + public int read() throws IOException { + checkClosed(); + if (nextReadPos >= contentLength) { + return -1; + } + int ret = readInBuffer(null, 0, 0, true); + if (ret >= 0) { + nextReadPos++; + } + return ret; + } + + @Override + public int read(byte[] buf, int off, int len) throws IOException { + checkClosed(); + if (nextReadPos >= contentLength) { + return -1; + } + if (nextReadPos + len > contentLength) { + len = (int) (contentLength - nextReadPos); + } + int bytesRead = readInBuffer(buf, off, len, false); + if (bytesRead > 0) { + nextReadPos += bytesRead; + } + return bytesRead; + } + + @Override + public void seek(long l) throws IOException { + checkClosed(); + if (l < 0) { + throw new EOFException("Cannot seek to negative offset"); + } + if (l > contentLength) { + throw new EOFException("seek pos " + l + " is larger than contentLength" + contentLength); + } + this.nextReadPos = l; + } + + @Override + public long skip(long n) throws IOException { + checkClosed(); + if (nextReadPos + n > contentLength) { + n = contentLength - nextReadPos; + } + nextReadPos += n; + return n; + } + + @Override + public long getPos() throws IOException { + checkClosed(); + return nextReadPos; + } + + @Override + public boolean seekToNewSource(long l) throws IOException { + checkClosed(); + return false; + } + + private void checkClosed() throws IOException { + if (closed) { + throw new IOException("MemArtsCCInputStream already closed"); + } + } + + @Override + public void close() throws IOException { + closed = true; + ccClient = null; + nextReadPos = 0; + } + + @Override + public void setReadahead(Long aLong) throws IOException, UnsupportedOperationException { + if (aLong <= 0) { + return; + } + this.prefetchRange = aLong; + } + + private void incrementBytesRead(final long bytesRead) { + if (statistics != null && bytesRead > 0) { + statistics.incrementBytesRead(bytesRead); + } + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/OBSInputStreamSimulator.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/OBSInputStreamSimulator.java new file mode 100644 index 0000000..977076f --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/OBSInputStreamSimulator.java @@ -0,0 +1,210 @@ +package org.apache.hadoop.fs.obs.memartscc; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.util.Locale; + +import static java.lang.Math.min; + +public class OBSInputStreamSimulator { + private final long contentLength; + + private final long readAheadRange; + + private long nextReadPos; + + private InputStreamSimulator mockInputStream; + + private long streamCurrentPos; + + private long contentRangeFinish; + + private long actualReadFromObs; + + private static final Logger LOG = LoggerFactory.getLogger(OBSInputStreamSimulator.class); + + public OBSInputStreamSimulator(final long fileStatusLength, final long readAheadRangeValue) { + Preconditions.checkArgument(fileStatusLength >= 0, "Negative content length"); + this.contentLength = fileStatusLength; + this.readAheadRange = readAheadRangeValue; + } + + private class InputStreamSimulator { + + private final int size; + + private int pos; + + public InputStreamSimulator(int bytes) { + size = bytes; + pos = 0; + } + + public int available() { + return size - pos; + } + + public long skip(long n) { + if (n <= 0) { + return 0; + } + + if (pos + n >= size) { + long skipped = size - pos; + pos = size; + return skipped; + } else { + pos += n; + return n; + } + } + } + + /** + * 模拟读取OBS的流量 + * @param len 需要读多少字节的数据 + * @return 实际上读取了多少OBS的数据 + */ + public synchronized long read(final int len) throws IOException { + LOG.debug("Simulator.read len: {}", len); + + actualReadFromObs = 0; + + if (len == 0) { + return 0; + } + boolean isTrue = this.contentLength == 0 || nextReadPos >= contentLength; + if (isTrue) { + LOG.debug("Simulator.read: no need to read"); + return -1; + } + + lazySeek(nextReadPos, len); + + // 这里模拟读取操作,假设读取len长度的数据全部成功,那么直接+len + if (mockInputStream == null) { + throw new IOException("mockInputStream closed, cannot read."); + } + int ioRead = Math.min(len, mockInputStream.available()); + streamCurrentPos += ioRead; + nextReadPos += ioRead; + + return actualReadFromObs; + } + + private void lazySeek(final long targetPos, final long len) throws IOException { + seekInStream(targetPos); + + if (isStreamClosed()) { + reopen(targetPos, len); + } + } + + private boolean isStreamClosed() { + return mockInputStream == null; + } + + private void seekInStream(final long targetPos) { + LOG.debug("Simulator.seekInStream targetPos: {}", targetPos); + + if (isStreamClosed()) { + LOG.debug("Simulator.seekInStream: the stream is not opened, seekInStream not operated"); + return; + } + long diff = targetPos - streamCurrentPos; + + if (diff == 0 && this.contentRangeFinish - this.streamCurrentPos > 0) { + LOG.debug("Simulator.seekInStream: the seek position does not require seek"); + return; + } + + if (diff > 0) { + int available = mockInputStream.available(); + long forwardSeekRange = Math.max(readAheadRange, available); + long remainingInCurrentRequest = this.contentRangeFinish - this.streamCurrentPos; + long forwardSeekLimit = min(remainingInCurrentRequest, forwardSeekRange); + boolean skipForward = remainingInCurrentRequest > 0 && diff <= forwardSeekLimit; + if (skipForward) { + long skippedOnce = mockInputStream.skip(diff); + while (diff > 0 && skippedOnce > 0) { + streamCurrentPos += skippedOnce; + diff -= skippedOnce; + skippedOnce = mockInputStream.skip(diff); + } + + if (streamCurrentPos == targetPos) { + return; + } + } + } + + closeStream(); + streamCurrentPos = targetPos; + } + + /** + * Simulate reopen operation in OBSInputStream + * @param targetPos the position which the stream should open from + * @param length the length of the stream + * @return the actual traffic when reading from OBS, unit: bytes + * @throws IOException + */ + public long reopen(final long targetPos, final long length) { + if (!isStreamClosed()) { + closeStream(); + } + + contentRangeFinish = calculateRequestLimit(targetPos, length, contentLength, readAheadRange); + long actualLength = contentRangeFinish - targetPos; + mockInputStream = new InputStreamSimulator((int)actualLength); + LOG.debug("Opened a simulated stream, param length: {}, length {}.", length, actualLength); + + actualReadFromObs += actualLength; + this.streamCurrentPos = targetPos; + return actualLength; + } + + /** + * 注意seek和seekInStream + * @param targetPos seek目标位置 + * @throws IOException + */ + public void seek(final long targetPos) throws IOException { + LOG.debug("Simulator.seek targetPos {} ", targetPos); + + if (targetPos > contentLength) { + LOG.error("The position of seek is beyond content length, targetPos:{}, contentLength:{}", + targetPos, contentLength); + throw new EOFException(String.format(Locale.ENGLISH, "%s %d", FSExceptionMessages.CANNOT_SEEK_PAST_EOF, targetPos)); + } + + if (targetPos < 0) { + LOG.error("The position of seek negative, targetPos:{}", targetPos); + throw new EOFException(String.format(Locale.ENGLISH, "%s %d", FSExceptionMessages.NEGATIVE_SEEK, targetPos)); + } + + if (this.contentLength <= 0) { + return; + } + nextReadPos = targetPos; + } + + protected void closeStream() { + LOG.debug("Closed a simulator stream"); + mockInputStream = null; + } + + public void close() throws IOException { + closeStream(); + } + + protected long calculateRequestLimit(final long targetPos, final long length, final long contentLength, + final long readAhead) { + return min(contentLength, length < 0 ? contentLength : targetPos + Math.max(readAhead, length)); + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectInfo.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectInfo.java new file mode 100644 index 0000000..072429e --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectInfo.java @@ -0,0 +1,20 @@ +package org.apache.hadoop.fs.obs.memartscc; + +/** + * 功能描述 + * + * @since 2021-05-20 + */ +public class ObjectInfo { + public String objectKey; + + public long modifyTime; + + public boolean isConsistencyCheck; + + public ObjectInfo(String objectKey, long modifyTime, boolean isConsistencyCheck) { + this.objectKey = objectKey; + this.modifyTime = modifyTime; + this.isConsistencyCheck = isConsistencyCheck; + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectShard.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectShard.java new file mode 100644 index 0000000..2d8a512 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/memartscc/ObjectShard.java @@ -0,0 +1,43 @@ +package org.apache.hadoop.fs.obs.memartscc; + +/** + * 功能描述 + * + * @since 2021-05-24 + */ +public class ObjectShard { + long start; + + long end; + + String[] hosts; + + int validHostNum; + + public ObjectShard(long start, long end, String[] hosts, int validHostNum) { + this.start = start; + this.end = end; + this.hosts = hosts; + this.validHostNum = validHostNum; + } + + public ObjectShard(String[] hosts) { + this.hosts = hosts; + } + + public long getStart() { + return start; + } + + public long getEnd() { + return end; + } + + public String[] getHosts() { + return hosts; + } + + public int getValidHostNum() { + return validHostNum; + } +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java index e525c6f..f944901 100644 --- a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenCapability.java @@ -9,6 +9,7 @@ * * @since 2021-09-15 */ +@Deprecated public interface DelegationTokenCapability { String getCanonicalServiceName(); diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenProvider.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenProvider.java new file mode 100644 index 0000000..4e94250 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/DelegationTokenProvider.java @@ -0,0 +1,21 @@ +package org.apache.hadoop.fs.obs.security; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.security.token.Token; + +import java.net.URI; + +/** + * description + * + * @since 2022-05-24 + */ +public interface DelegationTokenProvider { + + void initialize(FileSystem fs, URI uri, Configuration conf); + + Token getDelegationToken(String renewer); + + String getCanonicalServiceName(); +} diff --git a/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/ObsDelegationTokenManger.java b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/ObsDelegationTokenManger.java new file mode 100644 index 0000000..b87e000 --- /dev/null +++ b/hadoop-huaweicloud/src/main/java/org/apache/hadoop/fs/obs/security/ObsDelegationTokenManger.java @@ -0,0 +1,82 @@ +package org.apache.hadoop.fs.obs.security; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.OBSCommonUtils; +import org.apache.hadoop.fs.obs.OBSConstants; +import org.apache.hadoop.io.Text; +import org.apache.hadoop.security.Credentials; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; + +/** + * description + * + * @since 2022-05-24 + */ +public class ObsDelegationTokenManger { + private static final Logger logger = LoggerFactory.getLogger(ObsDelegationTokenManger.class); + private List providers = new ArrayList<>(); + private DelegationTokenProvider defaultProvider; + + public static boolean hasDelegationTokenProviders(Configuration conf) { + return UserGroupInformation.isSecurityEnabled() && OBSCommonUtils.isStringNotEmpty( + conf.getTrimmed(OBSConstants.DELEGATION_TOKEN_PROVIDERS, OBSConstants.DEFAULT_DELEGATION_TOKEN_PROVIDERS)); + } + + public void initialize(FileSystem fs, URI uri, Configuration conf) throws IOException { + List> providerClasses = Arrays.asList( + conf.getClasses(OBSConstants.DELEGATION_TOKEN_PROVIDERS, new Class[] {})); + + for (int i = 0; i < providerClasses.size(); i++) { + DelegationTokenProvider provider = null; + try { + provider = (DelegationTokenProvider)providerClasses.get(i).newInstance(); + } catch (InstantiationException | IllegalAccessException e) { + throw new IOException("Instantiation DelegationTokenProvider", e); + } + provider.initialize(fs, uri, conf); + if (i == 0) { + this.defaultProvider = provider; + } + this.providers.add(provider); + } + } + + public Token[] addDelegationTokens(String renewer, Credentials credentials) { + List> tokens = new ArrayList<>(); + for (DelegationTokenProvider provider : providers) { + final String serviceName = provider.getCanonicalServiceName(); + logger.info("get delegation token from provider {} with serviceName {}", + provider.getClass().getName(), serviceName); + if (serviceName != null) { + final Text service = new Text(serviceName); + Token token = credentials.getToken(service); + if (token == null) { + token = provider.getDelegationToken(renewer); + if (token != null) { + tokens.add(token); + credentials.addToken(service, token); + } + } + } + } + return tokens.toArray(new Token[tokens.size()]); + } + + public Token getDelegationToken(String renewer) { + return defaultProvider.getDelegationToken(renewer); + } + + public String getCanonicalServiceName() { + return defaultProvider.getCanonicalServiceName(); + } +} diff --git a/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier b/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier new file mode 100644 index 0000000..252e51e --- /dev/null +++ b/hadoop-huaweicloud/src/main/resources/META-INF/services/org.apache.hadoop.security.token.TokenIdentifier @@ -0,0 +1 @@ +org.apache.hadoop.fs.obs.memartscc.MemArtsCCDelegationTokenIdentifier \ No newline at end of file diff --git a/hadoop-huaweicloud/src/site/markdown/index.md b/hadoop-huaweicloud/src/site/markdown/index.md new file mode 100644 index 0000000..83cc403 --- /dev/null +++ b/hadoop-huaweicloud/src/site/markdown/index.md @@ -0,0 +1,379 @@ + + +# OBSA: HuaweiCloud OBS Adapter for Hadoop Support + + + +## Introduction + +The `hadoop-huaweicloud` module provides support for integration with the +[HuaweiCloud Object Storage Service (OBS)](https://www.huaweicloud.com/en-us/product/obs.html). +This support comes via the JAR file `hadoop-huaweicloud.jar`. + +## Features + +* Read and write data stored in a HuaweiCloud OBS account. +* Reference file system paths using URLs using the `obs` scheme. +* Present a hierarchical file system view by implementing the standard Hadoop `FileSystem` interface. +* Support multipart upload for a large file. +* Can act as a source of data in a MapReduce job, or a sink. +* Uses HuaweiCloud OBS’s Java SDK with support for latest OBS features and authentication schemes. +* Tested for scale. + +## Limitations + +Partial or no support for the following operations : + +* Symbolic link operations. +* Proxy users. +* File truncate. +* File concat. +* File checksum. +* File replication factor. +* Extended Attributes(XAttrs) operations. +* Snapshot operations. +* Storage policy. +* Quota. +* POSIX ACL. +* Delegation token operations. + +## Getting Started + +### Packages + +OBSA depends upon two JARs, alongside `hadoop-common` and its dependencies. + +* `hadoop-huaweicloud` JAR. +* `esdk-obs-java` JAR. + +The versions of `hadoop-common` and `hadoop-huaweicloud` must be identical. + +To import the libraries into a Maven build, add `hadoop-huaweicloud` JAR to the +build dependencies; it will pull in a compatible `esdk-obs-java` JAR. + +The `hadoop-huaweicloud` JAR *does not* declare any dependencies other than that +dependencies unique to it, the OBS SDK JAR. This is simplify excluding/tuning +Hadoop dependency JARs in downstream applications. The `hadoop-client` or +`hadoop-common` dependency must be declared. + + +```xml + + + 3.0.0 + + + + + org.apache.hadoop + hadoop-client + ${hadoop.version} + + + org.apache.hadoop + hadoop-huaweicloud + ${hadoop.version} + + +``` + +### Accessing OBS URLs + +Before access a URL, OBS implementation classes of Filesystem/AbstractFileSystem and +a region endpoint where a bucket is located shoud be configured as follows: + +```xml + + fs.obs.impl + org.apache.hadoop.fs.obs.OBSFileSystem + The OBS implementation class of the Filesystem. + + + + fs.AbstractFileSystem.obs.impl + org.apache.hadoop.fs.obs.OBS + The OBS implementation class of the AbstractFileSystem. + + + + fs.obs.endpoint + obs.region.myhuaweicloud.com + OBS region endpoint where a bucket is located. + +``` + +OBS URLs can then be accessed as follows: + +``` +obs:///path +``` + +The scheme `obs` identifies a URL on a Hadoop-compatible file system `OBSFileSystem` +backed by HuaweiCloud OBS. + +For example, the following +[FileSystem Shell](../hadoop-project-dist/hadoop-common/FileSystemShell.html) +commands demonstrate access to a bucket named `mybucket`. + +```bash +hadoop fs -mkdir obs://mybucket/testDir + +hadoop fs -put testFile obs://mybucket/testDir/testFile + +hadoop fs -cat obs://mybucket/testDir/testFile +test file content +``` + +For details on how to create a bucket, see +[**Help Center > Object Storage Service > Getting Started> Basic Operation Procedure**](https://support.huaweicloud.com/intl/en-us/qs-obs/obs_qs_0003.html) + +### Authenticating with OBS + +Except when interacting with public OBS buckets, the OBSA client +needs the credentials needed to interact with buckets. + +The client supports multiple authentication mechanisms. The simplest authentication mechanisms is +to provide OBS access key and secret key as follows. + +```xml + + fs.obs.access.key + OBS access key. + Omit for provider-based authentication. + + + + fs.obs.secret.key + OBS secret key. + Omit for provider-based authentication. + +``` + +**Do not share access key, secret key, and session token. They must be kept secret.** + +Custom implementations +of `com.obs.services.IObsCredentialsProvider` (see [**Creating an Instance of ObsClient**](https://support.huaweicloud.com/intl/en-us/sdk-java-devg-obs/en-us_topic_0142815570.html)) or +`org.apache.hadoop.fs.obs.BasicSessionCredential` may also be used for authentication. + +```xml + + fs.obs.security.provider + + Class name of security provider class which implements + com.obs.services.IObsCredentialsProvider, which will + be used to construct an OBS client instance as an input parameter. + + + + + fs.obs.credentials.provider + + lass nameCof credential provider class which implements + org.apache.hadoop.fs.obs.BasicSessionCredential, + which must override three APIs: getOBSAccessKeyId(), + getOBSSecretKey(), and getSessionToken(). + + +``` + +## General OBSA Client Configuration + +All OBSA client options are configured with options with the prefix `fs.obs.`. + +```xml + + fs.obs.connection.ssl.enabled + false + Enable or disable SSL connections to OBS. + + + + fs.obs.connection.maximum + 1000 + Maximum number of simultaneous connections to OBS. + + + + fs.obs.connection.establish.timeout + 120000 + Socket connection setup timeout in milliseconds. + + + + fs.obs.connection.timeout + 120000 + Socket connection timeout in milliseconds. + + + + fs.obs.idle.connection.time + 30000 + Socket idle connection time. + + + + fs.obs.max.idle.connections + 1000 + Maximum number of socket idle connections. + + + + fs.obs.socket.send.buffer + 256 * 1024 + Socket send buffer to be used in OBS SDK. Represented in bytes. + + + + fs.obs.socket.recv.buffer + 256 * 1024 + Socket receive buffer to be used in OBS SDK. Represented in bytes. + + + + fs.obs.threads.keepalivetime + 60 + Number of seconds a thread can be idle before being + terminated in thread pool. + + + + fs.obs.threads.max + 20 + Maximum number of concurrent active (part)uploads, + which each use a thread from thread pool. + + + + fs.obs.max.total.tasks + 20 + Number of (part)uploads allowed to the queue before + blocking additional uploads. + + + + fs.obs.delete.threads.max + 20 + Max number of delete threads. + + + + fs.obs.multipart.size + 104857600 + Part size for multipart upload. + + + + + fs.obs.multiobjectdelete.maximum + 1000 + Max number of objects in one multi-object delete call. + + + + + fs.obs.fast.upload.buffer + disk + Which buffer to use. Default is `disk`, value may be + `disk` | `array` | `bytebuffer`. + + + + + fs.obs.buffer.dir + dir1,dir2,dir3 + Comma separated list of directories that will be used to buffer file + uploads to. This option takes effect only when the option 'fs.obs.fast.upload.buffer' + is set to 'disk'. + + + + + fs.obs.fast.upload.active.blocks + 4 + Maximum number of blocks a single output stream can have active + (uploading, or queued to the central FileSystem instance's pool of queued + operations). + + + + + fs.obs.readahead.range + 1024 * 1024 + Bytes to read ahead during a seek() before closing and + re-opening the OBS HTTP connection. + + + + fs.obs.read.transform.enable + true + Flag indicating if socket connections can be reused by + position read. Set `false` only for HBase. + + + + fs.obs.list.threads.core + 30 + Number of core list threads. + + + + fs.obs.list.threads.max + 60 + Maximum number of list threads. + + + + fs.obs.list.workqueue.capacity + 1024 + Capacity of list work queue. + + + + fs.obs.list.parallel.factor + 30 + List parallel factor. + + + + fs.obs.trash.enable + false + Switch for the fast delete. + + + + fs.obs.trash.dir + The fast delete recycle directory. + + + + fs.obs.block.size + 128 * 1024 * 1024 + Default block size for OBS FileSystem. + + +``` + +## Testing the hadoop-huaweicloud Module +The `hadoop-huaweicloud` module includes a full suite of unit tests. +Most of the tests will run against the HuaweiCloud OBS. To run these +tests, please create `src/test/resources/auth-keys.xml` with OBS account +information mentioned in the above sections and the following properties. + +```xml + + fs.contract.test.fs.obs + obs://obsfilesystem-bucket + +``` \ No newline at end of file diff --git a/hadoop-huaweicloud/src/site/resources/css/site.css b/hadoop-huaweicloud/src/site/resources/css/site.css new file mode 100644 index 0000000..f830baa --- /dev/null +++ b/hadoop-huaweicloud/src/site/resources/css/site.css @@ -0,0 +1,30 @@ +/* +* Licensed to the Apache Software Foundation (ASF) under one or more +* contributor license agreements. See the NOTICE file distributed with +* this work for additional information regarding copyright ownership. +* The ASF licenses this file to You under the Apache License, Version 2.0 +* (the "License"); you may not use this file except in compliance with +* the License. You may obtain a copy of the License at +* +* http://www.apache.org/licenses/LICENSE-2.0 +* +* Unless required by applicable law or agreed to in writing, software +* distributed under the License is distributed on an "AS IS" BASIS, +* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +* See the License for the specific language governing permissions and +* limitations under the License. +*/ +#banner { + height: 93px; + background: none; +} + +#bannerLeft img { + margin-left: 30px; + margin-top: 10px; +} + +#bannerRight img { + margin: 17px; +} + diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/AbstractOBSTestBase.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/AbstractOBSTestBase.java new file mode 100644 index 0000000..18d1a02 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/AbstractOBSTestBase.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.contract.AbstractFSContractTestBase; +import org.apache.hadoop.fs.contract.ContractTestUtils; + +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.io.IOUtils; +import org.junit.Before; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.writeDataset; + +/** + * An extension of the contract test base set up for OBS tests. + */ +@Deprecated +public abstract class AbstractOBSTestBase extends AbstractFSContractTestBase { + + protected static final Logger LOG = + LoggerFactory.getLogger(AbstractOBSTestBase.class); + + @Override + protected AbstractFSContract createContract(Configuration conf) { + return new OBSContract(conf); + } + + @Override + public void teardown() throws Exception { + super.teardown(); + describe("closing file system"); + IOUtils.closeStream(getFileSystem()); + } + + @Before + public void nameThread() { + Thread.currentThread().setName("JUnit-" + methodName.getMethodName()); + } + + @Override + protected int getTestTimeoutMillis() { + return OBSTestConstants.OBS_TEST_TIMEOUT; + } + + protected Configuration getConfiguration() { + return getContract().getConf(); + } + + /** + * Get the filesystem as an OBS filesystem. + * + * @return the typecast FS + */ + @Override + public OBSFileSystem getFileSystem() { + return (OBSFileSystem) super.getFileSystem(); + } + + /** + * Describe a test in the logs. + * + * @param text text to print + * @param args arguments to format in the printing + */ + protected void describe(String text, Object... args) { + LOG.info("\n\n{}: {}\n", + methodName.getMethodName(), + String.format(text, args)); + } + + /** + * Write a file, read it back, validate the dataset. Overwrites the file if + * it is present + * + * @param name filename (will have the test path prepended to it) + * @param len length of file + * @return the full path to the file + * @throws IOException any IO problem + */ + protected Path writeThenReadFile(String name, int len) throws IOException { + Path path = path(name); + byte[] data = dataset(len, 'a', 'z'); + writeDataset(getFileSystem(), path, data, data.length, 1024 * 1024, + true); + ContractTestUtils.verifyFileContents(getFileSystem(), path, data); + return path; + } + + /** + * Assert that an exception failed with a specific status code. + * + * @param e exception + * @param code expected status code + * @throws OBSIOException rethrown if the status code does not match. + */ + protected void assertStatusCode(OBSIOException e, int code) + throws OBSIOException { + if (e.getCause().getResponseCode() != code) { + throw e; + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestListFiles.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestListFiles.java new file mode 100644 index 0000000..8f9611d --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestListFiles.java @@ -0,0 +1,248 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Random; +import java.util.Set; + +/** + * This class tests the FileStatus API. + */ +public class ITestListFiles { + static final long seed = 0xDEADBEEFL; + + private OBSFileSystem fs; + + private Configuration conf; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static Path TEST_DIR; + + final private static int FILE_LEN = 10; + + private static Path FILE1; + + private static Path DIR1; + + private static Path FILE2; + + private static Path FILE3; + + static { + setTestPaths(new Path(testRootPath, "main_")); + } + + /** + * Sets the root testing directory and reinitializes any additional test paths + * that are under the root. This method is intended to be called from a + * subclass's @BeforeClass method if there is a need to override the testing + * directory. + * + * @param testDir Path root testing directory + */ + protected static void setTestPaths(Path testDir) { + TEST_DIR = testDir; + FILE1 = new Path(TEST_DIR, "file1"); + DIR1 = new Path(TEST_DIR, "dir1"); + FILE2 = new Path(DIR1, "file2"); + FILE3 = new Path(DIR1, "file3"); + } + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + fs.delete(TEST_DIR, true); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + fs = null; + } + } + + private Path getTestPath(String testPath) { + return new Path(testRootPath + "/" + testPath); + } + + private static void writeFile(FileSystem fileSys, Path name, int fileSize) + throws IOException { + // Create and write a file that contains three blocks of data + FSDataOutputStream stm = fileSys.create(name); + byte[] buffer = new byte[fileSize]; + Random rand = new Random(seed); + rand.nextBytes(buffer); + stm.write(buffer); + stm.close(); + } + + /** + * Test when input path is a file + */ + @Test + public void testFile() throws IOException { + fs.mkdirs(TEST_DIR); + writeFile(fs, FILE1, FILE_LEN); + + RemoteIterator itor = fs.listFiles( + FILE1, true); + LocatedFileStatus stat = itor.next(); + assertFalse(itor.hasNext()); + assertTrue(stat.isFile()); + assertEquals(FILE_LEN, stat.getLen()); + assertEquals(fs.makeQualified(FILE1), stat.getPath()); + assertEquals(1, stat.getBlockLocations().length); + + itor = fs.listFiles(FILE1, false); + stat = itor.next(); + assertFalse(itor.hasNext()); + assertTrue(stat.isFile()); + assertEquals(FILE_LEN, stat.getLen()); + assertEquals(fs.makeQualified(FILE1), stat.getPath()); + assertEquals(1, stat.getBlockLocations().length); + + fs.delete(FILE1, true); + } + + /** + * Test when input path is a directory + */ + @Test + public void testDirectory() throws IOException { + fs.mkdirs(DIR1); + + // test empty directory + RemoteIterator itor = fs.listFiles( + DIR1, true); + assertFalse(itor.hasNext()); + itor = fs.listFiles(DIR1, false); + assertFalse(itor.hasNext()); + + // testing directory with 1 file + writeFile(fs, FILE2, FILE_LEN); + itor = fs.listFiles(DIR1, true); + LocatedFileStatus stat = itor.next(); + assertFalse(itor.hasNext()); + assertTrue(stat.isFile()); + assertEquals(FILE_LEN, stat.getLen()); + assertEquals(fs.makeQualified(FILE2), stat.getPath()); + assertEquals(1, stat.getBlockLocations().length); + + itor = fs.listFiles(DIR1, false); + stat = itor.next(); + assertFalse(itor.hasNext()); + assertTrue(stat.isFile()); + assertEquals(FILE_LEN, stat.getLen()); + assertEquals(fs.makeQualified(FILE2), stat.getPath()); + assertEquals(1, stat.getBlockLocations().length); + + // test more complicated directory + writeFile(fs, FILE1, FILE_LEN); + writeFile(fs, FILE3, FILE_LEN); + + Set filesToFind = new HashSet(); + filesToFind.add(fs.makeQualified(FILE1)); + filesToFind.add(fs.makeQualified(FILE2)); + filesToFind.add(fs.makeQualified(FILE3)); + + itor = fs.listFiles(TEST_DIR, true); + stat = itor.next(); + assertTrue(stat.isFile()); + assertTrue("Path " + stat.getPath() + " unexpected", + filesToFind.remove(stat.getPath())); + + stat = itor.next(); + assertTrue(stat.isFile()); + assertTrue("Path " + stat.getPath() + " unexpected", + filesToFind.remove(stat.getPath())); + + stat = itor.next(); + assertTrue(stat.isFile()); + assertTrue("Path " + stat.getPath() + " unexpected", + filesToFind.remove(stat.getPath())); + assertFalse(itor.hasNext()); + assertTrue(filesToFind.isEmpty()); + + itor = fs.listFiles(TEST_DIR, false); + stat = itor.next(); + assertTrue(stat.isFile()); + assertEquals(fs.makeQualified(FILE1), stat.getPath()); + assertFalse(itor.hasNext()); + + fs.delete(TEST_DIR, true); + } + + @Test + // 调用listStatus递归列举接口,recursive为true,返回目录及子目录下所有对象;recursive为false,只返回当前目录下对象 + public void testListStatusRecursive() throws Exception { + Path testDir = getTestPath("test_dir/"); + Path subDir = getTestPath("test_dir/sub_dir/"); + Path file1 = getTestPath("test_dir/file1"); + Path file2 = getTestPath("test_dir/sub_dir/file2"); + fs.mkdirs(testDir); + fs.mkdirs(subDir); + FSDataOutputStream outputStream = fs.create(file1, false); + outputStream.close(); + outputStream = fs.create(file2, false); + outputStream.close(); + + FileStatus[] objects = fs.listStatus(testDir, false); + assertEquals(2, objects.length); + + objects = fs.listStatus(testDir, true); + assertEquals(3, objects.length); + fs.delete(testDir, true); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppend.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppend.java new file mode 100644 index 0000000..ce35008 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppend.java @@ -0,0 +1,467 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.OutputStream; +import java.util.EnumSet; +import java.util.Random; +import java.util.UUID; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.NanoTimer; +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; +import static org.apache.hadoop.fs.contract.ContractTestUtils.bandwidth; +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyReceivedData; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ITestOBSAppend { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private int testBufferSize; + + private int modulus; + + private byte[] testBuffer; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024)); + fs = OBSTestUtils.createTestFileSystem(conf); + testBufferSize = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 128); + modulus = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_MODULUS_SIZE, 128); + testBuffer = new byte[testBufferSize]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % modulus); + } + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + fs = null; + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + /** + * 文件桶,对一个已经存在的文件做追加(每次小于multipart size),追加3次 + * + * @throws Exception + */ + @Test + public void testAppendPosixNormal001() throws Exception { + verifyAppend(1024 * 1024 * 10, 1024, 3, 4096); + } + + /** + * 文件桶,对一个已经存在的文件做追加(每次小于multipart size),追加10次 + * + * @throws Exception + */ + @Test + public void testAppendPosixNormal002() throws Exception { + verifyAppend(1024 * 1024 * 10, 1024, 10, 4096); + } + + /** + * 文件桶,对一个已经存在的文件做追加(每次大于multipart size),追加3次 + * + * @throws Exception + */ + @Test + public void testAppendPosixNormal003() throws Exception { + verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 10, 3, 4096); + } + + /** + * 文件桶,对一个已经存在的文件做追加(每次大于multipart size),追加10次 + * + * @throws Exception + */ + @Test + public void testAppendPosixNormal004() throws Exception { + verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 10, 10, 4096); + } + + /** + * 文件桶,追加多次,小于和大于100M交替 + * + * @throws Exception + */ + @Test + public void testAppendPosixNormal005() throws Exception { + verifyCrossSizeAppend(1024 * 1024 * 10, 1024, 1024 * 1024 * 200, + 1024 * 1024 * 100, 3, 4096); + } + + /** + * 文件桶,追加多次,每次追加随机大小 + * + * @throws Exception + */ + @Test + public void testAppendPosixNormal006() throws Exception { + verifyMultiSizeAppend(1024 * 1024 * 10, 1024, 1024 * 1024 * 200, 3, + 4096); + } + + /** + * 文件桶,追加多次,每次追加1字节 + * + * @throws Exception + */ + @Test + public void testAppendPosixNormal007() throws Exception { + verifyAppend(1024 * 1024 * 10, 1, 30, 4096); + } + + /** + * 文件桶,追加写大于100M,中间中断后继续追加 + * @throws Exception + */ + // @Test + // public void testAppendPosixNormal008() throws Exception { + // + // } + + /** + * 对象桶创建append stream,抛UnsupportedOperationException + * + * @throws Exception + */ + @Test + public void testAppendPosixAbnormal001() throws Exception { + if (fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + boolean hasException = false; + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + try { + outputStream = fs.append(testFile, 4096, null); + } catch (UnsupportedOperationException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertTrue( + "create append stream in object bucket should throw UnsupportedOperationException", + hasException); + + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + /** + * 路径为一个不存在的文件,抛出FileNotFoundException + * + * @throws Exception + */ + @Test + public void testAppendPosixAbnormal002() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + boolean hasException = false; + FSDataOutputStream outputStream = null; + try { + outputStream = fs.append(testFile, 4096, null); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertTrue( + "append on non exist file should throw FileNotFoundException", + hasException); + } + + /** + * 路径为一个目录,抛出FileAlreadyExistsException + * + * @throws Exception + */ + @Test + public void testAppendPosixAbnormal003() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testPath = getTestPath("test-dir"); + fs.mkdirs(testPath); + boolean hasException = false; + FSDataOutputStream outputStream = null; + try { + outputStream = fs.append(testPath, 4096, null); + } catch (FileAlreadyExistsException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertTrue( + "append on an exist directory should throw FileAlreadyExistsException", + hasException); + OBSFSTestUtil.deletePathRecursive(fs, testPath); + } + + /** + * 文件的父目录及上级目录不存在,抛FileNotFoundException + * + * @throws Exception + */ + @Test + public void testAppendPosixAbnormal004() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testPath = getTestPath("a001/b001/test_file"); + fs.delete(testPath.getParent(), true); + boolean hasException = false; + FSDataOutputStream outputStream = null; + try { + outputStream = fs.append(testPath, 4096, null); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertTrue( + "append on file whose parent not exists should throw FileNotFoundException", + hasException); + } + + /** + * 文件的父目录及上级目录非目录,抛AccessControlException + * + * @throws Exception + */ + @Test + public void testAppendPosixAbnormal005() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testPath = getTestPath("a001/b001/test_file"); + fs.delete(testPath.getParent(), true); + FSDataOutputStream outputStream = fs.create(testPath.getParent(), + false); + outputStream.close(); + + boolean hasException = false; + try { + outputStream = fs.append(testPath, 4096, null); + } catch (AccessControlException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertTrue( + "append on file whose parent is not directory should throw AccessControlException", + hasException); + } + + private void verifyAppend(long fileSize, long appendSize, int appendTimes, + int bufferSize) + throws IOException { + if (!fs.isFsBucket()) { + return; + } + long total = fileSize; + Path objectPath = createAppendFile(fileSize); + for (int i = 0; i < appendTimes; i++) { + appendFileByBuffer(objectPath, appendSize, bufferSize, total); + total = total + appendSize; + } + verifyReceivedData(fs, objectPath, total, testBufferSize, modulus); + OBSFSTestUtil.deletePathRecursive(fs, objectPath); + } + + private void verifyCrossSizeAppend(long fileSize, int appendMinSize, + int appendMaxSize, + int mediumSize, int appendTimes, int bufferSize) + throws IOException { + if (!fs.isFsBucket()) { + return; + } + long totoal = fileSize; + Random random = new Random(); + Path objectPath = createAppendFile(fileSize); + for (int i = 0; i < appendTimes; i++) { + int appendSize = + random.nextInt(appendMaxSize) % (appendMaxSize - appendMinSize + + 1) + appendMinSize; + if (i % 2 == 0) { + appendSize = appendSize > mediumSize + ? appendSize - mediumSize + : appendSize; + } else { + appendSize = appendSize < mediumSize + ? appendSize + mediumSize + : appendSize; + } + appendFileByBuffer(objectPath, appendSize, bufferSize, totoal); + totoal = totoal + appendSize; + } + verifyReceivedData(fs, objectPath, totoal, testBufferSize, modulus); + } + + private void verifyMultiSizeAppend(long fileSize, int appendMinSize, + int appendMaxSize, + int appendTimes, int bufferSize) + throws IOException { + if (!fs.isFsBucket()) { + return; + } + long totoal = fileSize; + Random random = new Random(); + Path objectPath = createAppendFile(fileSize); + for (int i = 0; i < appendTimes; i++) { + int appendSize = + random.nextInt(appendMaxSize) % (appendMaxSize - appendMinSize + + 1) + appendMinSize; + appendFileByBuffer(objectPath, appendSize, bufferSize, totoal); + totoal = totoal + appendSize; + } + verifyReceivedData(fs, objectPath, totoal, testBufferSize, modulus); + } + + private Path createAppendFile(long fileSize) throws IOException { + String objectName = UUID.randomUUID().toString(); + Path objectPath = getTestPath(objectName); + NanoTimer timer = new NanoTimer(); + + OutputStream outputStream = createAppendStream(objectPath); + writStream(outputStream, fileSize, 0); + bandwidth(timer, fileSize); + assertPathExists(fs, "not created successful", objectPath); + return objectPath; + } + + private void appendFileByBuffer(Path objectPath, long appendSize, + int bufferSize, long offset) throws IOException { + OutputStream outputStream = fs.append(objectPath, bufferSize, null); + writStream(outputStream, appendSize, offset); + assertPathExists(fs, "not created successful", objectPath); + } + + private void writStream(OutputStream outputStream, long fileSize, + long offset) throws IOException { + long bytesWritten = 0L; + Throwable var10 = null; + long diff; + try { + int off = (int) (offset % testBuffer.length); + while (bytesWritten < fileSize) { + diff = fileSize - bytesWritten; + if (diff + off <= (long) testBuffer.length) { + outputStream.write(testBuffer, off, (int) diff); + bytesWritten += diff; + break; + } else { + outputStream.write(testBuffer, off, + (testBuffer.length - off)); + bytesWritten += (long) testBuffer.length - off; + } + off = 0; + } + } catch (Throwable var21) { + var10 = var21; + throw var21; + } finally { + if (outputStream != null) { + if (var10 != null) { + try { + outputStream.close(); + } catch (Throwable var20) { + var10.addSuppressed(var20); + } + } else { + outputStream.close(); + } + } + + } + assertEquals(fileSize, bytesWritten); + } + + private FSDataOutputStream createAppendStream(Path objectPath) + throws IOException { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + FsPermission permission = new FsPermission((short) 00644); + return fs.create(objectPath, permission, flags, + fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(objectPath), + fs.getDefaultBlockSize(objectPath), + null, null); + } + + @Test + public void testAppendNormal() throws IOException { + if (fs.getMetricSwitch()) { + long fileSize = 10; + long appendSize = 10; + int bufferSize = 4096; + if (!fs.isFsBucket()) { + return; + } + long total = fileSize + appendSize; + + Path objectPath = createAppendFile(fileSize); + // appendFileByBuffer(objectPath, appendSize, bufferSize, total); + OutputStream outputStream = fs.append(objectPath, bufferSize, null); + writStream(outputStream, appendSize, total); + + OBSFSTestUtil.deletePathRecursive(fs, objectPath); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppendOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppendOutputStream.java new file mode 100644 index 0000000..d965bda --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSAppendOutputStream.java @@ -0,0 +1,271 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.EnumSet; +import java.util.UUID; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.*; +import static org.junit.Assert.assertEquals; + +public class ITestOBSAppendOutputStream { + + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private int testBufferSize; + + private int modulus; + + private byte[] testBuffer; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.FAST_UPLOAD, true); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024)); + fs = OBSTestUtils.createTestFileSystem(conf); + testBufferSize = fs.getConf().getInt( + ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 128); + modulus = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_MODULUS_SIZE + + ".size", 128); + testBuffer = new byte[testBufferSize]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % modulus); + } + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath() { + return new Path(testRootPath + "/test-obs"); + } + + private Path getRenameTestPath() { + return new Path(testRootPath + "/test-obs-rename"); + } + + @Test + // append空文件,校验append后大小 + public void testZeroSizeFileAppend() throws Exception { + if (!fs.isFsBucket()) { + return; + } + verifyAppend(0, 1024, 3, 4096); + } + + @Test + // 单次write小于缓存块大小,多次append,写满本地缓存块,触发上传到服务端,校验大小 + public void testBellowMultipartSizeAppend() throws Exception { + if (!fs.isFsBucket()) { + return; + } + // 在128字节 buffer写(单次write小于设置的5MB缓存的append),即会走OBSBlockOutputStream的275行逻辑 + verifyAppend(1024 * 1024, 1024 * 1024 * 80, 3, 4096); + verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 4, 3, 4096); + verifyAppend(1024 * 1024, 1024, 10, 4096); + verifyAppend(1024 * 1024, 1024 * 1024, 3, 4096); + } + + @Test + // 单次write大于缓存块大小,多次append,写满本地缓存块,触发上传到服务端,校验大小 + public void testAboveMultipartSizeAppend() throws Exception { + if (!fs.isFsBucket()) { + return; + } + // 在6MB buffer写(单次write大于设置的5MB缓存的append),即会走OBSBlockOutputStream的265行逻辑 + testBufferSize = fs.getConf() + .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 6 * 1024 * 1024); + modulus = fs.getConf().getInt(ContractTestUtils.IO_CHUNK_MODULUS_SIZE, 128); + testBuffer = new byte[testBufferSize]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % modulus); + } + verifyAppend(1024 * 1024, 1024 * 1024 * 11, 3, 6 * 1024 * 1024); + verifyAppend(1024 * 1024 * 10, 1024 * 1024 * 10, 3, 6 * 1024 * 1024); + verifyAppend(1024 * 1024, 1024, 10, 6 * 1024 * 1024); + verifyAppend(1024 * 1024, 1024 * 1024, 3, 6 * 1024 * 1024); + } + + @Test + // 文件rename后,append目标文件,校验大小正常 + public void testAppendAfterRename() throws Exception { + if (!fs.isFsBucket()) { + return; + } + long fileSize = 1024 * 1024; + long total = fileSize; + int appendTimes = 3; + long appendSize = 1024; + Path objectPath = createAppendFile(fileSize); + for (int i = 0; i < appendTimes; i++) { + appendFile(objectPath, appendSize, total); + total = total + appendSize; + } + + verifyReceivedData(fs, objectPath, total, testBufferSize, modulus); + + String objectName = objectPath.getName(); + Path renamePath = new Path(getRenameTestPath(), objectName); + fs.mkdirs(getRenameTestPath()); + fs.rename(objectPath, renamePath); + + for (int i = 0; i < appendTimes; i++) { + appendFile(renamePath, appendSize, total); + total = total + appendSize; + } + + verifyReceivedData(fs, renamePath, total, testBufferSize, modulus); + } + + @Test + // append流被关闭后,再写数据,抛出IOException + public void testAppendAfterClose() throws IOException { + if (!fs.isFsBucket()) { + return; + } + + Path dest = createAppendFile(1024); + OutputStream stream = creatAppendStream(dest); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + try { + stream.write(data); + stream.close(); + boolean hasException = false; + try { + stream.write(data); + } catch (IOException e) { + hasException = true; + } + assertTrue(hasException); + + } finally { + fs.delete(dest, false); + if (stream != null) { + stream.close(); + } + } + } + + private void verifyAppend(long fileSize, long appendSize, int appendTimes, + int bufferSize) throws IOException { + if (!fs.isFsBucket()) { + return; + } + long total = fileSize; + Path objectPath = createAppendFile(fileSize); + for (int i = 0; i < appendTimes; i++) { + appendFileByBuffer(objectPath, appendSize, bufferSize, total); + total = total + appendSize; + } + verifyReceivedData(fs, objectPath, total, testBufferSize, modulus); + } + + private Path createAppendFile(long fileSize) throws IOException { + + String objectName = UUID.randomUUID().toString(); + Path objectPath = new Path(getTestPath(), objectName); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + + OutputStream outputStream = creatAppendStream(objectPath); + writStream(outputStream, fileSize, 0); + bandwidth(timer, fileSize); + assertPathExists(fs, "not created successful", objectPath); + return objectPath; + } + + private void appendFile(Path objectPath, long appendSize, long offset) + throws IOException { + OutputStream outputStream = fs.append(objectPath, 4096, null); + writStream(outputStream, appendSize, offset); + assertPathExists(fs, "not created successful", objectPath); + } + + private void appendFileByBuffer(Path objectPath, long appendSize, + int bufferSize, long offset) throws IOException { + OutputStream outputStream = fs.append(objectPath, bufferSize, null); + writStream(outputStream, appendSize, offset); + assertPathExists(fs, "not created successful", objectPath); + } + + private void writStream(OutputStream outputStream, long fileSize, + long offset) throws IOException { + long bytesWritten = 0L; + Throwable var10 = null; + long diff; + try { + int off = (int) (offset % testBuffer.length); + while (bytesWritten < fileSize) { + diff = fileSize - bytesWritten; + if (diff + off <= (long) testBuffer.length) { + outputStream.write(testBuffer, off, (int) diff); + bytesWritten += diff; + break; + } else { + outputStream.write(testBuffer, off, + (testBuffer.length - off)); + bytesWritten += (long) testBuffer.length - off; + } + off = 0; + } + } catch (Throwable var21) { + var10 = var21; + throw var21; + } finally { + if (outputStream != null) { + if (var10 != null) { + try { + outputStream.close(); + } catch (Throwable var20) { + var10.addSuppressed(var20); + } + } else { + outputStream.close(); + } + } + + } + assertEquals(fileSize, bytesWritten); + } + + private OutputStream creatAppendStream(Path objectPath) throws IOException { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + FsPermission permission = new FsPermission((short) 00644); + return fs.create(objectPath, permission, flags, + fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(objectPath), + fs.getDefaultBlockSize(objectPath), null); + } +} + diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSArrayBufferOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSArrayBufferOutputStream.java new file mode 100644 index 0000000..76ff757 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSArrayBufferOutputStream.java @@ -0,0 +1,220 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.obs.OBSConstants.FAST_UPLOAD_BUFFER; +import static org.apache.hadoop.fs.obs.OBSConstants.FAST_UPLOAD_BUFFER_ARRAY; +import static org.apache.hadoop.fs.obs.OBSConstants.FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT; +import static org.apache.hadoop.fs.obs.OBSConstants.MULTIPART_MIN_SIZE; +import static org.apache.hadoop.fs.obs.OBSConstants.MULTIPART_SIZE; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.io.IOUtils; +import org.junit.After; +import org.junit.Assume; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.io.IOException; +import java.io.InputStream; +import java.util.Arrays; +import java.util.Collection; + +/** + * Tests small file upload functionality for {@link OBSBlockOutputStream} with + * the blocks buffered in byte arrays. + *

+ * File sizes are kept small to reduce test duration on slow connections; + * multipart tests are kept in scale tests. + */ +@Deprecated +@RunWith(Parameterized.class) +public class ITestOBSArrayBufferOutputStream extends AbstractOBSTestBase { + private static final int BLOCK_SIZE = 256 * 1024; + + private static byte[] dataset; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private boolean calcMd5; + + @Parameterized.Parameters + public static Collection calcMd5() { + return Arrays.asList(false, true); + } + + public ITestOBSArrayBufferOutputStream(boolean calcMd5) { + this.calcMd5 = calcMd5; + } + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Override + protected Configuration createConfiguration() { + Configuration conf = super.createConfiguration(); + OBSTestUtils.disableFilesystemCaching(conf); + conf.setInt(MULTIPART_SIZE, MULTIPART_MIN_SIZE); + conf.setBoolean(OBSConstants.FAST_UPLOAD, true); + conf.set(FAST_UPLOAD_BUFFER, getBlockOutputBufferName()); + conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, calcMd5); + return conf; + } + + @After + public void tearDown() throws Exception { + if (getFileSystem() != null) { + getFileSystem().delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + protected String getBlockOutputBufferName() { + return FAST_UPLOAD_BUFFER_ARRAY; + } + + @Test + // array内存缓存,首个缓存块大小为1M + public void testArrayFirstBufferSize() throws Exception { + Path dest = getTestPath("testMockRetryError"); + FSDataOutputStream stream = getFileSystem().create(dest, true); + OBSBlockOutputStream obs + = (OBSBlockOutputStream) stream.getWrappedStream(); + OBSDataBlocks.ByteArrayBlock block + = (OBSDataBlocks.ByteArrayBlock) obs.getActiveBlock(); + assertTrue(String.format("Array first buffer must 1MB. real (%d)", + block.firstBlockSize()), + block.firstBlockSize() + == FAST_UPLOAD_BUFFER_ARRAY_FIRST_BLOCK_SIZE_DEFAULT); + if (stream != null) { + stream.close(); + } + } + + @Test + // 上传0字节的文件 + public void testZeroByteUpload() throws IOException { + verifyUpload("0", 0); + } + + @Test + // 上传1K大小文件 + public void testRegularUpload() throws IOException { + verifyUpload("regular", 1024); + } + + @Test + // 流关闭后再写数据,抛IOException + public void testArrayWriteAfterStreamClose() throws IOException { + Path dest = getTestPath("testWriteAfterStreamClose"); + describe(" testWriteAfterStreamClose"); + FSDataOutputStream stream = getFileSystem().create(dest, true); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + try { + stream.write(data); + stream.close(); + boolean hasException = false; + try { + stream.write(data); + } catch (IOException e) { + hasException = true; + } + assertTrue(hasException); + } finally { + IOUtils.closeStream(stream); + } + } + + @Test + // 流关闭后,activeBlock置为空 + public void testBlocksClosed() throws Throwable { + Path dest = getTestPath("testBlocksClosed"); + describe(" testBlocksClosed"); + FSDataOutputStream stream = getFileSystem().create(dest, true); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + OBSBlockOutputStream obsStream = + (OBSBlockOutputStream) stream.getWrappedStream(); + assertEquals(null, obsStream.getActiveBlock()); + } + + private void verifyUpload(String name, int fileSize) throws IOException { + Path dest = getTestPath(name); + describe(name + " upload to " + dest); + ContractTestUtils.createAndVerifyFile( + getFileSystem(), + dest, + fileSize); + } + + /** + * Create a factory for used in mark/reset tests. + * + * @param fileSystem source FS + * @return the factory + */ + protected OBSDataBlocks.BlockFactory createFactory( + OBSFileSystem fileSystem) { + return new OBSDataBlocks.ByteArrayBlockFactory(fileSystem); + } + + private void markAndResetDatablock(OBSDataBlocks.BlockFactory factory) + throws Exception { + OBSDataBlocks.DataBlock block = factory.create(1, BLOCK_SIZE); + dataset = ContractTestUtils.dataset(BLOCK_SIZE, 0, 256); + block.write(dataset, 0, dataset.length); + InputStream stream = (InputStream) block.startUpload(); + assertNotNull(stream); + assertTrue("Mark not supported in " + stream, stream.markSupported()); + assertEquals(0, stream.read()); + stream.mark(BLOCK_SIZE); + // read a lot + long l = 0; + while (stream.read() != -1) { + // do nothing + l++; + } + stream.reset(); + assertEquals(1, stream.read()); + } + + @Test + // byte array流支持mark和reset操作 + public void testMarkReset() throws Throwable { + markAndResetDatablock(createFactory(getFileSystem())); + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockSize.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockSize.java new file mode 100644 index 0000000..a1a83f9 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockSize.java @@ -0,0 +1,126 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; + +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static junit.framework.TestCase.assertTrue; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createFile; +import static org.apache.hadoop.fs.contract.ContractTestUtils.dataset; +import static org.apache.hadoop.fs.contract.ContractTestUtils.fileStatsToString; +import static org.junit.Assert.assertEquals; + +/** + * OBS tests for configuring block size. + */ +public class ITestOBSBlockSize { + private FileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static final Logger LOG = + LoggerFactory.getLogger(ITestOBSBlockSize.class); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String testPath) { + return new Path(fs.getUri() + testRootPath + testPath); + } + + @Test + @SuppressWarnings("deprecation") + // getDefaultBlockSize返回128M,并且在已存在的fs实例修改conf配置,不会生效 + public void testBlockSize() throws Exception { + long defaultBlockSize = fs.getDefaultBlockSize(); + assertEquals("incorrect blocksize", + OBSConstants.DEFAULT_FS_OBS_BLOCK_SIZE, defaultBlockSize); + long newBlockSize = defaultBlockSize * 2; + fs.getConf().setLong(OBSConstants.FS_OBS_BLOCK_SIZE, newBlockSize); + + Path dir = getTestPath("/testBlockSize"); + Path file = new Path(dir, "file"); + createFile(fs, file, true, dataset(1024, 'a', 'z' - 'a')); + FileStatus fileStatus = fs.getFileStatus(file); + assertEquals( + "modify configuration in exist fs won't have effect: " + fileStatus, + defaultBlockSize, + fileStatus.getBlockSize()); + + // check the listing & assert that the block size is picked up by + // this route too. + boolean found = false; + FileStatus[] listing = fs.listStatus(dir); + fs.delete(file, false); + for (FileStatus stat : listing) { + LOG.info("entry: {}", stat); + if (file.equals(stat.getPath())) { + found = true; + assertEquals( + "modify configuration in exist fs won't have effect: " + + stat, + defaultBlockSize, + stat.getBlockSize()); + } + } + assertTrue("Did not find " + fileStatsToString(listing, ", "), found); + + } + + @Test + // 根目录blocksize大小为0 + public void testRootFileStatusBlockSize() throws Throwable { + FileStatus status = fs.getFileStatus(new Path("/")); + assertTrue("Invalid root blocksize", + status.getBlockSize() == 0); + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockingThreadPoolExecutorService.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockingThreadPoolExecutorService.java new file mode 100644 index 0000000..1d6646a --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBlockingThreadPoolExecutorService.java @@ -0,0 +1,223 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +import com.google.common.util.concurrent.ListenableFuture; + +import org.apache.hadoop.util.StopWatch; +import org.junit.AfterClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.TimeUnit; + +/** + * Basic test for OBS's blocking executor service. + */ +public class ITestOBSBlockingThreadPoolExecutorService { + + private static final Logger LOG = LoggerFactory.getLogger( + BlockingThreadPoolExecutorService.class); + + private static final int NUM_ACTIVE_TASKS = 4; + + private static final int NUM_WAITING_TASKS = 2; + + private static final int TASK_SLEEP_MSEC = 100; + + private static final int SHUTDOWN_WAIT_MSEC = 200; + + private static final int SHUTDOWN_WAIT_TRIES = 5; + + private static final int BLOCKING_THRESHOLD_MSEC = 50; + + private static final Integer SOME_VALUE = 1337; + + private static BlockingThreadPoolExecutorService tpe; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout(60 * 1000); + + @AfterClass + public static void afterClass() throws Exception { + ensureDestroyed(); + } + + /** + * Basic test of running one trivial task. + */ + @Test + // 线程池可正常提交Callable任务 + public void testSubmitCallable() throws Exception { + ensureCreated(); + ListenableFuture f = tpe.submit(callableSleeper); + Integer v = f.get(); + assertEquals(SOME_VALUE, v); + } + + /** + * More involved test, including detecting blocking when at capacity. + */ + @Test + // 线程池可正常提交Runnable任务 + public void testSubmitRunnable() throws Exception { + ensureCreated(); + verifyQueueSize(tpe, NUM_ACTIVE_TASKS + NUM_WAITING_TASKS); + } + + /** + * Verify the size of the executor's queue, by verifying that the first + * submission to block is {@code expectedQueueSize + 1}. + * + * @param executorService executor service to test + * @param expectedQueueSize size of queue + */ + protected void verifyQueueSize(ExecutorService executorService, + int expectedQueueSize) { + StopWatch stopWatch = new StopWatch().start(); + for (int i = 0; i < expectedQueueSize; i++) { + executorService.submit(sleeper); + assertDidntBlock(stopWatch); + } + executorService.submit(sleeper); + assertDidBlock(stopWatch); + } + + @Test + // 线程池可正常shutdown + public void testShutdown() throws Exception { + // Cover create / destroy, regardless of when this test case runs + ensureCreated(); + ensureDestroyed(); + + // Cover create, execute, destroy, regardless of when test case runs + ensureCreated(); + testSubmitRunnable(); + ensureDestroyed(); + } + + @Test + // 线程池任务可排队 + public void testChainedQueue() throws Throwable { + ensureCreated(); + int size = 2; + ExecutorService wrapper = new SemaphoredDelegatingExecutor(tpe, + size, true); + verifyQueueSize(wrapper, size); + } + + // Helper functions, etc. + + private void assertDidntBlock(StopWatch sw) { + try { + assertFalse("Non-blocking call took too long.", + sw.now(TimeUnit.MILLISECONDS) > BLOCKING_THRESHOLD_MSEC); + } finally { + sw.reset().start(); + } + } + + private void assertDidBlock(StopWatch sw) { + try { + if (sw.now(TimeUnit.MILLISECONDS) < BLOCKING_THRESHOLD_MSEC) { + throw new RuntimeException("Blocking call returned too fast."); + } + } finally { + sw.reset().start(); + } + } + + private Runnable sleeper = new Runnable() { + @Override + public void run() { + String name = Thread.currentThread().getName(); + try { + Thread.sleep(TASK_SLEEP_MSEC); + } catch (InterruptedException e) { + LOG.info("Thread {} interrupted.", name); + Thread.currentThread().interrupt(); + } + } + }; + + private Callable callableSleeper = new Callable() { + @Override + public Integer call() throws Exception { + sleeper.run(); + return SOME_VALUE; + } + }; + + /** + * Helper function to create thread pool under test. + */ + private synchronized static void ensureCreated() throws Exception { + if (tpe == null) { + LOG.debug("Creating thread pool"); + tpe = BlockingThreadPoolExecutorService.newInstance( + NUM_ACTIVE_TASKS, NUM_WAITING_TASKS, + 1, "btpetest"); + } + } + + /** + * Helper function to terminate thread pool under test, asserting that + * shutdown -> terminate works as expected. + */ + private static void ensureDestroyed() throws Exception { + if (tpe == null) { + return; + } + int shutdownTries = SHUTDOWN_WAIT_TRIES; + + tpe.shutdown(); + if (!tpe.isShutdown()) { + throw new RuntimeException("Shutdown had no effect."); + } + + while (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC, + TimeUnit.MILLISECONDS)) { + LOG.info("Waiting for thread pool shutdown."); + if (shutdownTries-- <= 0) { + LOG.error("Failed to terminate thread pool gracefully."); + break; + } + } + if (!tpe.isTerminated()) { + tpe.shutdownNow(); + if (!tpe.awaitTermination(SHUTDOWN_WAIT_MSEC, + TimeUnit.MILLISECONDS)) { + throw new RuntimeException( + "Failed to terminate thread pool in timely manner."); + } + } + tpe = null; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketAcl.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketAcl.java new file mode 100644 index 0000000..8da6a3b --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketAcl.java @@ -0,0 +1,217 @@ +package org.apache.hadoop.fs.obs; + +import com.obs.services.ObsClient; +import com.obs.services.model.*; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.util.Progressable; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.File; +import java.io.IOException; +import java.io.OutputStream; +import java.util.EnumSet; +import java.util.Set; +import java.util.UUID; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.*; + +public class ITestOBSBucketAcl { + private OBSFileSystem fs; + + private int testBufferSize; + + private int modulus; + + private byte[] testBuffer; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + String vAccountId = null; + + String permissionStr = null; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + testBufferSize = fs.getConf().getInt("io.chunk.buffer.size", 128); + modulus = fs.getConf().getInt("io.chunk.modulus.size", 128); + testBuffer = new byte[testBufferSize]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % modulus); + } + vAccountId = "domainiddomainiddomainiddo000001"; + permissionStr = String.valueOf(Permission.PERMISSION_FULL_CONTROL); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath() { + return new Path(testRootPath + "/test-obs"); + } + + @Test + // 上传文件,桶owner有FULL_CONTROL权限 + public void testUploadFileBucketOwnerPermission() throws IOException { + if (!fs.isFsBucket()) { + return; + } + verifyAppendAndAcl(1024 * 1024, 1024, 3); + } + + @Test + // 创建目录,桶owner有FULL_CONTROL权限 + public void testMkDirBucketOwnerPermission() throws IOException { + if (!fs.isFsBucket()) { + return; + } + Path dirPath = new Path(getTestPath(), "mkdir-zh"); + fs.mkdirs(dirPath); + String key = dirPath.toString().substring(1); + verifyAcl(key, vAccountId, permissionStr); + } + + @Test + // 从本地copy文件,桶owner有FULL_CONTROL权限 + public void testCopyFromLocalBucketOwnerPermission() throws IOException { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile); + Path localPath = new Path(localFile); + Path dstPath = new Path(testRootPath + "/copylocal-huge"); + + fs.copyFromLocalFile(false, true, localPath, dstPath); + String key = dstPath.toString().substring(1); + verifyAcl(key, vAccountId, permissionStr); + OBSFSTestUtil.deleteLocalFile(localFile); + } + + private void verifyAppendAndAcl(long fileSize, long appendSize, + int appendTimes) throws IOException { + long total = fileSize; + Path objectPath = createAppendFile(fileSize); + + for (int i = 0; i < appendTimes; i++) { + appendFile(objectPath, appendSize); + total = total + appendSize; + } + verifyReceivedData(fs, objectPath, total, testBufferSize, modulus); + String key = objectPath.toString().substring(1); + verifyAcl(key, vAccountId, permissionStr); + } + + private void verifyAcl(String key, String vAccountId, + String permissionStr) { + ObsClient client = fs.getObsClient(); + AccessControlList controlList = client.getObjectAcl(fs.getBucket(), + key); + Set grants = controlList.getGrants(); + boolean aclOk = false; + for (GrantAndPermission grant : grants) { + String accountId = grant.getGrantee().getIdentifier(); + String permission = grant.getPermission().getPermissionString(); + if (accountId.equals(vAccountId) && permission.equals( + permissionStr)) { + aclOk = true; + break; + } + } + assertTrue(aclOk); + } + + private void appendFile(Path objectPath, long appendSize) + throws IOException { + OutputStream outputStream = fs.append(objectPath, 4096, null); + writStream(outputStream, appendSize); + assertPathExists(fs, "not created successful", objectPath); + } + + private void writStream(OutputStream outputStream, long fileSize) + throws IOException { + long bytesWritten = 0L; + Throwable var10 = null; + long diff; + try { + while (bytesWritten < fileSize) { + diff = fileSize - bytesWritten; + if (diff < (long) testBuffer.length) { + outputStream.write(testBuffer, 0, (int) diff); + bytesWritten += diff; + } else { + outputStream.write(testBuffer); + bytesWritten += (long) testBuffer.length; + } + } + + diff = bytesWritten; + } catch (Throwable var21) { + var10 = var21; + throw var21; + } finally { + if (outputStream != null) { + if (var10 != null) { + try { + outputStream.close(); + } catch (Throwable var20) { + var10.addSuppressed(var20); + } + } else { + outputStream.close(); + } + } + + } + assertEquals(fileSize, diff); + } + + private Path createAppendFile(long fileSize) throws IOException { + + String objectName = UUID.randomUUID().toString(); + Path objectPath = new Path(getTestPath(), objectName); + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + + OutputStream outputStream = creatAppendStream(objectPath); + writStream(outputStream, fileSize); + bandwidth(timer, fileSize); + assertPathExists(fs, "not created successful", objectPath); + return objectPath; + } + + private OutputStream creatAppendStream(Path objectPath) throws IOException { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + FsPermission permission = new FsPermission((short) 00644); + return fs.create(objectPath, permission, flags, + fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(objectPath), + fs.getDefaultBlockSize(objectPath), (Progressable) null); + + } + +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketPolicy.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketPolicy.java new file mode 100644 index 0000000..cd7d226 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSBucketPolicy.java @@ -0,0 +1,402 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.codehaus.jettison.json.JSONArray; +import org.codehaus.jettison.json.JSONException; +import org.codehaus.jettison.json.JSONObject; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; +import java.util.concurrent.TimeUnit; + +public class ITestOBSBucketPolicy { + /** + * Resources_s3 前缀 + */ + private static final String RES_S3_PREFIX = "arn:aws:s3:::"; + + /** + * Resources_IAM 前缀 + */ + private static final String RES_IAM_PREFIX = "arn:aws:iam::"; + + private static final String NEW_USER_AK = "UDSIAMSTUBTEST000999"; + + private static final String NEW_USER_SK + = "Udsiamstubtest000000UDSIAMSTUBTEST000999"; + + private static final String NEW_USER_DOMAIN_ID + = "domainiddomainiddomainiddo000999"; + + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + fs.delete(new Path(testRootPath), true); + fs.mkdirs(new Path(testRootPath), new FsPermission((short) 00644)); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + fs = null; + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + private JSONObject getOtherTenantAccessPolicy(Path testDir) throws JSONException { + return new JSONObject().put("Statement", new JSONArray().put(new JSONObject().put("Effect", "Allow") + .put("Sid", "1") + .put("Principal", + new JSONObject().put("AWS", new JSONArray().put(RES_IAM_PREFIX + NEW_USER_DOMAIN_ID + ":root"))) + .put("Action", new JSONArray().put("*")) + .put("Resource", new JSONArray() + .put(RES_S3_PREFIX + fs.getBucket()) + .put(RES_S3_PREFIX + fs.getBucket() + testRootPath + "/") + .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir) + "/*") + .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir)) + ))); + } + + @Test + // 配置policy后,创建目录权限校验 + public void testMkdir() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + assertTrue(newFs.mkdirs(testDir, new FsPermission((short) 00644))); + + newFs.delete(testDir, true); + } + + private void setPolicy(JSONObject testDir) throws InterruptedException { + fs.setBucketPolicy(testDir.toString()); + // policy 有缓存,需要3秒才能生效 + TimeUnit.SECONDS.sleep(5); + } + + @Test + // 配置policy后,listStatus权限校验 + public void testListStatus() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + assertEquals(2, newFs.listStatus(testDir).length); + assertEquals(3, newFs.listStatus(testDir, true).length); + + fs.delete(testDir, true); + } + + @Test + // 配置policy后,listFiles权限校验 + public void testListFiles() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + FSDataOutputStream outputStream = fs.create(getTestPath("test_dir" + + "/subdir1/file"), false); + outputStream.close(); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + outputStream = fs.create(getTestPath("test_dir" + + "/subdir1/subsubdir1/file"), false); + outputStream.close(); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + outputStream = fs.create(getTestPath("test_dir" + + "/subdir2/file"), false); + outputStream.close(); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + RemoteIterator iterator = newFs.listFiles(testDir, + true); + int fileNum = 0; + while (iterator.hasNext()) { + iterator.next(); + fileNum++; + } + assertEquals(3, fileNum); + + fs.delete(testDir, true); + } + + @Test + // 配置policy后,ListLocatedStatus权限校验 + public void testListLocatedStatus() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + FSDataOutputStream outputStream = fs.create(getTestPath("test_dir" + + "/subdir1/file"), false); + outputStream.close(); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + outputStream = fs.create(getTestPath("test_dir" + + "/subdir1/subsubdir1/file"), false); + outputStream.close(); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + outputStream = fs.create(getTestPath("test_dir" + + "/subdir2/file"), false); + outputStream.close(); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + RemoteIterator iterator = newFs.listLocatedStatus( + testDir); + int fileNum = 0; + while (iterator.hasNext()) { + iterator.next(); + fileNum++; + } + assertEquals(2, fileNum); + + fs.delete(testDir, true); + } + + @Test + // 配置policy后,GetContentSummary权限校验 + public void testGetContentSummary() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + assertEquals(4, newFs.getContentSummary(testDir).getDirectoryCount()); + + fs.delete(testDir, true); + } + + @Test + // 配置policy后,GetFileStatus权限校验 + public void testGetFileStatus() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + assertTrue(newFs.getFileStatus(testDir).isDirectory()); + + fs.delete(testDir, true); + } + + @Test + // 配置policy后,Exist权限校验 + public void testExist() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + assertTrue(newFs.exists(testDir)); + + fs.delete(testDir, true); + } + + @Test + // 配置policy后,Delete权限校验 + public void testDelete() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + + setPolicy(getOtherTenantAccessPolicy(testDir)); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + assertTrue(newFs.delete(testDir, true)); + } + + @Test + // 配置policy后,Rename权限校验 + public void testRename() throws IOException, JSONException, InterruptedException { + if(!fs.isFsBucket()) { + return; + } + + Path testDir = getTestPath("test_dir"); + Path renameToDir = getTestPath("test_dir_to"); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir1/subsubdir1"), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(getTestPath("test_dir/subdir2"), + new FsPermission((short) 00644))); + + setPolicy(new JSONObject().put("Statement", new JSONArray().put(new JSONObject().put("Effect", "Allow") + .put("Sid", "1") + .put("Principal", + new JSONObject().put("AWS", new JSONArray().put(RES_IAM_PREFIX + NEW_USER_DOMAIN_ID + ":root"))) + .put("Action", new JSONArray().put("*")) + .put("Resource", new JSONArray().put(RES_S3_PREFIX + fs.getBucket()) + .put(RES_S3_PREFIX + fs.getBucket() + testRootPath + "/") + .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir) + "/*") + .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, testDir)) + .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, renameToDir) + "/*") + .put(RES_S3_PREFIX + fs.getBucket() + "/" + OBSCommonUtils.pathToKey(fs, renameToDir)))))); + + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.ACCESS_KEY, NEW_USER_AK); + conf.set(OBSConstants.SECRET_KEY, NEW_USER_SK); + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(conf); + + assertTrue(newFs.rename(testDir, renameToDir)); + + fs.delete(renameToDir, true); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseFunction.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseFunction.java new file mode 100644 index 0000000..3153eae --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseFunction.java @@ -0,0 +1,114 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.lang.reflect.Field; +import java.util.concurrent.ThreadPoolExecutor; + +public class ITestOBSCloseFunction { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath)); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // close时应关闭所有的OutputStream + public void testOutputStreamClose() throws Exception { + Path testFile = getTestPath("test_file"); + fs.delete(testFile); + FSDataOutputStream outputStream = fs.create(testFile, false); + OBSBlockOutputStream obsBlockOutputStream + = (OBSBlockOutputStream) outputStream.getWrappedStream(); + Field field = obsBlockOutputStream.getClass() + .getDeclaredField("closed"); + field.setAccessible(true); + boolean closed = (boolean) field.get(obsBlockOutputStream); + assertFalse(closed); + fs.close(); + closed = (boolean) field.get(obsBlockOutputStream); + fs = null; + assertTrue(closed); + } + + @Test + // finalize接口释放实例申请的各类资源 + public void testThreadPoolClose() throws Throwable { + + Field boundedCopyThreadPoolField = fs.getClass() + .getDeclaredField("boundedCopyThreadPool"); + boundedCopyThreadPoolField.setAccessible(true); + ThreadPoolExecutor boundedCopyThreadPool + = (ThreadPoolExecutor) boundedCopyThreadPoolField.get(fs); + + Field boundedDeleteThreadPoolField = fs.getClass() + .getDeclaredField("boundedDeleteThreadPool"); + boundedDeleteThreadPoolField.setAccessible(true); + ThreadPoolExecutor boundedDeleteThreadPool + = (ThreadPoolExecutor) boundedDeleteThreadPoolField.get(fs); + + Field boundedCopyPartThreadPoolField = fs.getClass() + .getDeclaredField("boundedCopyPartThreadPool"); + boundedCopyPartThreadPoolField.setAccessible(true); + ThreadPoolExecutor boundedCopyPartThreadPool + = (ThreadPoolExecutor) boundedCopyPartThreadPoolField.get(fs); + + Field boundedListThreadPoolField = fs.getClass() + .getDeclaredField("boundedListThreadPool"); + boundedListThreadPoolField.setAccessible(true); + ThreadPoolExecutor boundedListThreadPool + = (ThreadPoolExecutor) boundedListThreadPoolField.get(fs); + + fs.close(); + fs = null; + + if (boundedCopyThreadPool != null) { + assertTrue(boundedCopyThreadPool.isShutdown()); + } + if (boundedDeleteThreadPool != null) { + assertTrue(boundedDeleteThreadPool.isShutdown()); + } + if (boundedCopyPartThreadPool != null) { + assertTrue(boundedCopyPartThreadPool.isShutdown()); + } + if (boundedListThreadPool != null) { + assertTrue(boundedListThreadPool.isShutdown()); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseProtect.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseProtect.java new file mode 100644 index 0000000..8d415c6 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCloseProtect.java @@ -0,0 +1,947 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSExceptionMessages; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.junit.Assert.*; + +public class ITestOBSCloseProtect { + private static final Logger LOG = LoggerFactory.getLogger( + ITestOBSRename.class); + private OBSFileSystem fs; + + private static String testRootPath = OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + final int partSize = 5 * 1024 * 1024; + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(partSize)); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + if (fs.exists(new Path(testRootPath))) { + fs.delete(new Path(testRootPath), true); + } + } + + @After + public void tearDown() throws Exception { + } + + @Test + // fs关闭后再执行open操作,抛出IOException + public void testOpenAfterFSClose() throws IOException { + String fileName = "file"; + Path filePath = new Path(testRootPath + "/" + fileName); + + fs.close(); + + boolean closeException = false; + try { + fs.open(filePath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.open(filePath, 100); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行create操作,抛出IOException + public void testCreateAfterFSClose() throws IOException { + String fileName = "file"; + Path filePath = new Path(testRootPath + "/" + fileName); + + fs.close(); + + boolean closeException = false; + try { + fs.create(filePath, new FsPermission((short) 00644), false, 4096, + (short) 3, 128 * 1024 * 1024, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, new FsPermission((short) 00644), null, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, true); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, (short) 3); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, (short) 3, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, true, 4096); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, true, 4096, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, true, 4096, (short) 3, + 128 * 1024 * 1024); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.create(filePath, true, 4096, (short) 3, + 128 * 1024 * 1024, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行CreateNonRecursive操作,抛出IOException + public void testCreateNonRecursiveAfterFSClose() throws IOException { + String fileName = "file"; + Path filePath = new Path(testRootPath + "/" + fileName); + + fs.close(); + + boolean closeException = false; + try { + fs.createNonRecursive(filePath, new FsPermission((short) 00644), + true, 4096, (short) 3, 128 * 1024 * 1024, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.createNonRecursive(filePath, new FsPermission((short) 00644), + null, 4096, (short) 3, 128 * 1024 * 1024, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行append操作,抛出IOException + public void testAppendAfterFSClose() throws IOException { + String fileName = "file"; + Path filePath = new Path(testRootPath + "/" + fileName); + + fs.close(); + + boolean closeException = false; + try { + fs.append(filePath, 4096, null); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.append(filePath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行truncate操作,抛出IOException + public void testTruncateAfterFSClose() throws IOException { + String fileName = "file"; + Path filePath = new Path(testRootPath + "/" + fileName); + + fs.close(); + + boolean closeException = false; + try { + fs.truncate(filePath, 4096); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行exists操作,抛出IOException + public void testExistsAfterFSClose() throws IOException { + String fileName = "file"; + Path filePath = new Path(testRootPath + "/" + fileName); + + fs.close(); + + boolean closeException = false; + try { + fs.exists(filePath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行rename操作,抛出IOException + public void testRenameAfterFSClose() throws IOException { + String fileNameFrom = "fileFrom"; + Path filePathFrom = new Path( + testRootPath + "/" + fileNameFrom); + String fileNameTo = "fileTo"; + Path filePathTo = new Path( + testRootPath + "/" + fileNameTo); + + fs.close(); + + boolean closeException = false; + try { + fs.rename(filePathFrom, filePathTo); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行delete操作,抛出IOException + public void testDeleteAfterFSClose() throws IOException { + String dirName = "dir"; + Path dirPath = new Path(testRootPath + "/" + dirName); + + fs.close(); + + boolean closeException = false; + try { + fs.delete(dirPath, true); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行ListStatus操作,抛出IOException + public void testListStatusAfterFSClose() throws IOException { + String dirName = "dir"; + Path dirPath = new Path(testRootPath + "/" + dirName); + + fs.close(); + + boolean closeException = false; + try { + fs.listStatus(dirPath, true); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + fs.listStatus(dirPath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行Mkdirs操作,抛出IOException + public void testMkdirsAfterFSClose() throws IOException { + String dirName = "dir"; + Path dirPath = new Path(testRootPath + "/" + dirName); + + fs.close(); + + boolean closeException = false; + try { + fs.mkdirs(dirPath, new FsPermission((short) 00644)); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行GetFileStatus操作,抛出IOException + public void testGetFileStatusAfterFSClose() throws IOException { + String fileName = "file"; + Path filePath = new Path(testRootPath + "/" + fileName); + + fs.close(); + + boolean closeException = false; + try { + fs.getFileStatus(filePath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行GetContentSummary操作,抛出IOException + public void testGetContentSummaryAfterFSClose() throws IOException { + String dirName = "dir"; + Path dirPath = new Path(testRootPath + "/" + dirName); + + fs.close(); + + boolean closeException = false; + try { + fs.getContentSummary(dirPath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行CopyFromLocalFile操作,抛出IOException + public void testCopyFromLocalFileAfterFSClose() throws IOException { + String fileNameLocal = "fileLocal"; + Path filePathLocal = new Path("/usr/" + fileNameLocal); + String fileNameRemote = "fileRemote"; + Path filePathRemote = new Path( + testRootPath + "/" + fileNameRemote); + + fs.close(); + + boolean closeException = false; + try { + fs.copyFromLocalFile(true, true, filePathLocal, filePathRemote); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行ListFiles操作,抛出IOException + public void testListFilesAfterFSClose() throws IOException { + String dirName = "dir"; + Path dirPath = new Path(testRootPath + "/" + dirName); + + fs.close(); + + boolean closeException = false; + try { + fs.listFiles(dirPath, true); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行ListLocatedStatus操作,抛出IOException + public void testListLocatedStatusAfterFSClose() throws IOException { + String dirName = "dir"; + Path dirPath = new Path(testRootPath + "/" + dirName); + + fs.close(); + + boolean closeException = false; + try { + fs.listLocatedStatus(dirPath); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行InputStream相关操作,抛出IOException + public void testInputStreamAfterFSClose() throws Exception { + final int bufLen = 256 * 1024; + final int sizeMB = 10; + String fileName = "readTestFile_" + sizeMB + ".txt"; + Path readTestFile = new Path( + testRootPath + "/" + fileName); + long size = sizeMB * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256, + 255); + + FSDataInputStream instream = this.fs.open(readTestFile); + + fs.close(); + + boolean closeException = false; + try { + instream.getPos(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.seek(1024); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.seekToNewSource(1024); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.read(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + ByteBuffer byteBuffer = ByteBuffer.allocateDirect(bufLen); + instream.read(byteBuffer); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] buf = new byte[1024]; + instream.read(buf, 0, 1024); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.close(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.available(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] buf = new byte[1024]; + instream.readFully(1024, buf, 0, 1024); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns()); + } + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] buf = new byte[1024]; + instream.read(1024, buf, 0, 1024); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.setReadahead(1024L); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // fs关闭后再执行OutputStream相关操作,抛出IOException + public void testOutStreamAfterFSClose() throws Exception { + String fileName = "file"; + Path filePath = new Path( + testRootPath + "/" + fileName); + FSDataOutputStream outputStream = fs.create(filePath); + outputStream.write(10); + + fs.close(); + + boolean closeException = false; + try { + outputStream.close(); + } catch (IOException e) { + closeException = true; + } + assertFalse(closeException); + + closeException = false; + try { + outputStream.flush(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.write(10); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] testBuffer = new byte[1024]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % 255); + } + outputStream.write(testBuffer, 0 , 100); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.hsync(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.hflush(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.hsync(); + } catch (IOException e) { + if (e.getMessage().equals("OBSFilesystem closed")) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // InputStream关闭后再执行InputStream相关操作,抛出IOException + public void testInputStreamClose() throws Exception { + final int bufLen = 256 * 1024; + final int sizeMB = 10; + String fileName = "readTestFile_" + sizeMB + ".txt"; + Path readTestFile = new Path( + testRootPath + "/" + fileName); + long size = sizeMB * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256, + 255); + + FSDataInputStream instream = this.fs.open(readTestFile); + + instream.close(); + + boolean closeException = false; + try { + instream.getPos(); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.seek(1024); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.seekToNewSource(1024); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.read(); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + ByteBuffer byteBuffer = ByteBuffer.allocateDirect(bufLen); + instream.read(byteBuffer); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] buf = new byte[1024]; + instream.read(buf, 0, 1024); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.available(); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] buf = new byte[1024]; + instream.readFully(1024, buf, 0, 1024); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns()); + } + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] buf = new byte[1024]; + instream.read(1024, buf, 0, 1024); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns()); + } + } + } + assertTrue(closeException); + + closeException = false; + try { + instream.setReadahead(1024L); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + } + + @Test + // OutputStream关闭后再执行OutputStream相关操作,抛出IOException + public void testOutStreamClose() throws Exception { + String fileName = "file"; + Path filePath = new Path( + testRootPath + "/" + fileName); + FSDataOutputStream outputStream = fs.create(filePath); + outputStream.write(10); + + outputStream.close(); + + boolean closeException = false; + try { + outputStream.flush(); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.write(10); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + byte[] testBuffer = new byte[1024]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % 255); + } + outputStream.write(testBuffer, 0 , 100); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.hsync(); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.hflush(); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + + closeException = false; + try { + outputStream.hsync(); + } catch (IOException e) { + if (e.getMessage().contains(FSExceptionMessages.STREAM_IS_CLOSED)) { + closeException = true; + } + } + assertTrue(closeException); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSConfiguration.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSConfiguration.java new file mode 100644 index 0000000..7b398b0 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSConfiguration.java @@ -0,0 +1,518 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import com.obs.services.ObsClient; +import com.obs.services.internal.ObsProperties; +import com.obs.services.internal.RestConnectionService; +import com.obs.services.internal.RestStorageService; + +import org.apache.commons.lang.reflect.FieldUtils; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.ProviderUtils; +import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.alias.CredentialProvider; +import org.apache.hadoop.security.alias.CredentialProviderFactory; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.lang.reflect.Field; +import java.net.URI; +import java.security.PrivilegedExceptionAction; +import java.util.Collection; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.apache.hadoop.fs.obs.OBSConstants.OBS_SECURITY_CREDENTIAL_PROVIDER_PATH; +import static org.apache.hadoop.fs.obs.OBSConstants.USER_AGENT_PREFIX; +import static org.apache.hadoop.fs.obs.OBSTestConstants.TEST_FS_OBS_NAME; +import static org.apache.hadoop.fs.obs.OBSTestUtils.assertOptionEquals; +import static org.apache.hadoop.fs.obs.OBSTestUtils.setBucketOption; +import static org.apache.hadoop.fs.obs.OBSCommonUtils.CREDENTIAL_PROVIDER_PATH; +import static org.apache.hadoop.fs.obs.OBSCommonUtils.patchSecurityCredentialProviders; +import static org.apache.hadoop.fs.obs.OBSCommonUtils.propagateBucketOptions; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNotEquals; +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.fail; + +/** + * OBS tests for configuration. + */ +public class ITestOBSConfiguration { + private static final String EXAMPLE_ID = "AKASOMEACCESSKEY"; + + private static final String EXAMPLE_KEY = + "RGV0cm9pdCBSZ/WQgY2xl/YW5lZCB1cAEXAMPLE"; + + private Configuration conf; + + private OBSFileSystem fs; + + private static final Logger LOG = + LoggerFactory.getLogger(ITestOBSConfiguration.class); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout( + OBSTestConstants.OBS_TEST_TIMEOUT + ); + + @Rule + public final TemporaryFolder tempDir = new TemporaryFolder(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf);; + } + + @Test + // 使用无效的proxy host和port,创建fs会超时失败 + public void testProxyConnection() throws Exception { + conf.setInt(OBSConstants.MAX_ERROR_RETRIES, 2); + conf.set(OBSConstants.PROXY_HOST, "127.0.0.1"); + conf.setInt(OBSConstants.PROXY_PORT, 1); + String proxy = + conf.get(OBSConstants.PROXY_HOST) + ":" + conf.get( + OBSConstants.PROXY_PORT); + try { + fs = OBSTestUtils.createTestFileSystem(conf); + fail("Expected a connection error for proxy server at " + proxy); + } catch (OBSIOException e) { + // expected + } + } + + @Test + // proxy未指定host时,创建fs会超时失败 + public void testProxyPortWithoutHost() throws Exception { + conf.setInt(OBSConstants.MAX_ERROR_RETRIES, 2); + conf.setInt(OBSConstants.PROXY_PORT, 1); + try { + fs = OBSTestUtils.createTestFileSystem(conf); + fail("Expected a proxy configuration error"); + } catch (OBSIOException e) { + String msg = e.toString(); + if (!msg.contains(OBSConstants.PROXY_HOST) && + !msg.contains(OBSConstants.PROXY_PORT)) { + //expected + } + } + } + + void provisionAccessKeys(final Configuration conf) throws Exception { + // add our creds to the provider + final CredentialProvider provider = + CredentialProviderFactory.getProviders(conf).get(0); + provider.createCredentialEntry(OBSConstants.ACCESS_KEY, + EXAMPLE_ID.toCharArray()); + provider.createCredentialEntry(OBSConstants.SECRET_KEY, + EXAMPLE_KEY.toCharArray()); + provider.flush(); + } + + // 从userInfo中解析credentials,能正确解析出ak、sk + public void testCredsFromUserInfo() throws Exception { + final File file = tempDir.newFile("test.jks"); + final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider( + file.toURI()); + conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, + jks.toString()); + + LOG.warn("file URI: {}, jks: {}", file.toURI(), jks.toString()); + System.out.println("file URI: " + file.toURI() + ", jks: " + "jceks://obs/foobar," + jks.toString()); + + LOG.warn("provider: {}", conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH)); + System.out.println("provider: " + conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH)); + + try { + provisionAccessKeys(conf); + } catch (Exception e) { + e.printStackTrace(); + LOG.error(e.getMessage()); + System.exit(-1); + } + + conf.set(OBSConstants.ACCESS_KEY, EXAMPLE_ID + "LJM"); + URI uriWithUserInfo = new URI("obs://123:456@foobar"); + OBSLoginHelper.Login creds = + OBSCommonUtils.getOBSAccessKeys(uriWithUserInfo, conf); + assertEquals("AccessKey incorrect.", "123", creds.getUser()); + assertEquals("SecretKey incorrect.", "456", creds.getPassword()); + } + + // 测试可正常排除不兼容的S3ACredentialProvider + public void testExcludingS3ACredentialProvider() throws Exception { + // set up conf to have a cred provider + final File file = tempDir.newFile("test.jks"); + final URI jks = ProviderUtils.nestURIForLocalJavaKeyStoreProvider( + file.toURI()); + conf.set(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH, + "jceks://obs/foobar," + jks.toString()); + + LOG.warn("file URI: {}, jks: {}", file.toURI(), jks.toString()); + System.out.println("file URI: " + file.toURI() + ", jks: " + "jceks://obs/foobar," + jks.toString()); + + LOG.warn("provider: {}", conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH)); + System.out.println("provider: " + conf.get(CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH)); + + // first make sure that the obs based provider is removed + Configuration c = ProviderUtils.excludeIncompatibleCredentialProviders( + conf, OBSFileSystem.class); + String newPath = conf.get( + CredentialProviderFactory.CREDENTIAL_PROVIDER_PATH); + assertFalse("Provider Path incorrect", newPath.contains("obs://")); + + LOG.warn("after exclude, newPath: {}", newPath); + System.out.println("after exclude, newPath: " + newPath); + + // now let's make sure the new path is created by the OBSFileSystem + // and the integration still works. Let's provision the keys through + // the altered configuration instance and then try and access them + // using the original config with the obs provider in the path. + try { + provisionAccessKeys(c); + } catch (Exception e) { + e.printStackTrace(); + System.exit(-1); + } + + conf.set(OBSConstants.ACCESS_KEY, EXAMPLE_ID + "LJM"); + URI uriWithUserInfo = new URI("obs://123:456@foobar"); + OBSLoginHelper.Login creds = + OBSCommonUtils.getOBSAccessKeys(uriWithUserInfo, conf); + assertEquals("AccessKey incorrect.", "123", creds.getUser()); + assertEquals("SecretKey incorrect.", "456", creds.getPassword()); + + } + + // @Test + // public void testDirectoryAllocatorRR() throws Throwable { + // String localDir1 = "local_dir1"; + // OBSFSTestUtil.createLocalTestDir(localDir1); + // String localDir2 = "local_dir2"; + // OBSFSTestUtil.createLocalTestDir(localDir2); + // File dir1 = new File(localDir1); + // File dir2 = new File(localDir2); + // dir1.mkdirs(); + // dir2.mkdirs(); + // conf.set(OBSConstants.BUFFER_DIR, dir1 + ", " + dir2); + // fs = OBSTestUtils.createTestFileSystem(conf); + // File tmp1 = createTmpFileForWrite("out-", 1024, conf); + // tmp1.delete(); + // File tmp2 = createTmpFileForWrite("out-", 1024, conf); + // tmp2.delete(); + // assertNotEquals("round robin not working", + // tmp1.getParent(), tmp2.getParent()); + // + // File tmp = createTmpFileForWrite("out-", 1024, conf); + // assertTrue("not found: " + tmp, tmp.exists()); + // tmp.delete(); + // } + + @Test + // 测试readahead.range可正常配置 + public void testReadAheadRange() throws Exception { + conf.set(OBSConstants.READAHEAD_RANGE, "300K"); + fs = OBSTestUtils.createTestFileSystem(conf); + assertNotNull(fs); + long readAheadRange = fs.getReadAheadRange(); + assertNotNull(readAheadRange); + assertEquals("Read Ahead Range Incorrect.", 300 * 1024, readAheadRange); + } + + @Test + // 测试可正常获取UGI信息 + public void testUsernameFromUGI() throws Throwable { + final String alice = "alice"; + UserGroupInformation fakeUser = + UserGroupInformation.createUserForTesting(alice, + new String[] {"users", "administrators"}); + fs = fakeUser.doAs(new PrivilegedExceptionAction() { + @Override + public OBSFileSystem run() throws Exception { + return OBSTestUtils.createTestFileSystem(conf); + } + }); + assertEquals("username", alice, fs.getShortUserName()); + FileStatus status = fs.getFileStatus(new Path("/")); + assertEquals("owner in " + status, alice, status.getOwner()); + assertEquals("group in " + status, alice, status.getGroup()); + } + + /** + * Reads and returns a field from an object using reflection. If the field + * cannot be found, is null, or is not the expected type, then this method + * fails the test. + * + * @param target object to read + * @param fieldType type of field to read, which will also be the return type + * @param fieldName name of field to read + * @return field that was read + * @throws IllegalAccessException if access not allowed + */ + private static T getField(Object target, Class fieldType, + String fieldName) throws IllegalAccessException { + Object obj = FieldUtils.readField(target, fieldName, true); + assertNotNull(String.format( + "Could not read field named %s in object with class %s.", fieldName, + target.getClass().getName()), obj); + assertTrue(String.format( + "Unexpected type found for field named %s, expected %s, actual %s.", + fieldName, fieldType.getName(), obj.getClass().getName()), + fieldType.isAssignableFrom(obj.getClass())); + return fieldType.cast(obj); + } + + @Test + // 测试可正常将桶特定配置转化成通用obs配置 + public void testBucketConfigurationPropagation() throws Throwable { + Configuration config = new Configuration(false); + setBucketOption(config, "b", "base", "1024"); + String basekey = "fs.obs.base"; + assertOptionEquals(config, basekey, null); + String bucketKey = "fs.obs.bucket.b.base"; + assertOptionEquals(config, bucketKey, "1024"); + Configuration updated = propagateBucketOptions(config, "b"); + assertOptionEquals(updated, basekey, "1024"); + // original conf is not updated + assertOptionEquals(config, basekey, null); + + String[] sources = updated.getPropertySources(basekey); + assertEquals(1, sources.length); + String sourceInfo = sources[0]; + assertTrue("Wrong source " + sourceInfo, + sourceInfo.contains(bucketKey)); + } + + @Test + // 测试可正常解析配置文件中定义的变量字段 + public void testBucketConfigurationPropagationResolution() + throws Throwable { + Configuration config = new Configuration(false); + String basekey = "fs.obs.base"; + String baseref = "fs.obs.baseref"; + String baseref2 = "fs.obs.baseref2"; + config.set(basekey, "orig"); + config.set(baseref2, "${fs.obs.base}"); + setBucketOption(config, "b", basekey, "1024"); + setBucketOption(config, "b", baseref, "${fs.obs.base}"); + Configuration updated = propagateBucketOptions(config, "b"); + assertOptionEquals(updated, basekey, "1024"); + assertOptionEquals(updated, baseref, "1024"); + assertOptionEquals(updated, baseref2, "1024"); + } + + @Test + // key重复的桶配置项,解析到的值为最后一个配置项的值 + public void testMultipleBucketConfigurations() throws Throwable { + Configuration config = new Configuration(false); + setBucketOption(config, "b", USER_AGENT_PREFIX, "UA-b"); + setBucketOption(config, "c", USER_AGENT_PREFIX, "UA-c"); + config.set(USER_AGENT_PREFIX, "UA-orig"); + Configuration updated = propagateBucketOptions(config, "c"); + assertOptionEquals(updated, USER_AGENT_PREFIX, "UA-c"); + } + + @Test + // 传递桶特定配置到OBS通用配置时,不会覆盖OBS通用配置中相同配置项 + public void testBucketConfigurationSkipsUnmodifiable() throws Throwable { + Configuration config = new Configuration(false); + String impl = "fs.obs.impl"; + config.set(impl, "orig"); + setBucketOption(config, "b", impl, "b"); + String metastoreImpl = "fs.obs.metadatastore.impl"; + String ddb = "org.apache.hadoop.fs.obs.s3guard.DynamoDBMetadataStore"; + setBucketOption(config, "b", metastoreImpl, ddb); + setBucketOption(config, "b", "impl2", "b2"); + setBucketOption(config, "b", "bucket.b.loop", "b3"); + assertOptionEquals(config, "fs.obs.bucket.b.impl", "b"); + + Configuration updated = propagateBucketOptions(config, "b"); + assertOptionEquals(updated, impl, "orig"); + assertOptionEquals(updated, "fs.obs.impl2", "b2"); + assertOptionEquals(updated, metastoreImpl, ddb); + assertOptionEquals(updated, "fs.obs.bucket.b.loop", null); + } + + @Test + // 桶特定配置传递到OBS通用配置中,fs能正确识别到 + public void testConfOptionPropagationToFS() throws Exception { + String testFSName = conf.getTrimmed(TEST_FS_OBS_NAME, ""); + String bucket = new URI(testFSName).getHost(); + setBucketOption(conf, bucket, "propagation", "propagated"); + fs = OBSTestUtils.createTestFileSystem(conf); + Configuration updated = fs.getConf(); + assertOptionEquals(updated, "fs.obs.propagation", "propagated"); + } + + @Test + // 添加的obs CredentialProviders不会覆盖hadoop CredentialProviders + public void testSecurityCredentialPropagationNoOverride() { + conf.set(CREDENTIAL_PROVIDER_PATH, "base"); + patchSecurityCredentialProviders(conf); + assertOptionEquals(conf, CREDENTIAL_PROVIDER_PATH, + "base"); + } + + @Test + // 未设置hadoop CredentialProviders时,添加的obs CredentialProviders为全集 + public void testSecurityCredentialPropagationOverrideNoBase() { + conf.unset(CREDENTIAL_PROVIDER_PATH); + conf.set(OBS_SECURITY_CREDENTIAL_PROVIDER_PATH, "override"); + patchSecurityCredentialProviders(conf); + assertOptionEquals(conf, CREDENTIAL_PROVIDER_PATH, + "override"); + } + + @Test + // 同时设置hadoop CredentialProviders和obs CredentialProviders时,全集为两者集合 + public void testSecurityCredentialPropagationOverride() { + conf.set(CREDENTIAL_PROVIDER_PATH, "base"); + conf.set(OBS_SECURITY_CREDENTIAL_PROVIDER_PATH, "override"); + patchSecurityCredentialProviders(conf); + assertOptionEquals(conf, CREDENTIAL_PROVIDER_PATH, + "override,base"); + Collection all = conf.getStringCollection( + CREDENTIAL_PROVIDER_PATH); + assertTrue(all.contains("override")); + assertTrue(all.contains("base")); + } + + @Test + // 可将桶特定CredentialProviders配置传递到OBS通用CredentialProviders配置中 + public void testSecurityCredentialPropagationEndToEnd() { + conf.set(CREDENTIAL_PROVIDER_PATH, "base"); + setBucketOption(conf, "b", OBS_SECURITY_CREDENTIAL_PROVIDER_PATH, + "override"); + Configuration updated = propagateBucketOptions(conf, "b"); + patchSecurityCredentialProviders(updated); + assertOptionEquals(updated, CREDENTIAL_PROVIDER_PATH, + "override,base"); + } + + @Test + // 测试可正确配置鉴权协商开关 + public void testSetAuthTypeNegotiation() + throws Exception, NoSuchFieldException, SecurityException { + Configuration conf = OBSContract.getConfiguration(null); + + Pattern pattern = Pattern.compile( + "^((2[0-4]\\d|25[0-5]|[01]?\\d\\d?)\\.){3}(2[0-4]\\d|25[0-5]|[01]?\\d\\d?)$"); + Matcher m = pattern.matcher(conf.get(OBSConstants.ENDPOINT)); + + // 1、set to true, when endpoint is IP Address, authTypeNegotiationEnabled should be false; + // otherwise should be true + conf.set(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE, "true"); + OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf); + RestStorageService client = obsFs.getObsClient(); + Field propertiesField = RestConnectionService.class.getDeclaredField( + "obsProperties"); + + propertiesField.setAccessible(true); + ObsProperties properties = (ObsProperties) propertiesField.get(client); + String authTypeNegotiationEnabled = properties + .getStringProperty("httpclient.auth-type-negotiation", "false"); + if (m.matches()) { + assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false")); + } else { + assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("true")); + } + obsFs.close(); + + // 2、set to false, authTypeNegotiationEnabled should be false + conf.set(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE, "false"); + obsFs = OBSTestUtils.createTestFileSystem(conf); + client = (RestStorageService) obsFs.getObsClient(); + properties = (ObsProperties) propertiesField.get(client); + authTypeNegotiationEnabled = properties + .getStringProperty("httpclient.auth-type-negotiation", "false"); + assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false")); + obsFs.close(); + + // 3、set to other value, authTypeNegotiationEnabled should be default value false + conf.set(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE, "invalid"); + obsFs = OBSTestUtils.createTestFileSystem(conf); + client = (RestStorageService) obsFs.getObsClient(); + properties = (ObsProperties) propertiesField.get(client); + authTypeNegotiationEnabled = properties + .getStringProperty("httpclient.auth-type-negotiation", "false"); + assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false")); + obsFs.close(); + + // 4、unset, authTypeNegotiationEnabled should be false + conf.unset(OBSConstants.SDK_AUTH_TYPE_NEGOTIATION_ENABLE); + obsFs = OBSTestUtils.createTestFileSystem(conf); + client = (RestStorageService) obsFs.getObsClient(); + properties = (ObsProperties) propertiesField.get(client); + authTypeNegotiationEnabled = properties + .getStringProperty("httpclient.auth-type-negotiation", "false"); + assertTrue(authTypeNegotiationEnabled.equalsIgnoreCase("false")); + obsFs.close(); + } + + // @Test + // // 测试有无CanonicalServiceName配置项时,getCanonicalServiceName能返回正确的值 + // public void testGetCanonicalServiceName() throws Exception { + // // 1、未配置CanonicalServiceName开关时,接口默认返回null + // Configuration conf = OBSContract.getConfiguration(null); + // OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf); + // String canonicalServiceName = obsFs.getCanonicalServiceName(); + // assertEquals("expected null, but get " + canonicalServiceName, + // null, canonicalServiceName); + // obsFs.close(); + // + // // 2、配置CanonicalServiceName开关为开启时,接口返回字符串obs://{bucketName} + // conf.setBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE, true); + // obsFs = OBSTestUtils.createTestFileSystem(conf); + // canonicalServiceName = obsFs.getCanonicalServiceName(); + // String expected = obsFs.getScheme() + "://" + obsFs.getBucket(); + // assertEquals("expected " + expected + " but get " + canonicalServiceName, + // expected, canonicalServiceName); + // obsFs.close(); + // + // // 3、配置CanonicalServiceName开关为关闭时,接口返回null + // conf.setBoolean(OBSConstants.GET_CANONICAL_SERVICE_NAME_ENABLE, false); + // obsFs = OBSTestUtils.createTestFileSystem(conf); + // canonicalServiceName = obsFs.getCanonicalServiceName(); + // assertEquals("expected null, but get " + canonicalServiceName, + // null, canonicalServiceName); + // obsFs.close(); + // } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCopyFromLocalFile.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCopyFromLocalFile.java new file mode 100644 index 0000000..02abc3a --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCopyFromLocalFile.java @@ -0,0 +1,497 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ITestOBSCopyFromLocalFile { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件大小为0,copy后桶内文件大小也为0 + public void testCopyFromLocalFileNormal001() throws Exception { + String localFile = "local_file"; + String destFile = "test_file"; + copyFromLocalFile(localFile, destFile, 0); + assertTrue(fs.getFileStatus(getTestPath(destFile)).getLen() == 0); + OBSFSTestUtil.deleteLocalFile("local_file"); + } + + @Test + // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度小于段大小,为100B,copy后桶内文件大小也为100B + public void testCopyFromLocalFileNormal002() throws Exception { + String localFile = "local_file"; + String destFile = "test_file"; + copyFromLocalFile(localFile, destFile, 100); + assertTrue(fs.getFileStatus(getTestPath(destFile)).getLen() == 100); + OBSFSTestUtil.deleteLocalFile("local_file"); + } + + @Test + // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度小于段大小,为5M,copy后桶内文件大小也为5M + public void testCopyFromLocalFileNormal003() throws Exception { + String localFile = "local_file"; + String destFile = "test_file"; + copyFromLocalFile(localFile, destFile, 5 * 1024 * 1024); + assertEquals(5 * 1024 * 1024, + fs.getFileStatus(getTestPath(destFile)).getLen()); + OBSFSTestUtil.deleteLocalFile("local_file"); + } + + @Test + // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度等于段大小,为100M,copy后桶内文件大小也为100M + public void testCopyFromLocalFileNormal004() throws Exception { + String localFile = "local_file"; + String destFile = "test_file"; + copyFromLocalFile(localFile, destFile, 100 * 1024 * 1024); + assertEquals(100 * 1024 * 1024, + fs.getFileStatus(getTestPath(destFile)).getLen()); + OBSFSTestUtil.deleteLocalFile("local_file"); + } + + @Test + // src是文件,dst是文件,delSrc为true,overwrite为true, dst已存在;src文件长度大于段大小,为120M,copy后桶内文件大小也为120M + public void testCopyFromLocalFileNormal005() throws Exception { + String localFile = "local_file"; + String destFile = "test_file"; + copyFromLocalFile(localFile, destFile, 120 * 1024 * 1024); + assertEquals(120 * 1024 * 1024, + fs.getFileStatus(getTestPath(destFile)).getLen()); + OBSFSTestUtil.deleteLocalFile("local_file"); + } + + @Test + // src是文件,dst是文件,delSrc为true,overwrite为false, dst不存在 + public void testCopyFromLocalFileNormal006() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile); + + Path dstPath = getTestPath("test_file"); + OBSFSTestUtil.deletePathRecursive(fs, dstPath); + + fs.copyFromLocalFile(true, false, new Path(localFile), dstPath); + OBSFSTestUtil.assertPathExistence(fs, dstPath, true); + OBSFSTestUtil.deleteLocalFile(localFile); + } + + @Test + // src是文件,dst是文件,delSrc为true,overwrite为true, dst不存在 + public void testCopyFromLocalFileNormal007() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile); + + Path dstPath = getTestPath("test_file"); + OBSFSTestUtil.deletePathRecursive(fs, dstPath); + + fs.copyFromLocalFile(false, true, new Path(localFile), dstPath); + OBSFSTestUtil.assertPathExistence(fs, dstPath, true); + OBSFSTestUtil.deleteLocalFile(localFile); + } + + @Test + // src是文件,dst是根目录,文件拷贝到根目录下 + public void testCopyFromLocalFileNormal008() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile); + + Path dstPath = new Path("/"); + fs.copyFromLocalFile(false, true, new Path(localFile), dstPath); + OBSFSTestUtil.assertPathExistence(fs, new Path("/" + localFile), true); + + OBSFSTestUtil.deletePathRecursive(fs, new Path("/" + localFile)); + OBSFSTestUtil.deleteLocalFile(localFile); + } + + @Test + // src是文件,dst是多级目录,文件拷贝到多级目录下 + public void testCopyFromLocalFileNormal009() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile); + + Path dstPath = getTestPath("a001/b001"); + fs.mkdirs(dstPath); + fs.copyFromLocalFile(false, true, new Path(localFile), dstPath); + OBSFSTestUtil.assertPathExistence(fs, + getTestPath("a001/b001/" + localFile), true); + + OBSFSTestUtil.deleteLocalFile(localFile); + } + + @Test + // src是空目录,dst是目录,dst不存在,创建dst,将源目录child列表拷贝到dst下 + public void testCopyFromLocalFileNormal010() throws Exception { + String localDir = "local_dir"; + String localFile = "local_dir/local_file"; + OBSFSTestUtil.createLocalTestDir(localDir); + new File(testRootPath, "." + localFile + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile); + + Path dstPath = getTestPath("a001/b001"); + fs.mkdirs(dstPath.getParent()); + fs.copyFromLocalFile(false, true, new Path(localDir), dstPath); + assertTrue(fs.listStatus(dstPath).length == 1); + OBSFSTestUtil.deleteLocalFile(localDir); + fs.delete(dstPath.getParent(), true); + } + + @Test + // src是非空目录,dst是目录,dst不存在,创建dst,将源目录child列表拷贝到dst下 + public void testCopyFromLocalFileNormal011() throws Exception { + String localDir = "local_dir"; + OBSFSTestUtil.createLocalTestDir(localDir); + + Path dstPath = getTestPath("a001/b001"); + fs.mkdirs(dstPath.getParent()); + fs.copyFromLocalFile(false, true, new Path(localDir), dstPath); + assertTrue(fs.listStatus(dstPath).length == 0); + OBSFSTestUtil.deleteLocalFile(localDir); + } + + @Test + // src是目录,dst是根目录,根目录下不存在与src同名的子目录,正常 + public void testCopyFromLocalFileNormal012() throws Exception { + String localDir = "local_dir"; + OBSFSTestUtil.createLocalTestDir(localDir); + + Path dstPath = new Path("/"); + fs.mkdirs(dstPath); + boolean hasException = false; + try { + fs.copyFromLocalFile(false, true, new Path(localDir), dstPath); + } catch (IOException e) { + hasException = true; + } + assertFalse(hasException); + + OBSFSTestUtil.deleteLocalFile(localDir); + fs.delete(new Path("/local_dir"), true); + } + + @Test + // src是目录,dst是多级空目录,dst已存在,正常 + public void testCopyFromLocalFileNormal013() throws Exception { + String localDir = "local_dir"; + OBSFSTestUtil.createLocalTestDir(localDir); + + Path dstPath = getTestPath("a001/b001"); + fs.mkdirs(dstPath); + boolean hasException = false; + try { + fs.copyFromLocalFile(false, true, new Path(localDir), dstPath); + } catch (IOException e) { + hasException = true; + } + assertFalse(hasException); + OBSFSTestUtil.deleteLocalFile(localDir); + } + + @Test + // src是文件,dst是文件,overwrite为false,dst已存在,抛FileAlreadyExistsException + public void testCopyFromLocalFileAbnormal001() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile); + + Path dstPath = getTestPath("test_file"); + OBSFSTestUtil.deletePathRecursive(fs, dstPath); + FSDataOutputStream outputStream = fs.create(dstPath, false); + outputStream.close(); + OBSFSTestUtil.assertPathExistence(fs, dstPath, true); + + boolean hasException = false; + try { + fs.copyFromLocalFile(false, false, new Path(localFile), dstPath); + } catch (IOException e) { + hasException = true; + } + assertTrue(hasException); + + fs.delete(dstPath, true); + OBSFSTestUtil.deleteLocalFile(localFile); + } + + @Test + // src不存在,抛FileNotFoundException + public void testCopyFromLocalFileAbnormal002() throws Exception { + String localFile = "local_file"; + OBSFSTestUtil.deleteLocalFile(localFile); + + Path dstPath = getTestPath("test_file"); + + boolean hasException = false; + try { + fs.copyFromLocalFile(false, true, new Path(localFile), dstPath); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + + fs.delete(dstPath, true); + OBSFSTestUtil.deleteLocalFile(localFile); + } + + @Test + // 测试基类copyFromLocalFile(boolean delSrc, boolean overwrite, Path[] srcs, Path dst)方法 + public void testCplParentMethod01() throws Exception { + String localFile1 = "local_file1"; + String localFile2 = "local_file2"; + new File(testRootPath, "." + localFile1 + ".crc").delete(); + new File(testRootPath, "." + localFile2 + ".crc").delete(); + OBSFSTestUtil.createLocalTestFile(localFile1); + OBSFSTestUtil.createLocalTestFile(localFile2); + Path[] srcFiles = {new Path(localFile1), new Path(localFile2)}; + + Path dstDir = getTestPath("test_dir"); + fs.mkdirs(dstDir); + fs.copyFromLocalFile(true, false, srcFiles, dstDir); + OBSFSTestUtil.assertPathExistence(fs, + getTestPath("test_dir/local_file1"), true); + OBSFSTestUtil.assertPathExistence(fs, + getTestPath("test_dir/local_file2"), true); + + fs.delete(getTestPath("test_dir"), true); + OBSFSTestUtil.deleteLocalFile(localFile1); + OBSFSTestUtil.deleteLocalFile(localFile2); + } + + @Test + // 测试基类copyFromLocalFile(boolean delSrc, Path src, Path dst)方法,overwrite始终为true + public void testCplParentMethod02() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + File file = OBSFSTestUtil.createLocalTestFile(localFile); + OBSFSTestUtil.writeLocalFile(file, 16); + + String dstFile = "test_file"; + FSDataOutputStream outputStream = fs.create(getTestPath(dstFile), + false); + outputStream.close(); + + fs.copyFromLocalFile(true, new Path(localFile), getTestPath(dstFile)); + assertEquals(16, fs.getFileStatus(getTestPath(dstFile)).getLen()); + assertFalse(file.exists()); + + fs.delete(getTestPath(dstFile), true); + } + + @Test + // 测试基类copyFromLocalFile(Path src, Path dst)方法, overwrite为true, desSrc为false + public void testCplParentMethod03() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + File file = OBSFSTestUtil.createLocalTestFile(localFile); + OBSFSTestUtil.writeLocalFile(file, 16); + + String dstFile = "test_file"; + FSDataOutputStream outputStream = fs.create(getTestPath(dstFile), + false); + outputStream.close(); + + fs.copyFromLocalFile(new Path(localFile), getTestPath(dstFile)); + assertEquals(16, fs.getFileStatus(getTestPath(dstFile)).getLen()); + assertTrue(file.exists()); + + fs.delete(getTestPath(dstFile), true); + OBSFSTestUtil.deleteLocalFile(localFile); + } + + @Test + // 测试基类moveFromLocalFile(Path[] srcs, Path dst)方法,执行copyFromLocalFile(true, true, srcs, dst) + public void testCplParentMethod04() throws Exception { + String localFile1 = "local_file1"; + String localFile2 = "local_file2"; + new File(testRootPath, "." + localFile1 + ".crc").delete(); + new File(testRootPath, "." + localFile2 + ".crc").delete(); + File file1 = OBSFSTestUtil.createLocalTestFile(localFile1); + File file2 = OBSFSTestUtil.createLocalTestFile(localFile2); + OBSFSTestUtil.writeLocalFile(file1, 16); + OBSFSTestUtil.writeLocalFile(file2, 16); + Path[] srcFiles = {new Path(localFile1), new Path(localFile2)}; + + String dstDir = "dest_dir"; + fs.mkdirs(getTestPath(dstDir)); + + fs.moveFromLocalFile(srcFiles, getTestPath(dstDir)); + OBSFSTestUtil.assertPathExistence(fs, + getTestPath("dest_dir/local_file1"), true); + OBSFSTestUtil.assertPathExistence(fs, + getTestPath("dest_dir/local_file2"), true); + assertFalse(file1.exists()); + assertFalse(file2.exists()); + + fs.delete(getTestPath(dstDir), true); + } + + @Test + // 测试基类moveFromLocalFile(Path src, Path dst)方法,执行copyFromLocalFile(delSrc = true, src, dst) + public void testCplParentMethod05() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + File file = OBSFSTestUtil.createLocalTestFile(localFile); + OBSFSTestUtil.writeLocalFile(file, 16); + + String dstFile = "test_file"; + FSDataOutputStream outputStream = fs.create(getTestPath(dstFile), + false); + outputStream.close(); + + fs.moveFromLocalFile(new Path(localFile), getTestPath(dstFile)); + assertEquals(16, fs.getFileStatus(getTestPath(dstFile)).getLen()); + assertFalse(file.exists()); + + fs.delete(getTestPath(dstFile), true); + } + + @Test + // 测试基类copyToLocalFile(boolean delSrc, Path src, Path dst, boolean useRawLocalFileSystem)方法 + // overwrite始终为true + public void testCplParentMethod06() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + File file = OBSFSTestUtil.createLocalTestFile(localFile); + OBSFSTestUtil.writeLocalFile(file, 16); + + String srcFile = "test_file"; + FSDataOutputStream outputStream = fs.create(getTestPath(srcFile), + false); + outputStream.close(); + + fs.copyToLocalFile(true, getTestPath(srcFile), new Path(localFile), + true); + assertEquals(0, file.length()); + OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), false); + + OBSFSTestUtil.deleteLocalFile(localFile); + fs.delete(getTestPath(srcFile), true); + } + + @Test + // 测试基类copyToLocalFile(boolean delSrc, Path src, Path dst)方法,useRawLocalFileSystem为false + public void testCplParentMethod07() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + File file = OBSFSTestUtil.createLocalTestFile(localFile); + OBSFSTestUtil.writeLocalFile(file, 16); + + String srcFile = "test_file"; + FSDataOutputStream outputStream = fs.create(getTestPath(srcFile), + false); + outputStream.close(); + + fs.copyToLocalFile(true, getTestPath(srcFile), new Path(localFile)); + assertEquals(0, file.length()); + OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), false); + + OBSFSTestUtil.deleteLocalFile(localFile); + fs.delete(getTestPath(srcFile), true); + } + + @Test + // 测试基类copyToLocalFile(Path src, Path dst)方法, delSrc为false,useRawLocalFileSystem为false + public void testCplParentMethod08() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + File file = OBSFSTestUtil.createLocalTestFile(localFile); + OBSFSTestUtil.writeLocalFile(file, 16); + + String srcFile = "test_file"; + FSDataOutputStream outputStream = fs.create(getTestPath(srcFile), + false); + outputStream.close(); + + fs.copyToLocalFile(getTestPath(srcFile), new Path(localFile)); + assertEquals(0, file.length()); + OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), true); + + OBSFSTestUtil.deleteLocalFile(localFile); + fs.delete(getTestPath(srcFile), true); + } + + @Test + // 测试基类moveToLocalFile(Path src, Path dst)方法, delSrc为true, useRawLocalFileSystem为false + public void testCplParentMethod09() throws Exception { + String localFile = "local_file"; + new File(testRootPath, "." + localFile + ".crc").delete(); + File file = OBSFSTestUtil.createLocalTestFile(localFile); + OBSFSTestUtil.writeLocalFile(file, 16); + + String srcFile = "test_file"; + FSDataOutputStream outputStream = fs.create(getTestPath(srcFile), + false); + outputStream.close(); + + fs.moveToLocalFile(getTestPath(srcFile), new Path(localFile)); + assertEquals(0, file.length()); + OBSFSTestUtil.assertPathExistence(fs, getTestPath(srcFile), false); + + OBSFSTestUtil.deleteLocalFile(localFile); + fs.delete(getTestPath(srcFile), true); + } + + private void copyFromLocalFile(String srcFile, String destFile, + long srcFileSize) throws IOException { + Path localPath = new Path(srcFile); + new File(testRootPath, "." + srcFile + ".crc").delete(); + File localFile = OBSFSTestUtil.createLocalTestFile(srcFile); + OBSFSTestUtil.writeLocalFile(localFile, srcFileSize); + + Path dstPath = getTestPath(destFile); + fs.delete(dstPath, true); + FSDataOutputStream outputStream = OBSFSTestUtil.createStream(fs, + dstPath); + outputStream.close(); + + fs.copyFromLocalFile(true, true, localPath, dstPath); + OBSFSTestUtil.assertPathExistence(fs, dstPath, true); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCreate.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCreate.java new file mode 100644 index 0000000..0e71dd0 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCreate.java @@ -0,0 +1,987 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.util.EnumSet; + +public class ITestOBSCreate { + private OBSFileSystem fs; + + private static long writeBufferSize = 100 * 1024 * 1024; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + writeBufferSize = OBSCommonUtils + .getMultipartSizeProperty(conf, OBSConstants.MULTIPART_SIZE, + OBSConstants.DEFAULT_MULTIPART_SIZE); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath)); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 创建一个stream,overwrite为false + public void testCreateNormal001() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("test_file"); + + FSDataOutputStream outputStream = null; + // create with CreateFlag argument + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + + // create with overwrite argument + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + //normal create test case + String position = "0"; + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 创建一个stream,路径为一个存在的文件,overwrite为true + public void testCreateNormal002() throws Exception { + Path testFile = getTestPath("test_file"); + + FSDataOutputStream outputStream = null; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + null, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + + // overwrite when exist with overwrite argument + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + true, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + + // overwrite when exist with createflag argument + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // createNonRecursive stream,父目录已存在 + public void testCreateNormal003() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + fs.mkdirs(testFile.getParent()); + + byte[] testBuffer = new byte[128]; + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % 128); + } + FSDataOutputStream outputStream = null; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + outputStream.write(testBuffer); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length); + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + outputStream.write(testBuffer); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length); + OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent()); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 创建write stream,调用write(),写入100字节 + public void testCreateNormal004() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + + byte[] testBuffer = new byte[100]; + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % 128); + } + FSDataOutputStream outputStream = null; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + outputStream.write(testBuffer); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length); + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + outputStream.write(testBuffer); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 创建write stream,调用write(byte[] source, int offset, int len),写入100MB + public void testCreateNormal005() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + FSDataOutputStream outputStream = null; + + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 100 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, 100 * 1024 * 1024); + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.writeData(outputStream, 100 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, 100 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 创建write stream,调用write(byte[] source, int offset, int len),写入110MB + public void testCreateNormal006() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + FSDataOutputStream outputStream = null; + + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, 110 * 1024 * 1024); + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, 110 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 创建write stream,调用write(byte[] source, int offset, int len),写入110MB,调用hflush刷数据上云,数据正常。 + // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB + public void testCreateNormal007() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + FSDataOutputStream outputStream = null; + + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 10 * 1024 * 1024); + outputStream.hflush(); + if (fs.isFsBucket()) { + OBSFSTestUtil.verifyReceivedData(fs, testFile, + 10 * 1024 * 1024); + } + + OBSFSTestUtil.writeData(outputStream, 100); + OBSFSTestUtil.writeData(outputStream, 15 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, + 10 * 1024 * 1024 + 100 + 15 * 1024 * 1024); + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.writeData(outputStream, 10 * 1024 * 1024); + outputStream.hflush(); + if (fs.isFsBucket()) { + OBSFSTestUtil.verifyReceivedData(fs, testFile, + 10 * 1024 * 1024); + } + + System.out.println(fs.getFileStatus(testFile).getLen()); + OBSFSTestUtil.writeData(outputStream, 100); + System.out.println(fs.getFileStatus(testFile).getLen()); + OBSFSTestUtil.writeData(outputStream, 15 * 1024 * 1024); + System.out.println(fs.getFileStatus(testFile).getLen()); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, + 10 * 1024 * 1024 + 100 + 15 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // create的文件不存在,overwrite参数为true,文件创建成功 + public void testCreateNormal009() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + + // create with overwrite argument + boolean hasException = false; + FSDataOutputStream outputStream = null; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + true, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + // assertEquals("create non exist file with overwrite flag should throw exception.", + // true, hasException); + + // create with create flag + hasException = false; + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + // assertEquals("create non exist file with overwrite flag should throw exception.", + // true, hasException); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 创建write stream,调用write(byte[] source, int offset, int len),写入110MB,调用hsync刷数据上云,数据正常。 + // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB + public void testCreateNormal008() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + FSDataOutputStream outputStream = null; + + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024); + outputStream.hsync(); + if (fs.isFsBucket()) { + OBSFSTestUtil.verifyReceivedData(fs, testFile, + 110 * 1024 * 1024); + } + + OBSFSTestUtil.writeData(outputStream, 100); + OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, + 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024); + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024); + outputStream.hsync(); + if (fs.isFsBucket()) { + OBSFSTestUtil.verifyReceivedData(fs, testFile, + 110 * 1024 * 1024); + } + + OBSFSTestUtil.writeData(outputStream, 100); + OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.assertFileHasLength(fs, testFile, + 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 测试fs.obs.file.visibility.enable开关有效性 + public void testCreateNormal010() throws Exception { + // 1、默认不开启fs.obs.file.visibility.enable开关 + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + Path testFile = getTestPath("test_file"); + fs.delete(testFile, true); + + FSDataOutputStream outputStream = fs.create(testFile); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + outputStream.close(); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + fs.delete(testFile, true); + fs.close(); + + // 2、打开fs.obs.file.visibility.enable开关后,create接口创建空文件 + conf.setBoolean("fs.obs.file.visibility.enable", true); + fs = OBSTestUtils.createTestFileSystem(conf); + outputStream = fs.create(testFile); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + outputStream.close(); + fs.delete(testFile, true); + fs.close(); + + // 3、关闭fs.obs.file.visibility.enable开关后,create接口不会创建空文件 + conf.setBoolean("fs.obs.file.visibility.enable", false); + fs = OBSTestUtils.createTestFileSystem(conf); + outputStream = fs.create(testFile); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + outputStream.close(); + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + fs.delete(testFile, true); + } + + @Test + // 文件桶创建一个append stream,路径为一个文件 + public void testCreatePosixNormal001() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + + FSDataOutputStream outputStream = null; + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertPathExistence(fs, testFile, true); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 文件桶createNonRecursive append stream,写入128字节 + public void testCreatePosixNormal002() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + fs.mkdirs(testFile.getParent()); + + byte[] testBuffer = new byte[128]; + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % 128); + } + FSDataOutputStream outputStream = null; + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.createNonRecursive(testFile, + new FsPermission((short) 00644), flags, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + outputStream.write(testBuffer); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length); + OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent()); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 文件桶create append stream, 写入100字节 + public void testCreatePosixNormal003() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + + byte[] testBuffer = new byte[128]; + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % 128); + } + FSDataOutputStream outputStream = null; + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + outputStream.write(testBuffer); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertFileHasLength(fs, testFile, testBuffer.length); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 文件桶create append stream, 写入100M + public void testCreatePosixNormal004() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + FSDataOutputStream outputStream = null; + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 100 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertFileHasLength(fs, testFile, 100 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 文件桶create append stream, 写入110M + public void testCreatePosixNormal005() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + FSDataOutputStream outputStream = null; + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertFileHasLength(fs, testFile, 110 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 文件桶create append stream,调用write(byte[] source, int offset, int len),写入110MB,调用hflush刷数据上云,数据正常。 + // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB + public void testCreatePosixNormal006() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + FSDataOutputStream outputStream = null; + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024); + outputStream.hflush(); + OBSFSTestUtil.verifyReceivedData(fs, testFile, 110 * 1024 * 1024); + + OBSFSTestUtil.writeData(outputStream, 100); + OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertFileHasLength(fs, testFile, + 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 文件桶create append stream,调用write(byte[] source, int offset, int len),写入110MB,调用hsync刷数据上云,数据正常。 + // 然后调用write()写100字节,再调用write(byte[] source, int offset, int len)写150MB + public void testCreatePosixNormal007() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + FSDataOutputStream outputStream = null; + + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + OBSFSTestUtil.writeData(outputStream, 110 * 1024 * 1024); + outputStream.hsync(); + OBSFSTestUtil.verifyReceivedData(fs, testFile, 110 * 1024 * 1024); + + OBSFSTestUtil.writeData(outputStream, 100); + OBSFSTestUtil.writeData(outputStream, 150 * 1024 * 1024); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.assertFileHasLength(fs, testFile, + 110 * 1024 * 1024 + 100 + 150 * 1024 * 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // create的文件已存在,overwrite参数为false,抛出FileAlreadyExistsException + public void testCreateAbnormal001() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + + FSDataOutputStream outputStream = null; + try { + outputStream = OBSFSTestUtil.createStream(fs, testFile, false); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + // create with overwrite argument + boolean hasException = false; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + } catch (FileAlreadyExistsException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create exist file with no overwrite flag should throw exception.", + true, hasException); + + // create with createflag argument + hasException = false; + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + } catch (FileAlreadyExistsException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create exist file with no overwrite flag should throw exception.", + true, hasException); + + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // create的路径是目录,抛出FileAlreadyExistsException + public void testCreateAbnormal002() throws Exception { + Path testDir = getTestPath("a001/b001/test_file"); + fs.mkdirs(testDir); + + FSDataOutputStream outputStream = null; + boolean hasException = false; + try { + outputStream = fs.create(testDir, new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + } catch (FileAlreadyExistsException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + hasException = false; + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.OVERWRITE); + outputStream = fs.create(testDir, new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + } catch (FileAlreadyExistsException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals("create stream for folder should throw exception.", + true, hasException); + OBSFSTestUtil.deletePathRecursive(fs, testDir); + } + + @Test + // createNonRecursive 路径的父目录是文件,抛出ParentNotDirectoryException + public void testCreateAbnormal003() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + Path parentPath = testFile.getParent(); + + FSDataOutputStream outputStream = null; + boolean hasException = false; + try { + outputStream = OBSFSTestUtil.createStream(fs, parentPath); + outputStream.close(); + + outputStream = OBSFSTestUtil.createNonRecursiveStream(fs, testFile); + } catch (ParentNotDirectoryException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create non recursive stream for path whose parent path " + + "with same file should throw exception.", + true, + hasException); + OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent()); + } + + @Test + // createNonRecursive 路径的父目录不存在,抛出FileNotFoundException + public void testCreateAbnormal004() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + + FSDataOutputStream outputStream = null; + boolean hasException = false; + try { + outputStream = fs.createNonRecursive(testFile, + new FsPermission((short) 00644), + EnumSet.noneOf(CreateFlag.class), + 4096, (short) 3, 128 * 1024 * 1024, null); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create non recursive stream for non exist path should throw exception.", + true, + hasException); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // create 携带LAZY_PERSIST、NEW_BLOCK、NO_LOCAL_WRITE、SHOULD_REPLICATE + // 、IGNORE_CLIENT_LOCALITY, + // 抛出UnsupportedOperationException + // hadoop2.8.3 无SHOULD_REPLICATE和IGNORE_CLIENT_LOCALITY flag + public void testCreateAbnormal005() throws Exception { + Path testFile = getTestPath("a001/b001/test_file"); + fs.mkdirs(testFile.getParent()); + + FSDataOutputStream outputStream = null; + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + boolean hasException; + + // LAZY_PERSIST + flags.add(CreateFlag.LAZY_PERSIST); + hasException = false; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, + (short) 3, 128 * 1024 * 1024, null, null); + } catch (UnsupportedOperationException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create stream with LAZY_PERSIST flag throw UnsupportedOperationException.", + true, + hasException); + + hasException = false; + try { + outputStream = fs.createNonRecursive(testFile, + new FsPermission((short) 00644), flags, + 4096, (short) 3, 128 * 1024 * 1024, null); + } catch (UnsupportedOperationException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create stream with LAZY_PERSIST flag throw UnsupportedOperationException.", + true, + hasException); + flags.clear(); + + // NEW_BLOCK + flags.add(CreateFlag.NEW_BLOCK); + hasException = false; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, + (short) 3, 128 * 1024 * 1024, null, null); + } catch (UnsupportedOperationException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create stream with NEW_BLOCK flag throw UnsupportedOperationException.", + true, + hasException); + + hasException = false; + try { + outputStream = fs.createNonRecursive(testFile, + new FsPermission((short) 00644), flags, + 4096, (short) 3, 128 * 1024 * 1024, null); + } catch (UnsupportedOperationException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create stream with NEW_BLOCK flag throw UnsupportedOperationException.", + true, + hasException); + flags.clear(); + + // NO_LOCAL_WRITE + flags.add(CreateFlag.NO_LOCAL_WRITE); + hasException = false; + try { + outputStream = fs.create(testFile, new FsPermission((short) 00644), + flags, 4096, + (short) 3, 128 * 1024 * 1024, null, null); + } catch (UnsupportedOperationException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create stream with NO_LOCAL_WRITE flag throw UnsupportedOperationException.", + true, + hasException); + + hasException = false; + try { + outputStream = fs.createNonRecursive(testFile, + new FsPermission((short) 00644), flags, + 4096, (short) 3, 128 * 1024 * 1024, null); + } catch (UnsupportedOperationException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertEquals( + "create stream with NO_LOCAL_WRITE flag throw UnsupportedOperationException.", + true, + hasException); + flags.clear(); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCredentialsInURL.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCredentialsInURL.java new file mode 100644 index 0000000..dcdffda --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSCredentialsInURL.java @@ -0,0 +1,214 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.io.IOUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.UnsupportedEncodingException; +import java.net.URI; +import java.net.URLEncoder; + +import static org.apache.hadoop.fs.obs.OBSTestConstants.TEST_FS_OBS_NAME; + +/** + * Tests that credentials can go into the URL. This includes a valid + * set, and a check that an invalid set do at least get stripped out + * of the final URI + */ +public class ITestOBSCredentialsInURL extends Assert { + private OBSFileSystem fs; + private Configuration conf; + private static final Logger LOG = + LoggerFactory.getLogger(ITestOBSCredentialsInURL.class); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout(30 * 60 * 1000); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void teardown() { + IOUtils.closeStream(fs); + } + + /** + * Test instantiation. + * + * @throws Throwable + */ + @Test + // uri中携带credential信息,编码后ak、sk不包含/和+符号 + public void testInstantiateFromURL() throws Throwable { + + String accessKey = conf.get(OBSConstants.ACCESS_KEY); + String secretKey = conf.get(OBSConstants.SECRET_KEY); + String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, ""); + Assume.assumeNotNull(fsname, accessKey, secretKey); + URI original = new URI(fsname); + URI secretsURI = createUriWithEmbeddedSecrets(original, + accessKey, secretKey); + if (secretKey.contains("/")) { + assertTrue("test URI encodes the / symbol", secretsURI.toString(). + contains("%252F")); + } + if (secretKey.contains("+")) { + assertTrue("test URI encodes the + symbol", secretsURI.toString(). + contains("%252B")); + } + assertFalse("Does not contain secrets", original.equals(secretsURI)); + + conf.set(TEST_FS_OBS_NAME, secretsURI.toString()); + conf.unset(OBSConstants.ACCESS_KEY); + conf.unset(OBSConstants.SECRET_KEY); + String fsURI = fs.getUri().toString(); + assertFalse("FS URI contains a @ symbol", fsURI.contains("@")); + assertFalse("FS URI contains a % symbol", fsURI.contains("%")); + if (!original.toString().startsWith(fsURI)) { + fail("Filesystem URI does not match original"); + } + validate("original path", new Path(original)); + validate("bare path", new Path("/")); + validate("secrets path", new Path(secretsURI)); + if (fs != null) { + fs.close(); + fs = null; + } + } + + @Test + // 测试标准化uri功能正常 + public void testCanonicalizeUri() throws Exception { + assertNotNull("OBSFileSystem is null.", fs); + URI testURI = new URI(OBSTestConstants.TEST_FS_OBS_NAME); + URI canonicalizedURI = fs.canonicalizeUri(testURI); + assertEquals(testURI, canonicalizedURI); + if (fs != null) { + fs.close(); + fs = null; + } + } + + @Test + // 校验Path/URL符合OBSFS的规范 + public void testCheckPath() throws IOException { + assertNotNull("OBSFileSystem is null.", fs); + + // 1. no schema + Path testPath = new Path("/test/abc"); + fs.checkPath(testPath); + + // 2. schema not match + testPath = new Path("sss://testfs/abc"); + boolean hasException = false; + try { + fs.checkPath(testPath); + } catch (IllegalArgumentException e) { + hasException = true; + } + assertTrue(hasException); + + // 3.1. host is null, fs be same with defaultFs + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, + OBSTestConstants.TEST_FS_OBS_NAME); + testPath = new Path("obs:///test"); + hasException = false; + try { + fs.checkPath(testPath); + } catch (IllegalArgumentException e) { + hasException = true; + } + assertTrue(hasException); + + // 3.2. host is null, fs not same with defaultFs + conf.set(CommonConfigurationKeysPublic.FS_DEFAULT_NAME_KEY, + "obs://NOT-SAME-bucket-4521/"); + testPath = new Path("obs:///test"); + hasException = false; + try { + fs.checkPath(testPath); + } catch (IllegalArgumentException e) { + hasException = true; + } + assertTrue(hasException); + if (fs != null) { + fs.close(); + fs = null; + } + } + + private void validate(String text, Path path) throws IOException { + try { + fs.canonicalizeUri(path.toUri()); + fs.checkPath(path); + assertTrue(text + " Not a directory", + fs.getFileStatus(new Path("/")).isDirectory()); + fs.globStatus(path); + } catch (Exception e) { + LOG.debug("{} failure: {}", text, e, e); + fail(text + " Test failed"); + } + } + + private URI createUriWithEmbeddedSecrets(URI original, + String accessKey, + String secretKey) throws UnsupportedEncodingException { + String encodedSecretKey = URLEncoder.encode(secretKey, "UTF-8"); + String formattedString = String.format("%s://%s:%s@%s/%s/", + original.getScheme(), + accessKey, + encodedSecretKey, + original.getHost(), + original.getPath()); + URI testURI; + try { + testURI = new Path(formattedString).toUri(); + } catch (IllegalArgumentException e) { + // inner cause is stripped to keep any secrets out of stack traces + throw new IllegalArgumentException( + "Could not encode Path: " + formattedString); + } + return testURI; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDataBlocks.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDataBlocks.java new file mode 100644 index 0000000..9c7286c --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDataBlocks.java @@ -0,0 +1,203 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; + +import java.io.IOException; +import java.util.Arrays; +import java.util.List; + +/** + * Unit tests for {@link OBSDataBlocks}. + */ +public class ITestOBSDataBlocks extends Assert { + private OBSFileSystem fs; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout(30 * 1000); + + private List factoryPolicies = Arrays.asList(OBSConstants.FAST_UPLOAD_BUFFER_DISK, + OBSConstants.FAST_UPLOAD_BUFFER_ARRAY, OBSConstants.FAST_UPLOAD_BYTEBUFFER); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void nameThread() throws IOException { + Thread.currentThread().setName("JUnit"); + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + /** + * Test the {@link OBSDataBlocks.ByteBufferBlockFactory}. That code + * implements an input stream over a ByteBuffer, and has to return the + * buffer to the pool after the read complete. + *

+ * This test verifies the basic contract of the process. + */ + @Test + // 测试基于byte buffer写缓存的读写基本功能 + public void testByteBufferIO() throws Throwable { + OBSDataBlocks.ByteBufferBlockFactory factory = new OBSDataBlocks.ByteBufferBlockFactory(fs); + int limit = 128; + OBSDataBlocks.ByteBufferBlock block = factory.create(1, limit); + assertOutstandingBufferCount(factory, 1); + + byte[] buffer = ContractTestUtils.toAsciiByteArray("test data"); + int bufferLen = buffer.length; + block.write(buffer, 0, bufferLen); + assertEquals(bufferLen, block.dataSize()); + assertEquals(limit - bufferLen, block.remainingCapacity()); + assertTrue(block.hasCapacity(64)); + assertTrue(block.hasCapacity(limit - bufferLen)); + + // now start the write + OBSDataBlocks.ByteBufferBlock.ByteBufferInputStream stream = + (OBSDataBlocks.ByteBufferBlock.ByteBufferInputStream) block.startUpload(); + assertTrue(stream.markSupported()); + assertTrue(stream.hasRemaining()); + int expected = bufferLen; + assertEquals(expected, stream.available()); + + assertEquals('t', stream.read()); + stream.mark(limit); + expected--; + assertEquals(expected, stream.available()); + + // read into a byte array with an offset + int offset = 5; + byte[] data = new byte[limit]; + int read = stream.read(data, offset, 2); + assertEquals(2, read); + assertEquals('e', data[offset]); + assertEquals('s', data[offset + 1]); + expected -= 2; + assertEquals(expected, stream.available()); + + // read to end + byte[] remains = new byte[limit]; + int value; + int index = 0; + while ((value = stream.read()) >= 0) { + remains[index++] = (byte) value; + } + assertEquals(expected, index); + assertEquals('a', remains[--index]); + + assertEquals(0, stream.available()); + assertTrue(!stream.hasRemaining()); + + // go the mark point + stream.reset(); + assertEquals('e', stream.read()); + + // when the stream is closed, the data should be returned + stream.close(); + assertOutstandingBufferCount(factory, 1); + block.close(); + assertOutstandingBufferCount(factory, 0); + stream.close(); + assertOutstandingBufferCount(factory, 0); + } + + @Test + public void testNotCalcChecksum() throws IOException { + for (String factory : factoryPolicies) { + Configuration conf = OBSContract.getConfiguration(null); + OBSDataBlocks.DataBlock dataBlock = + startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data")); + assertNull("factory is " + factory, dataBlock.getChecksum()); + assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.NONE, dataBlock.getChecksumType()); + dataBlock.close(); + } + } + + @Test + public void testCalcMD5() throws IOException { + for (String factory : factoryPolicies) { + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, true); + OBSDataBlocks.DataBlock dataBlock = + startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data")); + assertEquals("factory is " + factory, "63M6AMDJ0zbmVpGjerVCkw==", dataBlock.getChecksum()); + assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.MD5, dataBlock.getChecksumType()); + dataBlock.close(); + } + } + + @Test + public void testCalcMD5UseNewConfig() throws IOException { + for (String factory : factoryPolicies) { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_MD5); + OBSDataBlocks.DataBlock dataBlock = + startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data")); + assertEquals("factory is " + factory, "63M6AMDJ0zbmVpGjerVCkw==", dataBlock.getChecksum()); + assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.MD5, dataBlock.getChecksumType()); + dataBlock.close(); + } + } + + @Test + public void testCalcSha256() throws IOException { + for (String factory : factoryPolicies) { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE_SHA256); + OBSDataBlocks.DataBlock dataBlock = + startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data")); + assertEquals("factory is " + factory, "916f0027a575074ce72a331777c3478d6513f786a591bd892da1a577bf2335f9", + dataBlock.getChecksum()); + assertEquals("factory is " + factory, OBSDataBlocks.ChecksumType.SHA256, dataBlock.getChecksumType()); + dataBlock.close(); + } + } + + @Test + public void testWrongChecksumType() { + for (String factory : factoryPolicies) { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.FAST_UPLOAD_CHECKSUM_TYPE, "wrongType"); + try { + startUploadDataBlock(factory, conf, ContractTestUtils.toAsciiByteArray("test data")); + fail("should be throw exception when type is wrong. factoryPolicy: " + factory); + } catch (IOException e) { + assertTrue(e.getMessage(), e.getMessage().startsWith("Unsupported fast upload checksum type")); + } + } + } + + private OBSDataBlocks.DataBlock startUploadDataBlock(String factoryName, Configuration conf, byte[] data) + throws IOException { + OBSFileSystem fileSystem = OBSTestUtils.createTestFileSystem(conf); + OBSDataBlocks.BlockFactory factory = OBSDataBlocks.createFactory(fileSystem, factoryName); + OBSDataBlocks.DataBlock dataBlock = factory.create(1, 128); + dataBlock.write(data, 0, data.length); + dataBlock.startUpload(); + return dataBlock; + } + + /** + * Assert the number of buffers active for a block factory. + * + * @param factory factory + * @param expectedCount expected count. + */ + private static void assertOutstandingBufferCount(OBSDataBlocks.ByteBufferBlockFactory factory, int expectedCount) { + assertEquals("outstanding buffers in " + factory, expectedCount, factory.getOutstandingBufferCount()); + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDefaultInformation.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDefaultInformation.java new file mode 100644 index 0000000..9baca07 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDefaultInformation.java @@ -0,0 +1,86 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.UserGroupInformation; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +public class ITestOBSDefaultInformation { + private OBSFileSystem fs; + private Configuration conf; + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath)); + } + } + + @Test + // 返回客户端可配的默认的BlockSize + public void testGetDefaultBlockSize01() throws Exception { + long defaultBlockSize = 64 * 1024 * 1024; + conf.setLong(OBSConstants.FS_OBS_BLOCK_SIZE, defaultBlockSize); + + OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf); + assertEquals(defaultBlockSize, obsFs.getDefaultBlockSize()); + } + + @Test + // 校验BlockSize默认值为128MB + public void testGetDefaultBlockSize02() { + assertEquals(128 * 1024 * 1024, fs.getDefaultBlockSize()); + } + + @Test + // BlockSize由fs.obs.block.size配置决定,修改dfs.blocksize应无效 + public void testGetDefaultBlockSize03() throws Exception { + conf.setLong(OBSConstants.FS_OBS_BLOCK_SIZE, 128 * 1024 * 1024); + conf.setLong("dfs.blocksize", 64 * 1024 * 1024); + + OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf); + assertEquals(128 * 1024 * 1024, obsFs.getDefaultBlockSize()); + } + + @Test + // getHomeDirectory()要返回用户的Home路径:"/user/" + ugi.getShortUserName() + public void testGetHomeDirectory() { + String homeDirectory = fs.getHomeDirectory().toString(); + + String username; + try { + username = UserGroupInformation.getCurrentUser().getShortUserName(); + } catch (IOException ex) { + username = System.getProperty("user.name"); + } + + assertTrue(homeDirectory.endsWith("/user/" + username)); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDeleteAndRenameManyFiles.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDeleteAndRenameManyFiles.java new file mode 100644 index 0000000..2bac2a5 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDeleteAndRenameManyFiles.java @@ -0,0 +1,430 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.concurrent.Callable; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Executors; +import java.util.concurrent.Future; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertPathExists; +import static org.apache.hadoop.fs.contract.ContractTestUtils.bandwidth; +import static org.apache.hadoop.fs.contract.ContractTestUtils.verifyReceivedData; +import static org.junit.Assert.assertEquals; + +/** + * Test some scalable operations related to file renaming and deletion. + */ +public class ITestOBSDeleteAndRenameManyFiles { + private OBSFileSystem fs; + + Configuration conf; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static final Logger LOG = + LoggerFactory.getLogger(ITestOBSDeleteAndRenameManyFiles.class); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath)); + } + } + + private Path getTestPath() { + return new Path(testRootPath + "/test-obs"); + } + + @Test + // 批量rename多级目录结构路径,rename后,源不存在(对象桶先拷贝后删除,文件桶直接rename) + public void testBulkRenameAndDelete() throws Throwable { + final Path scaleTestDir = new Path(getTestPath(), + "testBulkRenameAndDelete"); + final Path srcDir = new Path(scaleTestDir, "1/2/3/src"); + final Path distDir = new Path(scaleTestDir, "1/2/src1"); + final Path finalDir = new Path(scaleTestDir, "1/2/src1/src"); + final long count = + conf.getLong(OBSTestConstants.KEY_OPERATION_COUNT, + OBSTestConstants.DEFAULT_OPERATION_COUNT); + + boolean needCreat = true; + ContractTestUtils.rm(fs, new Path("/1"), true, false); + ContractTestUtils.rm(fs, scaleTestDir, true, false); + fs.mkdirs(srcDir); + fs.mkdirs(distDir); + + int testBufferSize = fs.getConf() + .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE, + ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE); + // use Executor to speed up file creation + ExecutorService exec = Executors.newFixedThreadPool(16); + final ExecutorCompletionService completionService = + new ExecutorCompletionService<>(exec); + try { + final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a', + 'z'); + + if (needCreat) { + for (int i = 0; i < count; ++i) { + final String fileName = "foo-" + i; + completionService.submit(new Callable() { + @Override + public Boolean call() throws IOException { + ContractTestUtils.createFile(fs, + new Path(srcDir, fileName), false, data); + return fs.exists(new Path(srcDir, fileName)); + } + }); + } + } + + for (int i = 0; i < 999; ++i) { + final Future future = completionService.take(); + try { + if (!future.get()) { + LOG.warn("cannot create file"); + } + } catch (ExecutionException e) { + LOG.warn("Error while uploading file", e.getCause()); + throw e; + } + } + } finally { + exec.shutdown(); + } + + int nSrcFiles = fs.listStatus(srcDir).length; + + fs.rename(srcDir, distDir); + assertEquals(nSrcFiles, fs.listStatus(finalDir).length); + ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename", + new Path(srcDir, "foo-" + 0)); + ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename", + new Path(srcDir, "foo-" + count / 2)); + ContractTestUtils.assertPathDoesNotExist(fs, "not deleted after rename", + new Path(srcDir, "foo-" + (count - 1))); + assertPathExists(fs, "not renamed to dest dir", + new Path(finalDir, "foo-" + 0)); + assertPathExists(fs, "not renamed to dest dir", + new Path(finalDir, "foo-" + count / 2)); + assertPathExists(fs, "not renamed to dest dir", + new Path(finalDir, "foo-" + (count - 1))); + + fs.delete(scaleTestDir, true); + } + + @Test + // 不带delimiter list多级目录,测试功能正确性 + public void testListWithoutDelimiter() throws Throwable { + final Path scaleTestDir = new Path(getTestPath(),"testListDelimiter"); + final Path srcDir = new Path(scaleTestDir, "1/2/3/src"); + final Path finalDir = new Path(scaleTestDir, "1/2/src1"); + final long count = conf.getLong(OBSTestConstants.KEY_OPERATION_COUNT, + 10); + + boolean needCreat = true; + ContractTestUtils.rm(fs, new Path("/1"), true, false); + ContractTestUtils.rm(fs, scaleTestDir, true, false); + fs.mkdirs(srcDir); + fs.mkdirs(finalDir); + + int testBufferSize = fs.getConf() + .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE, + ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE); + // use Executor to speed up file creation + ExecutorService exec = Executors.newFixedThreadPool(16); + final ExecutorCompletionService completionService = + new ExecutorCompletionService<>(exec); + try { + final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a', + 'z'); + + if (needCreat) { + for (int i = 0; i < count; ++i) { + final String fileName = "foo-" + i; + completionService.submit(new Callable() { + @Override + public Boolean call() throws IOException { + ContractTestUtils.createFile(fs, + new Path(srcDir, fileName), false, data); + return fs.exists(new Path(srcDir, fileName)); + } + }); + } + final Path srcSub1Dir = new Path(scaleTestDir, + "1/2/3/src/sub1"); + for (int i = 0; i < 5; ++i) { + final String fileName = "foo1-" + i; + ContractTestUtils.createFile(fs, + new Path(srcSub1Dir, fileName), false, data); + } + final Path srcSub2Dir = new Path(scaleTestDir, + "1/2/3/src/sub1/sub2"); + for (int i = 0; i < 5; ++i) { + final String fileName = "foo2-" + i; + ContractTestUtils.createFile(fs, + new Path(srcSub2Dir, fileName), false, data); + } + } + + for (int i = 0; i < count; ++i) { + final Future future = completionService.take(); + try { + if (!future.get()) { + LOG.warn("cannot create file"); + } + } catch (ExecutionException e) { + LOG.warn("Error while uploading file", e.getCause()); + throw e; + } + } + } finally { + exec.shutdown(); + } + String sreKey = pathToKey(srcDir); + if (!sreKey.endsWith("/")) { + sreKey = sreKey + "/"; + } + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(fs.getBucket()); + request.setPrefix(sreKey); + request.setMaxKeys(1000); + + fs.getObsClient().listObjects(request); + fs.delete(scaleTestDir, true); + } + + @Test + // 测试大文件的rename,rename后校验目标文件内容和源文件相同 + public void testHugeFileRename() throws Throwable { + int testBufferSize = fs.getConf() + .getInt( + ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 1024 * 1024); + int modulus = fs.getConf().getInt( + ContractTestUtils.IO_CHUNK_MODULUS_SIZE, 128); + long fileSize = 1024 * 1024 * 1024L; + final Path srcDir = new Path(getTestPath(), "src"); + final Path distDir = new Path(getTestPath(), "final"); + final Path finalDir = new Path(distDir, "src"); + + ContractTestUtils.rm(fs, getTestPath(), true, false); + fs.mkdirs(srcDir); + fs.mkdirs(distDir); + Path objectPath = new Path(srcDir, "copy-test-file"); + Path renamePath = new Path(finalDir, "copy-test-file"); + boolean needUpload = true; + ContractTestUtils.NanoTimer timer = new ContractTestUtils.NanoTimer(); + + byte[] testBuffer = new byte[testBufferSize]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % modulus); + } + + long bytesWritten = 0L; + + Throwable var10 = null; + + long diff; + if (needUpload) { + OutputStream outputStream = fs.create(objectPath, false); + try { + while (bytesWritten < fileSize) { + diff = fileSize - bytesWritten; + if (diff < (long) testBuffer.length) { + outputStream.write(testBuffer, 0, (int) diff); + bytesWritten += diff; + } else { + outputStream.write(testBuffer); + bytesWritten += (long) testBuffer.length; + } + } + + diff = bytesWritten; + } catch (Throwable var21) { + var10 = var21; + throw var21; + } finally { + if (outputStream != null) { + if (var10 != null) { + try { + outputStream.close(); + } catch (Throwable var20) { + var10.addSuppressed(var20); + } + } else { + outputStream.close(); + } + } + + } + } + + fs.rename(srcDir, distDir); + // assertEquals(fileSize, diff); + assertPathExists(fs, "not created successful", renamePath); + timer.end("Time to write %d bytes", fileSize); + bandwidth(timer, fileSize); + + try { + verifyReceivedData(fs, renamePath, fileSize, testBufferSize, + modulus); + } finally { + fs.delete(renamePath, false); + } + } + + @Test + // 带delimiter list大目录,测试功能正确性 + public void testListingWithDelimiter() throws Throwable { + final Path scaleTestDir = new Path(getTestPath(), "testListing"); + final Path srcDir = new Path(scaleTestDir, "1/src"); + final Path finalDir = new Path(scaleTestDir, "1/2/src2"); + final long count = conf.getLong(OBSTestConstants.KEY_OPERATION_COUNT, + 1200); + + boolean needCreat = true; + ContractTestUtils.rm(fs, new Path("/1"), true, false); + ContractTestUtils.rm(fs, scaleTestDir, true, false); + fs.mkdirs(srcDir); + fs.mkdirs(finalDir); + + int testBufferSize = fs.getConf() + .getInt(ContractTestUtils.IO_CHUNK_BUFFER_SIZE, + ContractTestUtils.DEFAULT_IO_CHUNK_BUFFER_SIZE); + // use Executor to speed up file creation + ExecutorService exec = Executors.newFixedThreadPool(16); + final ExecutorCompletionService completionService = + new ExecutorCompletionService<>(exec); + try { + final byte[] data = ContractTestUtils.dataset(testBufferSize, 'a', + 'z'); + + if (needCreat) { + for (int i = 0; i < count; ++i) { + final String fileName = "foo-" + i; + completionService.submit(new Callable() { + @Override + public Boolean call() throws IOException { + ContractTestUtils.createFile(fs, + new Path(srcDir, fileName), false, data); + return fs.exists(new Path(srcDir, fileName)); + } + }); + } + final Path srcSub1Dir = new Path(scaleTestDir, "1/src/sub1"); + for (int i = 0; i < 5; ++i) { + final String fileName = "foo1-" + i; + ContractTestUtils.createFile(fs, + new Path(srcSub1Dir, fileName), false, data); + } + final Path srcSub2Dir = new Path(scaleTestDir, + "1/src/sub1/sub2"); + for (int i = 0; i < 5; ++i) { + final String fileName = "foo2-" + i; + ContractTestUtils.createFile(fs, + new Path(srcSub2Dir, fileName), false, data); + } + } + + for (int i = 0; i < count; ++i) { + final Future future = completionService.take(); + try { + if (!future.get()) { + LOG.warn("cannot create file"); + } + } catch (ExecutionException e) { + LOG.warn("Error while uploading file", e.getCause()); + throw e; + } + } + } finally { + exec.shutdown(); + } + String sreKey = pathToKey(srcDir); + if (!sreKey.endsWith("/")) { + sreKey = sreKey + "/"; + } + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(fs.getBucket()); + request.setPrefix(sreKey); + request.setDelimiter("/"); + request.setMaxKeys(1000); + + ObjectListing objects = fs.getObsClient().listObjects(request); + + while (true) { + if (!objects.isTruncated()) { + break; + } + + objects = OBSCommonUtils.continueListObjects(fs, objects); + } + fs.delete(scaleTestDir, true); + } + + private String pathToKey(Path path) { + if (!path.isAbsolute()) { + path = new Path(fs.getWorkingDirectory(), path); + } + + if (path.toUri().getScheme() != null && path.toUri() + .getPath() + .isEmpty()) { + return ""; + } + + return path.toUri().getPath().substring(1); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDisguisePermissionSupport.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDisguisePermissionSupport.java new file mode 100644 index 0000000..e686e40 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDisguisePermissionSupport.java @@ -0,0 +1,154 @@ +package org.apache.hadoop.fs.obs; + +import com.obs.services.model.SetObjectMetadataRequest; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Test; +import org.junit.Rule; + +import java.util.Map; + +import static org.junit.Assert.assertEquals; + +public class ITestOBSDisguisePermissionSupport { + private OBSFileSystem fs; + + private static String testRootPath = OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.PERMISSIONS_MODE, OBSConstants.PERMISSIONS_MODE_DISGUISE); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath, relativePath); + } + + @Test + public void testOBSSetOwner() throws Exception { + if (!fs.supportDisguisePermissionsMode()) { + return; + } + Path testFile = getTestPath("file_for_permission_test"); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.setOwner(testFile, "hadoop", "root"); + Map objMap = fs.getObsClient().getObjectMetadata(fs.getBucket(), + OBSCommonUtils.pathToKey(fs, testFile)).getAllMetadata(); + assertEquals("hadoop", objMap.get("user").toString()); + assertEquals("root", objMap.get("group").toString()); + } + + @Test + public void testOBSSetPermission() throws Exception { + if (!fs.supportDisguisePermissionsMode()) { + return; + } + Path testFile = getTestPath("file_for_owner_and_group_test"); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.setPermission(testFile, new FsPermission((short)00400)); + Map objMap = fs.getObsClient().getObjectMetadata(fs.getBucket(), + OBSCommonUtils.pathToKey(fs, testFile)).getAllMetadata(); + assertEquals("256", objMap.get("permission").toString()); + } + + @Test + public void testOBSGetFileStatusForFile() throws Exception { + if (!fs.supportDisguisePermissionsMode()) { + return; + } + Path testFile= getTestPath("file_for_get_file_status_test"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.close(); + testOBDGetFileStatus(testFile); + } + + @Test + public void testOBSGetFileStatusForFolder() throws Exception { + if (!fs.supportDisguisePermissionsMode()) { + return; + } + Path testFolder = getTestPath("folder_for_get_file_status_test"); + fs.mkdirs(testFolder); + testOBDGetFileStatus(testFolder); + } + + private void testOBDGetFileStatus(Path p) throws Exception { + fs.setPermission(p, new FsPermission((short) 00640)); + fs.setOwner(p, "root", "supergroup"); + FileStatus status = fs.getFileStatus(p); + assertEquals("rw-r-----", status.getPermission().toString()); + assertEquals("root", status.getOwner()); + assertEquals("supergroup", status.getGroup()); + } + + @Test + public void testOBSGetFileStatusPermissionInvalid() throws Exception { + if (!fs.supportDisguisePermissionsMode()) { + return; + } + Path testFile= getTestPath("file_for_get_file_status_test"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.close(); + SetObjectMetadataRequest req = new SetObjectMetadataRequest(fs.getBucket(), + OBSCommonUtils.pathToKey(fs, testFile)); + req.addUserMetadata("permission", "invalid_permission"); + fs.getObsClient().setObjectMetadata(req); + FileStatus status = fs.getFileStatus(testFile); + assertEquals("rw-rw-rw-", status.getPermission().toString()); + } + + @Test + public void testOBSListFiles() throws Exception { + if (!fs.supportDisguisePermissionsMode()) { + return; + } + Path testFolder = getTestPath("folder_for_list_files_test"); + fs.mkdirs(testFolder); + fs.setPermission(testFolder, new FsPermission((short) 00644)); + fs.setOwner(testFolder, "root", "supergroup"); + + Path testFile = getTestPath("file_for_list_files_test"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.close(); + fs.setPermission(testFile, new FsPermission((short) 00644)); + fs.setOwner(testFile, "root", "supergroup"); + + RemoteIterator fileStatus = fs.listFiles(new Path(testRootPath), true); + while(fileStatus.hasNext()) { + LocatedFileStatus status = fileStatus.next(); + assertEquals("rw-r--r--", status.getPermission().toString()); + assertEquals("root", status.getOwner()); + assertEquals("supergroup", status.getGroup()); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDiskBufferOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDiskBufferOutputStream.java new file mode 100644 index 0000000..645d71a --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSDiskBufferOutputStream.java @@ -0,0 +1,283 @@ +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.assertEquals; +import static org.apache.hadoop.fs.contract.ContractTestUtils.createAndVerifyFile; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyObject; + +import com.obs.services.ObsClient; +import com.obs.services.model.AbortMultipartUploadRequest; +import com.obs.services.model.ListMultipartUploadsRequest; +import com.obs.services.model.MultipartUpload; +import com.obs.services.model.MultipartUploadListing; +import com.obs.services.model.fs.ObsFSAttribute; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.LocalDirAllocator; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.io.IOUtils; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.mockito.Mockito; +import org.powermock.reflect.Whitebox; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.File; +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.Date; + +@RunWith(Parameterized.class) +public class ITestOBSDiskBufferOutputStream { + private static final Logger LOG = LoggerFactory.getLogger(ITestOBSDiskBufferOutputStream.class); + private OBSFileSystem fs; + private boolean calcMd5; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Parameterized.Parameters + public static Collection calcMd5() { + return Arrays.asList(false, true); + } + + public ITestOBSDiskBufferOutputStream(boolean calcMd5) { + this.calcMd5 = calcMd5; + } + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.FAST_UPLOAD, true); + conf.setLong(OBSConstants.MULTIPART_SIZE, 5 * 1024 * 1024); + conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, calcMd5); + fs = OBSTestUtils.createTestFileSystem(conf); + deleteTmpDir(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // disk写缓存,上传对象时构造mock异常,触发异常后,清理上传的碎片 + public void testMockUploadPartError() throws Exception { + Path dest = getTestPath("testMockUploadPartError"); + IOException exception = null; + IOException closeException = null; + FSDataOutputStream stream = null; + try { + stream = fs.create(dest, true); + ((OBSBlockOutputStream) stream.getWrappedStream()).mockPutPartError( + true); + byte[] data = ContractTestUtils.dataset(10 * 1204 * 1024, 'a', 26); + stream.write(data); + stream.close(); + stream.write(data); + } catch (IOException e) { + exception = e; + } finally { + try { + if (stream != null) { + stream.close(); + } + } catch (IOException e) { + closeException = e; + } + fs.delete(dest, false); + } + assertTrue(exception != null && exception.getMessage() + .contains("Multi-part upload")); + assertTrue(closeException != null && closeException.getMessage() + .contains("Multi-part upload")); + + // 删除多段碎片 + final Date purgeBefore = new Date(); + ListMultipartUploadsRequest request + = new ListMultipartUploadsRequest(fs.getBucket()); + while (true) { + // List + purge + MultipartUploadListing uploadListing = fs.getObsClient() + .listMultipartUploads(request); + for (MultipartUpload upload : uploadListing.getMultipartTaskList()) { + LOG.info("MultipartTask:create time {},purge time {}",upload.getInitiatedDate(),purgeBefore); + if (upload.getInitiatedDate().compareTo(purgeBefore) < 0) { + LOG.info("abort MultipartTask"); + fs.getObsClient().abortMultipartUpload( + new AbortMultipartUploadRequest(fs.getBucket(), upload.getObjectKey(), upload.getUploadId())); + } + } + if (!uploadListing.isTruncated()) { + break; + } + request.setUploadIdMarker( + uploadListing.getNextUploadIdMarker()); + request.setKeyMarker(uploadListing.getNextKeyMarker()); + } + + // 校验缓存目录为空 + assertTrue(verifyTmpDirEmpty(fs.getConf())); + } + + @Test + // disk写缓存,上传对象并校验数据通过 + public void testBlockUpload() throws IOException { + // ContractTestUtils.createAndVerifyFile(fs, testPath, 0); + // verifyUpload(100*1024-1); + verifyUpload(10 * 1024 * 1024 + 1); + // 校验缓存目录为空 + assertTrue(verifyTmpDirEmpty(fs.getConf())); + // verifyUpload(10*1024*1024); + // verifyUpload(10*1024*1024-1); + } + + @Test + // disk写缓存,测试0字节大小文件上传并校验文件大小 + public void testDiskZeroUpload() throws IOException { + // ContractTestUtils.createAndVerifyFile(fs, testPath, 0); + // verifyUpload(100*1024-1); + verifyUpload(0); + // 校验缓存目录为空 + assertTrue(verifyTmpDirEmpty(fs.getConf())); + } + + @Test + // disk写缓存,上传对象关闭流后,再写数据抛出IOException + public void testDiskWriteAfterStreamClose() throws IOException { + Path dest = getTestPath("testWriteAfterStreamClose"); + FSDataOutputStream stream = fs.create(dest, true); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + try { + stream.write(data); + stream.close(); + boolean hasException = false; + try { + stream.write(data); + } catch (IOException e) { + hasException = true; + } + assertTrue(hasException); + + } finally { + fs.delete(dest, false); + IOUtils.closeStream(stream); + } + // 校验缓存目录为空 + assertTrue(verifyTmpDirEmpty(fs.getConf())); + } + + @Test + // disk写缓存,上传对象流关闭后缓存块被清空 + public void testBlocksClosed() throws Throwable { + Path dest = getTestPath("testBlocksClosed"); + + FSDataOutputStream stream = fs.create(dest, true); + OBSBlockOutputStream obsStream = + (OBSBlockOutputStream)stream.getWrappedStream(); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + assertTrue(null == obsStream.getActiveBlock()); + fs.delete(dest, false); + // 校验缓存目录为空 + assertTrue(verifyTmpDirEmpty(fs.getConf())); + } + + private void verifyUpload(long fileSize) throws IOException { + createAndVerifyFile(fs, getTestPath("test_file"), fileSize); + } + + @Test + // append写,当head返回文件长度与客户端记录长度不一致时,以大的长度为准 + public void testAppendWithIncorrectContentLen() throws Exception { + if (!fs.isFsBucket()) { + return; + } + OBSFileSystem mockFs = Mockito.spy(fs); + ObsClient client = fs.getObsClient(); + ObsClient mockClient = Mockito.spy(client); + Whitebox.setInternalState(mockFs, client, mockClient); + + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {0, 1, 2, 3, 4, 5, 6, 7, 8, 9}; + outputStream.write(data); + outputStream.close(); + + // 第一次append调用实际head接口,第二次append使用mock的ObsFSAttribute + ObsFSAttribute attribute = new ObsFSAttribute(); + attribute.setContentLength(5L); + Mockito.doCallRealMethod() + .doReturn(attribute) + .when(mockClient) + .getAttribute(anyObject()); + + // 第一次写正常,写完后文件长度为20 + outputStream = mockFs.append(testFile, 4096, null); + outputStream.write(data); + outputStream.hflush(); + assertEquals(20, fs.getFileStatus(testFile).getLen()); + + // 第二次getAttribute得到的文件大小不正确,以大的为准,从20追加写,写完后文件长度为30 + outputStream.write(data); + outputStream.close(); + assertEquals(30, fs.getFileStatus(testFile).getLen()); + + fs.delete(testFile, false); + // 校验缓存目录为空 + assertTrue(verifyTmpDirEmpty(fs.getConf())); + } + + private boolean verifyTmpDirEmpty(Configuration conf) throws IOException { + String bufferDir = conf.get(OBSConstants.BUFFER_DIR) != null + ? OBSConstants.BUFFER_DIR + : "hadoop.tmp.dir"; + LocalDirAllocator allocator = new LocalDirAllocator(bufferDir); + String tmpBuff = "obs-block-0001"; + Path path = allocator.getLocalPathForWrite(tmpBuff, conf); + File parentDir = new File(path.getParent().toUri().toString()); + assertTrue(parentDir.isDirectory()); + return parentDir.list().length == 0; + } + + private void deleteTmpDir(Configuration conf) throws IOException { + String bufferDir = conf.get(OBSConstants.BUFFER_DIR) != null + ? OBSConstants.BUFFER_DIR + : "hadoop.tmp.dir"; + LocalDirAllocator allocator = new LocalDirAllocator(bufferDir); + String tmpBuff = "obs-block-0001"; + Path path = allocator.getLocalPathForWrite(tmpBuff, conf); + File parentDir = new File(path.getParent().toUri().toString()); + + File[] children = parentDir.listFiles(); + for (int i = 0; i < children.length; i++) { + children[i].delete(); + } + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFSDataOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFSDataOutputStream.java new file mode 100644 index 0000000..225823c --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFSDataOutputStream.java @@ -0,0 +1,242 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; + +import java.util.Arrays; +import java.util.Collection; + +@RunWith(Parameterized.class) +public class ITestOBSFSDataOutputStream { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private final int fileSize4 = 4 * 1024 * 1024; + + private final int fileSize5 = 5 * 1024 * 1024; + + private final int fileSize6 = 6 * 1024 * 1024; + + private final boolean caclMd5; + + private final String blockPolicy; + + @Parameterized.Parameters + public static Collection digestPolicy() { + return Arrays.asList( + new PolicyParam(false, OBSConstants.FAST_UPLOAD_BUFFER_DISK), + new PolicyParam(false, OBSConstants.FAST_UPLOAD_BUFFER_ARRAY), + new PolicyParam(false, OBSConstants.FAST_UPLOAD_BYTEBUFFER), + new PolicyParam(true, OBSConstants.FAST_UPLOAD_BUFFER_DISK), + new PolicyParam(true, OBSConstants.FAST_UPLOAD_BUFFER_ARRAY), + new PolicyParam(true, OBSConstants.FAST_UPLOAD_BYTEBUFFER)); // 是否开启md5校验 + } + + public ITestOBSFSDataOutputStream(PolicyParam param) { + this.caclMd5 = param.calcMd5; + this.blockPolicy = param.blockPolicy; + } + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024)); + conf.setBoolean(OBSConstants.OUTPUT_STREAM_ATTACH_MD5, caclMd5); + conf.set(OBSConstants.FAST_UPLOAD_BUFFER, blockPolicy); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 执行write操作写入5字节,调用close前,未将数据刷到服务端;调用close后,强制将客户端数据刷到服务端; + public void testClose() throws Exception { + if(!fs.isFsBucket()) { + return; + } + + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + assertFalse(fs.exists(testFile)); + byte[] data = {0, 1, 2, 3, 4}; + outputStream.write(data); + + outputStream.hflush(); + assertEquals(5, fs.getFileStatus(testFile).getLen()); + + outputStream.write(data); + assertEquals(5, fs.getFileStatus(testFile).getLen()); + + outputStream.close(); + assertEquals(10, fs.getFileStatus(testFile).getLen()); + fs.delete(testFile, false); + } + + @Test + // create file write 5M ,write4M,write6M hflush ,write5M hsync,write4M + // 共24M + public void testCombination001() throws Exception { + // if (fs.getMetricSwitch()) { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("test_file"); + if (fs.exists(testFile)) { + fs.delete(testFile); + } + FSDataOutputStream outputStream = fs.create(testFile); + byte[] data5 = ContractTestUtils.dataset(fileSize5, 'a', + 26); + outputStream.write(data5); + byte[] data4 = ContractTestUtils.dataset(fileSize4, 'a', + 26); + outputStream.write(data4); + byte[] data6 = ContractTestUtils.dataset(fileSize6, 'a', + 26); + outputStream.write(data6); + outputStream.hflush(); + outputStream.write(data5); + assertEquals(fileSize5 * 4, fs.getFileStatus(testFile).getLen()); + outputStream.hsync(); + assertEquals(fileSize5 * 4, fs.getFileStatus(testFile).getLen()); + + outputStream.write(data4); + outputStream.hsync(); + System.out.println(fs.getFileStatus(testFile).getLen()); + if (outputStream != null) { + outputStream.close(); + } + fs.delete(testFile, false); + // } + } + + @Test + // create file write 5M close ,apend10M, write6M ,write5M,write4M close, + // 共30M + public void testCombination002() throws Exception { + // if (fs.getMetricSwitch()) { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("test_file"); + if (fs.exists(testFile)) { + fs.delete(testFile); + } + + FSDataOutputStream outputStream = fs.create(testFile); + byte[] data5 = ContractTestUtils.dataset(fileSize5, 'a', + 26); + outputStream.write(data5); + outputStream.close(); + + outputStream = fs.append(testFile); + byte[] data10 = ContractTestUtils.dataset(fileSize5 * 2, 'a', + 26); + outputStream.write(data10); + + + byte[] data6 = ContractTestUtils.dataset(fileSize6, 'a', + 26); + outputStream.write(data6); + + outputStream.write(data5); + + + byte[] data4 = ContractTestUtils.dataset(fileSize4, 'a', + 26); + outputStream.write(data4); + + outputStream.close(); + + assertEquals(fileSize5 * 6, fs.getFileStatus(testFile).getLen()); + + fs.delete(testFile, false); + // } + } + + @Test + // create file write 5M close,apend4M, write6M,hflush,write10M,hflush,close, + // 共25M + public void testCombination003() throws Exception { + // if (fs.getMetricSwitch()) { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("test_file"); + if (fs.exists(testFile)) { + fs.delete(testFile); + } + + FSDataOutputStream outputStream = fs.create(testFile); + byte[] data5 = ContractTestUtils.dataset(fileSize5, 'a', + 26); + outputStream.write(data5); + outputStream.close(); + + outputStream = fs.append(testFile); + byte[] data4 = ContractTestUtils.dataset(fileSize4, 'a', + 26); + outputStream.write(data4); + + byte[] data6 = ContractTestUtils.dataset(fileSize6, 'a', + 26); + outputStream.write(data6); + + outputStream.hflush(); + assertEquals(fileSize5 * 3, fs.getFileStatus(testFile).getLen()); + + byte[] data10 = ContractTestUtils.dataset(fileSize5 * 2, 'a', + 26); + outputStream.write(data10); + outputStream.hflush(); + outputStream.close(); + + assertEquals(fileSize5 * 5, fs.getFileStatus(testFile).getLen()); + + fs.delete(testFile, false); + // } + } + + public static class PolicyParam { + boolean calcMd5; + String blockPolicy; + + public PolicyParam(boolean calcMd5, String blockPolicy) { + this.calcMd5 = calcMd5; + this.blockPolicy = blockPolicy; + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDelete.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDelete.java new file mode 100644 index 0000000..182b8da --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDelete.java @@ -0,0 +1,270 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Ignore; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.runners.MockitoJUnitRunner; +import org.powermock.core.classloader.annotations.PowerMockIgnore; +import org.powermock.core.classloader.annotations.PrepareForTest; +import org.powermock.modules.junit4.PowerMockRunner; +import org.powermock.reflect.Whitebox; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.ArrayList; +import java.util.List; +import java.util.regex.Matcher; +import java.util.regex.Pattern; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.anyString; +import static org.powermock.api.mockito.PowerMockito.mockStatic; +import static org.powermock.api.mockito.PowerMockito.when; + +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; + +@RunWith(MockitoJUnitRunner.class) +public class ITestOBSFastDelete { + OBSFileSystem fs; + + OBSFileSystem mockFs; + + ObsClient obsClient; + + ObsClient mockObsClient; + + static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static String trashPathStr = "trash"; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.FAST_DELETE_ENABLE, true); + conf.set(OBSConstants.FAST_DELETE_DIR, trashPathStr); + conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 2); + fs = OBSTestUtils.createTestFileSystem(conf); + obsClient = fs.getObsClient(); + initMock(); + } + + @After + public void tearDown() throws Exception { + if (fs != null && fs.isFsBucket()) { + fs.delete(new Path(testRootPath), true); + fs.delete(new Path(fs.getFastDeleteDir()), true); + } + Mockito.reset(mockFs, mockObsClient); + } + + private void initMock() { + mockFs = Mockito.spy(fs); + mockObsClient = Mockito.spy(obsClient); + Whitebox.setInternalState(mockFs, obsClient, mockObsClient); + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/ITestOBSTrash/" + relativePath); + } + + @Test + // 删除目录,校验trash路径下包含被删除的目录 + public void delFolderToTrash() throws IOException { + if (!fs.isFsBucket()) { + return; + } + final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z'); + List pathList = new ArrayList(); + for (int i = 0; i < 3; i++) { + String objectName = "objectINfolder-" + i; + Path objectPath = getTestPath(objectName); + ContractTestUtils.createFile(fs, objectPath, false, data); + pathList.add(objectPath); + } + fs.delete(new Path(testRootPath), true); + + Path trashPath = + new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr)); + for (Path path : pathList) { + assertTrue(fs.exists(new Path(trashPath, + OBSCommonUtils.pathToKey(fs, path)))); + } + } + + @Test + // 删除文件,校验trash路径下包含被删除的目录 + public void delObjectToTrash() throws IOException { + if (!fs.isFsBucket()) { + return; + } + final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z'); + Path objectPath = getTestPath("test_delete_object"); + ContractTestUtils.createFile(fs, objectPath, false, data); + fs.delete(objectPath, true); + + Path trashPath = + new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr)); + assertTrue(fs.exists(new Path(trashPath, + OBSCommonUtils.pathToKey(fs, objectPath)))); + } + + @Test + // 开启trash机制,先后删除两个同名文件,验证第二个文件trash后的文件名中不包含冒号 + public void testTrashSameNameObject01() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testFile, false); + + outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testFile, false); + + Path trashPath = + new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr)); + Path trashDir = + new Path(trashPath, OBSCommonUtils.pathToKey(fs, + testFile.getParent())); + FileStatus[] files = fs.listStatus(trashDir); + assertTrue(files.length >= 2); + for (FileStatus file : files) { + String path = file.getPath().toString(); + assertFalse(path.substring(path.indexOf("://") + 1).contains(":")); + } + + fs.delete(trashDir, true); + } + + @Test + // 开启trash机制,先后删除两个同名非空目录,验证第二个目录trash后的目录名中不包含冒号 + public void testTrashSameNameObject02() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testDir = getTestPath("test_dir/"); + Path testFile = getTestPath("test_dir/test_file"); + fs.mkdirs(testDir); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testDir, true); + + fs.mkdirs(testDir); + outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testDir, true); + + Path trashPath = + new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr)); + Path trashDir = + new Path(trashPath, OBSCommonUtils.pathToKey(fs, + testDir.getParent())); + FileStatus[] files = fs.listStatus(trashDir); + assertTrue(files.length >= 2); + for (FileStatus file : files) { + String path = file.getPath().toString(); + assertFalse(path.substring(path.indexOf("://") + 1).contains(":")); + } + + fs.delete(trashDir, true); + } + + @Test + // 开启trash机制,先后删除两个同名非空目录和文件,验证第二个文件trash后的文件名中不包含冒号 + public void testTrashSameNameObject03() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testDir = getTestPath("test_dir/"); + Path testFile = getTestPath("test_dir/test_file"); + fs.mkdirs(testDir); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testDir, true); + + outputStream = fs.create(testDir, false); + outputStream.close(); + fs.delete(testDir, true); + + Path trashPath = + new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr)); + Path trashDir = + new Path(trashPath, OBSCommonUtils.pathToKey(fs, + testDir.getParent())); + FileStatus[] files = fs.listStatus(trashDir); + assertTrue(files.length >= 2); + for (FileStatus file : files) { + String path = file.getPath().toString(); + assertFalse(path.substring(path.indexOf("://") + 1).contains(":")); + } + + fs.delete(trashDir, true); + } + + @Test + // 开启trash机制,先后删除两个同名非空目录和文件,验证第二个文件trash后的文件名中不包含冒号 + public void testTrashSameNameObject04() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testDir = getTestPath("test_dir/"); + Path testFile = getTestPath("test_dir/test_file"); + fs.mkdirs(testDir); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testDir, true); + + outputStream = fs.create(testDir, false); + outputStream.close(); + fs.delete(testDir, true); + + Path trashPath = + new Path(OBSCommonUtils.maybeAddBeginningSlash(trashPathStr)); + Path trashDir = + new Path(trashPath, OBSCommonUtils.pathToKey(fs, + testDir.getParent())); + FileStatus[] files = fs.listStatus(trashDir); + assertTrue(files.length >= 2); + for (FileStatus file : files) { + String path = file.getPath().toString(); + assertFalse(path.substring(path.indexOf("://") + 1).contains(":")); + } + + fs.delete(trashDir, true); + } + + @Test + @Ignore + // 测试多客户端执行trash到同一目录时,服务端返409,客户端添加时间戳重试 + public void testTrashWithConflict() throws Exception { + + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDeleteV2.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDeleteV2.java new file mode 100644 index 0000000..a219e64 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFastDeleteV2.java @@ -0,0 +1,210 @@ +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.obs.OBSConstants.FAST_DELETE_VERSION_V2; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.Before; +import org.junit.Ignore; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.runners.MockitoJUnitRunner; +import org.powermock.reflect.Whitebox; + +import java.io.IOException; +import java.text.SimpleDateFormat; +import java.util.ArrayList; +import java.util.Date; +import java.util.List; +import java.util.Locale; + +@RunWith(MockitoJUnitRunner.class) +public class ITestOBSFastDeleteV2 extends ITestOBSFastDelete { + + private static String trashPathStr = "trashV2"; + + @Before + @Override + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.FAST_DELETE_ENABLE, true); + conf.set(OBSConstants.FAST_DELETE_DIR, trashPathStr); + conf.set(OBSConstants.FAST_DELETE_VERSION, FAST_DELETE_VERSION_V2); + conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 2); + fs = OBSTestUtils.createTestFileSystem(conf); + obsClient = fs.getObsClient(); + initMock(); + } + + private void initMock() { + mockFs = Mockito.spy(fs); + mockObsClient = Mockito.spy(obsClient); + Whitebox.setInternalState(mockFs, obsClient, mockObsClient); + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/ITestOBSTrash/" + relativePath); + } + + private Path formatTrashPathV2(String trashPathStr) { + SimpleDateFormat dateFmt = new SimpleDateFormat(OBSConstants.FAST_DELETE_VERSION_V2_CHECKPOINT_FORMAT); + String checkpointStr = dateFmt.format(new Date()); + String checkpointDir = String.format(Locale.ROOT, "%s%s/", + OBSCommonUtils.maybeAddTrailingSlash(trashPathStr), checkpointStr); + return new Path(OBSCommonUtils.maybeAddBeginningSlash(checkpointDir)); + } + + @Test + @Override + // 删除目录,校验trash路径下包含被删除的目录 + public void delFolderToTrash() throws IOException { + if (!fs.isFsBucket()) { + return; + } + final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z'); + List pathList = new ArrayList(); + for (int i = 0; i < 3; i++) { + String objectName = "objectINfolder-" + i; + Path objectPath = getTestPath(objectName); + ContractTestUtils.createFile(fs, objectPath, false, data); + pathList.add(objectPath); + } + fs.delete(new Path(testRootPath), true); + + Path trashPath = formatTrashPathV2(trashPathStr); + + for (Path path : pathList) { + assertTrue(fs.exists(new Path(trashPath, + OBSCommonUtils.pathToKey(fs, path)))); + } + } + + @Test + @Override + // 删除文件,校验trash路径下包含被删除的目录 + public void delObjectToTrash() throws IOException { + if (!fs.isFsBucket()) { + return; + } + final byte[] data = ContractTestUtils.dataset(1024, 'a', 'z'); + Path objectPath = getTestPath("test_delete_object"); + ContractTestUtils.createFile(fs, objectPath, false, data); + fs.delete(objectPath, true); + + Path trashPath = formatTrashPathV2(trashPathStr); + + assertTrue(fs.exists(new Path(trashPath, + OBSCommonUtils.pathToKey(fs, objectPath)))); + } + + + @Test + @Override + // 开启trash机制,先后删除两个同名文件,验证第二个文件trash后的文件名中不包含冒号 + public void testTrashSameNameObject01() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testFile, false); + + outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testFile, false); + + Path trashPath = formatTrashPathV2(trashPathStr); + Path trashDir = + new Path(trashPath, OBSCommonUtils.pathToKey(fs, + testFile.getParent())); + FileStatus[] files = fs.listStatus(trashDir); + assertTrue(files.length >= 2); + for (FileStatus file : files) { + String path = file.getPath().toString(); + assertFalse(path.substring(path.indexOf("://") + 1).contains(":")); + } + + fs.delete(trashDir, true); + } + + @Test + @Override + // 开启trash机制,先后删除两个同名非空目录,验证第二个目录trash后的目录名中不包含冒号 + public void testTrashSameNameObject02() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testDir = getTestPath("test_dir/"); + Path testFile = getTestPath("test_dir/test_file"); + fs.mkdirs(testDir); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testDir, true); + + fs.mkdirs(testDir); + outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testDir, true); + + Path trashPath = formatTrashPathV2(trashPathStr); + Path trashDir = + new Path(trashPath, OBSCommonUtils.pathToKey(fs, + testDir.getParent())); + FileStatus[] files = fs.listStatus(trashDir); + assertTrue(files.length >= 2); + for (FileStatus file : files) { + String path = file.getPath().toString(); + assertFalse(path.substring(path.indexOf("://") + 1).contains(":")); + } + + fs.delete(trashDir, true); + } + + @Test + @Override + // 开启trash机制,先后删除两个同名非空目录和文件,验证第二个文件trash后的文件名中不包含冒号 + public void testTrashSameNameObject03() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testDir = getTestPath("test_dir/"); + Path testFile = getTestPath("test_dir/test_file"); + fs.mkdirs(testDir); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + fs.delete(testDir, true); + + outputStream = fs.create(testDir, false); + outputStream.close(); + fs.delete(testDir, true); + + Path trashPath = formatTrashPathV2(trashPathStr); + Path trashDir = + new Path(trashPath, OBSCommonUtils.pathToKey(fs, + testDir.getParent())); + FileStatus[] files = fs.listStatus(trashDir); + assertTrue(files.length >= 2); + for (FileStatus file : files) { + String path = file.getPath().toString(); + assertFalse(path.substring(path.indexOf("://") + 1).contains(":")); + } + + fs.delete(trashDir, true); + } + + @Test + @Ignore + @Override + // 重复用例,不需要跑 + public void testTrashSameNameObject04() throws Exception { + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystem.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystem.java new file mode 100644 index 0000000..5cae078 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystem.java @@ -0,0 +1,260 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.obs.services.ObsClient; +import com.obs.services.model.ListBucketsRequest; +import com.obs.services.model.ObsBucket; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.Collection; +import java.util.List; + +public class ITestOBSFileSystem { + + private static final int EXPECTED_PORT = -1; + + private static final String EXPECTED_SCHEMA = "obs"; + + private Configuration conf; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + } + + @Test + // 通过configuration初始化FS,功能正常 + public void testInitialization() throws IOException { + String bucketName = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + initializationTest(bucketName); + } + + @Test + public void testInitializeNotExistsBucket() throws URISyntaxException, IOException { + String initializationUri = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + URI uri = new URI(initializationUri); + String scheme = uri.getScheme(); + String bucketName = uri.getAuthority(); + while (true) { + bucketName = String.format("%s-not-exists", bucketName); + OBSFileSystem fs = new OBSFileSystem(); + try { + initializationUri = String.format("%s://%s", scheme, bucketName); + fs.initialize(URI.create(initializationUri), conf); + } catch (FileNotFoundException e) { + assertEquals("Bucket " + bucketName + " does not exist", e.getMessage()); + return; + } + List buckets = getBucketList(fs.getObsClient()); + boolean isBucketInList = false; + for (ObsBucket bucket : buckets) { + if (bucket.getBucketName().equals(fs.getBucket())) { + isBucketInList = true; + break; + } + } + if (isBucketInList) { + continue; + } + assertTrue("not in bucket list, but initalize success", false); + } + } + + @Test + public void testGetTrashRootV1() throws IOException { + Path path = new Path("test"); + try (OBSFileSystem fs = OBSTestUtils.createTestFileSystem(conf)) { + Path obsTrashRoot = fs.getTrashRoot(path); + Path oldTrashRoot = fs.makeQualified(new Path(fs.getHomeDirectory().toUri().getPath(), ".Trash")); + assertEquals(oldTrashRoot, obsTrashRoot); + } + } + + @Test + public void testGetTrashRootV2() throws IOException { + Path path = new Path("test"); + Configuration newConf = new Configuration(conf); + newConf.set(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V2); + try (OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(newConf)) { + Path obsTrashRoot2 = newFs.getTrashRoot(path); + Path ourTrashRoot = + newFs.makeQualified(new Path(OBSConstants.DEFAULT_HDFS_TRASH_PREFIX, OBSFileSystem.getUsername())); + assertEquals(ourTrashRoot, obsTrashRoot2); + } + } + + @Test + public void testCustomTrashPrefix() throws IOException { + Path path = new Path("test"); + Configuration newConf = new Configuration(conf); + newConf.set(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V2); + String prefix = "/tmp/.trash"; + newConf.set(OBSConstants.HDFS_TRASH_PREFIX, prefix); + try (OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(newConf)) { + Path obsTrashRoot2 = newFs.getTrashRoot(path); + Path ourTrashRoot = + newFs.makeQualified(new Path(prefix, OBSFileSystem.getUsername())); + assertEquals(ourTrashRoot, obsTrashRoot2); + } + } + + @Test + public void testGetAllTrashRootsV1() throws IOException { + testGetTrashRootsV1(true); + } + + @Test + public void testGetCurUserTrashRootsV1() throws IOException { + testGetTrashRootsV1(false); + } + + @Test + public void testGetAllTrashRootsV2() throws IOException { + testGetTrashRootsV2(true, null); + } + + @Test + public void testCustomPrefixGetAllTrashRootsV2() throws IOException { + testGetTrashRootsV2(true, "/tmp/.Trash"); + } + + @Test + public void testGetCurUserTrashRootsV2() throws IOException { + testGetTrashRootsV2(false, null); + } + + @Test + public void testCustomPrefixGetCurUserTrashRootsV2() throws IOException { + testGetTrashRootsV2(false, "/tmp/.Trash"); + } + + public void testGetTrashRootsV1(boolean allUsers) throws IOException { + OBSFileSystem fs = OBSTestUtils.createTestFileSystem(conf); + Path user66TrashRoot = new Path(fs.getHomeDirectory().getParent(), "user66/.Trash"); + Path curUserTrashRoot = fs.makeQualified(new Path(fs.getHomeDirectory().toUri().getPath(), ".Trash")); + try { + fs.mkdirs(user66TrashRoot); + fs.mkdirs(curUserTrashRoot); + Collection trashRoots = fs.getTrashRoots(allUsers); + if (allUsers) { + assertTrue(containsPathInFileStatuses(trashRoots, user66TrashRoot)); + assertTrue(containsPathInFileStatuses(trashRoots, curUserTrashRoot)); + } else { + assertEquals(1, trashRoots.size()); + assertEquals(curUserTrashRoot, trashRoots.toArray(new FileStatus[0])[0].getPath()); + } + } finally { + fs.delete(user66TrashRoot.getParent(), true); + fs.delete(curUserTrashRoot.getParent(), true); + fs.close(); + } + } + + public void testGetTrashRootsV2(boolean allUsers, String hdfsTrashPrefix) throws IOException { + Configuration newConf = new Configuration(conf); + newConf.set(OBSConstants.HDFS_TRASH_VERSION, OBSConstants.HDFS_TRASH_VERSION_V2); + if (hdfsTrashPrefix == null) { + hdfsTrashPrefix = OBSConstants.DEFAULT_HDFS_TRASH_PREFIX; + } else { + newConf.set(OBSConstants.HDFS_TRASH_PREFIX, hdfsTrashPrefix); + } + + OBSFileSystem newFs = OBSTestUtils.createTestFileSystem(newConf); + Path ourCurUserTrashRoot = newFs.makeQualified(new Path(hdfsTrashPrefix, OBSFileSystem.getUsername())); + Path user88TrashDir = new Path(ourCurUserTrashRoot.getParent(), "user88"); + try { + newFs.mkdirs(user88TrashDir); + newFs.mkdirs(ourCurUserTrashRoot); + Collection trashRoots = newFs.getTrashRoots(allUsers); + if (allUsers) { + assertTrue(containsPathInFileStatuses(trashRoots, user88TrashDir)); + assertTrue(containsPathInFileStatuses(trashRoots, ourCurUserTrashRoot)); + } else { + assertEquals(1, trashRoots.size()); + assertEquals(ourCurUserTrashRoot, trashRoots.toArray(new FileStatus[0])[0].getPath()); + } + } finally { + newFs.delete(user88TrashDir, true); + newFs.delete(ourCurUserTrashRoot, true); + newFs.close(); + } + } + + private boolean containsPathInFileStatuses(Collection fileStatuses, Path path) { + for (FileStatus status : fileStatuses) { + if (status.getPath().equals(path)) { + return true; + } + } + + return false; + } + + private void initializationTest(String initializationUri) throws IOException { + OBSFileSystem fs = new OBSFileSystem(); + fs.initialize(URI.create(initializationUri), conf); + ObsClient obsTest = fs.getObsClient(); + List buckets = getBucketList(obsTest); + + boolean isBucketInList = false; + for (ObsBucket bucket : buckets) { + if (bucket.getBucketName().equals(fs.getBucket())) { + isBucketInList = true; + break; + } + } + + URI EXPECTED_URI = URI.create(conf.get(OBSTestConstants.TEST_FS_OBS_NAME)); + assertTrue(isBucketInList); + assertEquals(EXPECTED_SCHEMA, fs.getScheme()); + assertEquals(EXPECTED_URI, fs.getUri()); + assertEquals(EXPECTED_PORT, fs.getDefaultPort()); + } + + private List getBucketList(ObsClient obsTest) { + ListBucketsRequest request = new ListBucketsRequest(); + request.setQueryLocation(true); + List buckets = obsTest.listBuckets(request); + return buckets; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystemContract.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystemContract.java new file mode 100644 index 0000000..b84ea62 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSFileSystemContract.java @@ -0,0 +1,722 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystemContractBaseTest; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.TestName; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.ArrayList; +import java.util.List; + +import static junit.framework.TestCase.assertTrue; +import static junit.framework.TestCase.fail; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** + * Tests a live OBS system. If your keys and bucket aren't specified, all tests + * are marked as passed. + *

+ * This uses BlockJUnit4ClassRunner because FileSystemContractBaseTest from + * TestCase which uses the old Junit3 runner that doesn't ignore assumptions + * properly making it impossible to skip the tests if we don't have a valid + * bucket. + **/ +public class ITestOBSFileSystemContract extends FileSystemContractBaseTest { + + protected static final Logger LOG = + LoggerFactory.getLogger(ITestOBSFileSystemContract.class); + + private Path basePath; + + private OBSFileSystem obsFS; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public TestName methodName = new TestName(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + obsFS = OBSTestUtils.createTestFileSystem(conf); + fs = obsFS; + basePath = fs.makeQualified( + OBSTestUtils.createTestPath(new Path("/obsfilesystemcontract"))); + } + + @Override + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(basePath, true); + } + super.tearDown(); + } + + @Override + public void testMkdirsWithUmask() throws Exception { + // not supported + } + + @Override + // rename源目录到已存在目标目录,源目录下对象被rename到目标目录下 + public void testRenameDirectoryAsExistingDirectory() throws Exception { + if (!renameSupported()) { + return; + } + + // Prepare the source folder with some nested files or sub folders. + Path src = path("/test/hadoop/dir"); + fs.mkdirs(src); + + createFile(path("/test/hadoop/dir/file1")); + createFile(path("/test/hadoop/dir/subdir1/file11")); + createFile(path("/test/hadoop/dir/subdir2/file21")); + createFile(path("/test/hadoop/dir/subdir1/subdir11/file111")); + createFile(path("/test/hadoop/dir/file2")); + fs.mkdirs(path("/test/hadoop/dir/subdir3/subdir31")); + + // Prepare the destination folder. + Path dst = path("/test/new/newdir"); + fs.mkdirs(dst); + + // Execute the rename operation. + rename(src, dst, true, false, true); + + // Assert that all nested files or sub folders under the source + // folder should be not existed after rename. + assertFalse("Nested file1 exists", + fs.exists(path("/test/hadoop/dir/file1"))); + assertFalse("Nested file2 exists", + fs.exists(path("/test/hadoop/dir/file2"))); + assertFalse("Nested file11 exists", + fs.exists(path("/test/hadoop/dir/subdir1/file11"))); + assertFalse("Nested file111 exists", + fs.exists(path("/test/hadoop/dir/subdir1/subdir11/file111"))); + assertFalse("Nested file111 exists", + fs.exists(path("/test/hadoop/dir/subdir1/subdir11"))); + assertFalse("Nested file111 exists", + fs.exists(path("/test/hadoop/dir/subdir1"))); + assertFalse("Nested file21 exists", + fs.exists(path("/test/hadoop/dir/subdir2/file21"))); + assertFalse("Nested file21 exists", + fs.exists(path("/test/hadoop/dir/subdir2"))); + assertFalse("Nested subdir31 exists", + fs.exists(path("/test/hadoop/dir/subdir3/subdir31"))); + assertFalse("Nested subdir31 exists", + fs.exists(path("/test/hadoop/dir/subdir3"))); + assertFalse("Nested subdir31 exists", + fs.exists(path("/test/hadoop/dir"))); + + // Assert that all nested files or sub folders should be under the destination folder after rename. + assertTrue("Renamed nested file1 exists", + fs.exists(path("/test/new/newdir/dir/file1"))); + assertTrue("Renamed nested file2 exists", + fs.exists(path("/test/new/newdir/dir/file2"))); + assertTrue("Renamed nested file11 exists", + fs.exists(path("/test/new/newdir/dir/subdir1"))); + assertTrue("Renamed nested file11 exists", + fs.exists(path("/test/new/newdir/dir/subdir1/file11"))); + assertTrue("Renamed nested file111 exists", + fs.exists(path("/test/new/newdir/dir/subdir1/subdir11"))); + assertTrue("Renamed nested file111 exists", + fs.exists(path("/test/new/newdir/dir/subdir1/subdir11/file111"))); + assertTrue("Renamed nested file21 exists", + fs.exists(path("/test/new/newdir/dir/subdir2"))); + assertTrue("Renamed nested file21 exists", + fs.exists(path("/test/new/newdir/dir/subdir2/file21"))); + assertTrue("Renamed nested file21 exists", + fs.exists(path("/test/new/newdir/dir/subdir3"))); + assertTrue("Renamed nested subdir31 exists", + fs.exists(path("/test/new/newdir/dir/subdir3/subdir31"))); + } + + @Override + // + public void testRenameDirectoryMoveToExistingDirectory() throws Exception { + if (this.renameSupported()) { + Path src = this.path("/test/hadoop/dir"); + this.fs.mkdirs(src); + this.createFile(this.path("/test/hadoop/dir/file1")); + this.createFile(this.path("/test/hadoop/dir/subdir/file2")); + Path dst = this.path("/test/new/newdir"); + this.fs.mkdirs(dst.getParent()); + this.rename(src, dst, true, false, true); + + assertFalse("Nested file2 exists", + this.fs.exists(this.path("/test/hadoop/dir"))); + assertTrue("Renamed nested newdir exists", + this.fs.exists(this.path("/test/new/newdir"))); + assertTrue("Renamed nested file1 exists", + this.fs.exists(this.path("/test/new/newdir/file1"))); + assertTrue("Renamed nested subdir exists", + this.fs.exists(this.path("/test/new/newdir/subdir"))); + assertTrue("Renamed nested file2 exists", + this.fs.exists(this.path("/test/new/newdir/subdir/file2"))); + } + } + + // @Override + public void testMoveDirUnderParent() throws Throwable { + // not support because + // Fails if dst is a directory that is not empty. + } + + public void testRecursivelyDeleteDirectory() throws Exception { + + // Prepare the source folder with some nested files or sub folders. + Path p_dir = path("/test/hadoop/dir"); + fs.mkdirs(p_dir); + + createFile(path("/test/hadoop/dir/FILE_0")); + createFile(path("/test/hadoop/dir/SDIR_0/FILE_00")); + createFile(path("/test/hadoop/dir/SDIR_0/SDIR_00/FILE_000")); + fs.mkdirs(path("/test/hadoop/dir/SDIR_0/SDIR_00/SDIR_000")); + createFile(path("/test/hadoop/dir/SDIR_0/SDIR_00/file_001")); + fs.mkdirs(path("/test/hadoop/dir/SDIR_0/SDIR_00/sdir_001")); + createFile(path("/test/hadoop/dir/SDIR_0/file_01")); + createFile(path("/test/hadoop/dir/SDIR_0/sdir_01/FILE_010")); + fs.mkdirs(path("/test/hadoop/dir/SDIR_0/sdir_01/SDIR_010")); + createFile(path("/test/hadoop/dir/SDIR_0/sdir_01/file_011")); + fs.mkdirs(path("/test/hadoop/dir/SDIR_0/sdir_01/sdir_011")); + createFile(path("/test/hadoop/dir/file_1")); + createFile(path("/test/hadoop/dir/sdir_1/FILE_10")); + createFile(path("/test/hadoop/dir/sdir_1/SDIR_10/FILE_100")); + fs.mkdirs(path("/test/hadoop/dir/sdir_1/SDIR_10/SDIR_100")); + createFile(path("/test/hadoop/dir/sdir_1/SDIR_10/file_101")); + fs.mkdirs(path("/test/hadoop/dir/sdir_1/SDIR_10/sdir_101")); + createFile(path("/test/hadoop/dir/sdir_1/file_11")); + createFile(path("/test/hadoop/dir/sdir_1/sdir_11/FILE_110")); + fs.mkdirs(path("/test/hadoop/dir/sdir_1/sdir_11/SDIR_110")); + createFile(path("/test/hadoop/dir/sdir_1/sdir_11/file_111")); + fs.mkdirs(path("/test/hadoop/dir/sdir_1/sdir_11/sdir_111")); + + // Assert that all nested files or sub folders under the source + // folder should be existed before delete. + assertTrue("created", fs.exists(path("/test"))); + assertTrue("created", fs.exists(path("/test/hadoop"))); + assertTrue("created", fs.exists(path("/test/hadoop/dir"))); + assertTrue("created", fs.exists(path("/test/hadoop/dir/FILE_0"))); + assertTrue("created", fs.exists(path("/test/hadoop/dir/SDIR_0"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/FILE_00"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/FILE_000"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/SDIR_000"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/file_001"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/sdir_001"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/file_01"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/FILE_010"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/SDIR_010"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/file_011"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/sdir_011"))); + assertTrue("created", fs.exists(path("/test/hadoop/dir/file_1"))); + assertTrue("created", fs.exists(path("/test/hadoop/dir/sdir_1"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/FILE_10"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/FILE_100"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/SDIR_100"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/file_101"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/sdir_101"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/file_11"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/FILE_110"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/SDIR_110"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/file_111"))); + assertTrue("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/sdir_111"))); + + assertTrue(fs.delete(p_dir, true)); + + // Assert that all nested files or sub folders under the source + // folder should be not existed after delete. + assertFalse("created", fs.exists(path("/test/hadoop/dir"))); + assertFalse("created", fs.exists(path("/test/hadoop/dir/FILE_0"))); + assertFalse("created", fs.exists(path("/test/hadoop/dir/SDIR_0"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/FILE_00"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/FILE_000"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/SDIR_000"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/file_001"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/SDIR_00/sdir_001"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/file_01"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/FILE_010"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/SDIR_010"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/file_011"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/SDIR_0/sdir_01/sdir_011"))); + assertFalse("created", fs.exists(path("/test/hadoop/dir/file_1"))); + assertFalse("created", fs.exists(path("/test/hadoop/dir/sdir_1"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/FILE_10"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/FILE_100"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/SDIR_100"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/file_101"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/SDIR_10/sdir_101"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/file_11"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/FILE_110"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/SDIR_110"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/file_111"))); + assertFalse("created", + fs.exists(path("/test/hadoop/dir/sdir_1/sdir_11/sdir_111"))); + assertTrue("created", fs.exists(path("/test"))); + assertTrue("created", fs.exists(path("/test/hadoop"))); + + return; + } + + private long getDigitsNumber(long num) { + long num1 = num - 1; + long len = 0; + while (num1 > 0) { + len++; + num1 = num1 / 10; + } + return (len == 0 ? 1 : len); + } + + private long prepareObjectsTree(String parent, int[] arWidth, + String[] arNamePrefix, boolean isFile) + throws Exception { + // Check the input parameters. + int depth = arWidth.length; + if (arNamePrefix.length != depth) { + throw new Exception("Unmatched depth."); + } + + String digitFormats[] = new String[depth]; + for (int dep = 0; dep < depth; dep++) { + if (arWidth[dep] <= 0) { + throw new Exception("The width of any level must be positive."); + } + digitFormats[dep] = String.format("%%0%dd", + getDigitsNumber(arWidth[dep])); + } + + String p_key = (!parent.endsWith("/") + ? parent + : parent.substring(0, parent.length() - 1)); + + long totalNum = 0; + long leavesNum = 1; + long factors[] = new long[depth]; + for (int dep = 0; dep < depth; dep++) { + if (dep == 0) { + factors[depth - 1] = 1; + } else { + factors[depth - dep - 1] = factors[depth - dep] * arWidth[depth + - dep]; + } + leavesNum = leavesNum * arWidth[dep]; + totalNum += leavesNum; + } + + for (long seq = 0; seq < leavesNum; seq++) { + String seqStr = p_key; + String digitStr = ""; + long seqTmp = seq; + for (int dep = 0; dep < depth; dep++) { + long fac = factors[dep]; + digitStr = digitStr + "_" + String.format(digitFormats[dep], + seqTmp / fac); + seqTmp = seqTmp % fac; + seqStr = seqStr + "/" + arNamePrefix[dep] + digitStr; + } + if (isFile) { + createFile(path(seqStr)); + } else { + fs.mkdirs(path(seqStr)); + } + } + + return totalNum; + } + + private long prepareFilesTree(String parent, int[] arWidth, + String[] arNamePrefix) + throws Exception { + return prepareObjectsTree(parent, arWidth, arNamePrefix, true); + } + + private long prepareFoldersTree(String parent, int[] arWidth, + String[] arNamePrefix) + throws Exception { + return prepareObjectsTree(parent, arWidth, arNamePrefix, false); + } + + private long prepareObjectsTree(String parent, int objNumOfEachPart) + throws Exception { + Path parentPath = path(parent); + fs.mkdirs(parentPath); + + long partsNum = 0; + long totalNum = 0; + + // Prepare files tree. + int[] arWidth1 = {objNumOfEachPart}; + String[] arPrefix1 = {"file"}; + partsNum = prepareFilesTree(parent, arWidth1, arPrefix1); + assertEquals(objNumOfEachPart, partsNum); + totalNum += partsNum; + + int[] arWidth2 = {objNumOfEachPart}; + String[] arPrefix2 = {"FILE"}; + partsNum = prepareFilesTree(parent, arWidth2, arPrefix2); + assertEquals(objNumOfEachPart, partsNum); + totalNum += partsNum; + + int[] arWidth3 = {objNumOfEachPart, 2}; + String[] arPrefix3 = {"sdir", "FILE"}; + partsNum = prepareFilesTree(parent, arWidth3, arPrefix3); + assertEquals(objNumOfEachPart * 3, partsNum); + totalNum += partsNum; + + int[] arWidth4 = {objNumOfEachPart, 2}; + String[] arPrefix4 = {"SDIR", "file"}; + partsNum = prepareFilesTree(parent, arWidth4, arPrefix4); + assertEquals(objNumOfEachPart * 3, partsNum); + totalNum += partsNum; + + return totalNum; + } + + public class ObjectClusterDescription { + boolean isFile; + + String prefix; + + int width; + + String seqFmt; + + public ObjectClusterDescription(boolean isFile, String prefix, + int width) { + this.isFile = isFile; + this.prefix = prefix; + this.width = width; + if (this.width > 0) { + this.seqFmt = String.format("%%0%dd", + getDigitsNumber(this.width)); + } else { + this.seqFmt = null; + } + } + + public void statistic(ObjectTreeInfo info) { + if (this.isFile) { + info.filesNum += this.width; + } else { + info.foldersNum += this.width; + } + } + } + + public class ObjectTreeInfo { + public int filesNum = 0; + + public int foldersNum = 0; + + public int leafFoldersNum = 0; + + public ObjectTreeInfo() { + reset(); + } + + public ObjectTreeInfo(int filesNum, int foldersNum, + int leafFoldersNum) { + this.filesNum = filesNum; + this.foldersNum = foldersNum; + this.leafFoldersNum = leafFoldersNum; + } + + public boolean isEmtpy() { + return ((this.filesNum == 0) && (this.foldersNum == 0)); + } + + public void reset() { + filesNum = 0; + foldersNum = 0; + leafFoldersNum = 0; + } + } + + private ObjectTreeInfo getObjectTreeInfoOfOneDepth( + ObjectClusterDescription[] arDesc) { + ObjectTreeInfo info = new ObjectTreeInfo(); + for (int i = 0; i < arDesc.length; i++) { + if (arDesc[i] == null) { + continue; + } + arDesc[i].statistic(info); + } + return info; + } + + // prepare sub objects of one cluster that described by one object + // cluster description. + private void prepareObjectsOfOneCluster(ObjectClusterDescription desc, + String parent, + boolean atDeepestDepth, + List leaves, + List currFolders) { + if (desc == null) { + return; + } + for (int seq = 0; seq < desc.width; seq++) { + String key = parent + desc.prefix + "_" + String.format(desc.seqFmt, + seq); + if (desc.isFile) { + // file must be leaf. + leaves.add(key); + continue; + } + // folder + key += "/"; + if (atDeepestDepth) { + // folder at deepest depth is leaf. + leaves.add(key); + } + currFolders.add(key); + } + } + + // prepare sub objects of clusters that described by an array of object + // clusters description. + private void prepareSonObjectsOfOneFolder(ObjectClusterDescription[] arDesc, + String parentFolder, + boolean deepest, + List leaves, + List currFolders) { + for (int clusterIdx = 0; clusterIdx < arDesc.length; clusterIdx++) { + ObjectClusterDescription desc = arDesc[clusterIdx]; + prepareObjectsOfOneCluster(desc, parentFolder, deepest, leaves, + currFolders); + } + } + + private void createLeafObjects(List leaves) throws IOException { + for (int i = 0; i < leaves.size(); i++) { + String key = leaves.get(i); + if (key.endsWith("/")) { + // Create leaf folder. + fs.mkdirs(path(key.substring(0, key.length() - 1))); + } else { + // Create file. + createFile(path(key)); + } + } + } + + private ObjectTreeInfo getObjectTreeInfo( + ObjectClusterDescription[][] arDescription) { + int filesNum = 0; + int foldersNum = 0; + int foldersNumAtPreviousDepth = 1; + int depth = arDescription.length; + assertTrue(depth > 0); + + for (int currDepth = 0; currDepth < depth; currDepth++) { + ObjectClusterDescription[] desc = arDescription[currDepth]; + + // Statistic the number of files and folders at current depth of + // a parent folder at previous depth. + ObjectTreeInfo info = getObjectTreeInfoOfOneDepth(desc); + assertTrue(!info.isEmtpy()); + + // Statistic the number of files at current depth of all parent + // folders at previous depth. + info.filesNum *= foldersNumAtPreviousDepth; + filesNum += info.filesNum; + + // Statistic the number of folders at current depth of all parent + // folders at previous depth. + info.foldersNum *= foldersNumAtPreviousDepth; + foldersNum += info.foldersNum; + + // Assert that the depth that has no folder must be the deepest one. + assertTrue((info.foldersNum > 0) || ((currDepth + 1) == depth)); + + // Save the number of folders at current depth and and the + // deepest folders are leaves. + foldersNumAtPreviousDepth = info.foldersNum; + } + + return new ObjectTreeInfo(filesNum, foldersNum, + foldersNumAtPreviousDepth); + } + + public long prepareObjectsTree(String parent, + ObjectClusterDescription[][] arDescription, + List leaves) + throws Exception { + Path parentPath = path(parent); + fs.mkdirs(parentPath); + parent = (parent.endsWith("/") ? parent : (parent + "/")); + + // Statistic the number of sub objects of parent. + ObjectTreeInfo info = getObjectTreeInfo(arDescription); + + // allocate two lists to save folders names at two adjacent depth. + int leavesNum = info.filesNum + info.leafFoldersNum; + List prevFolders = null; + List currFolders = new ArrayList(1); + currFolders.add(parent); + + // prepare sub objects of each depth one by one. + int depth = arDescription.length; + for (int currDepth = 0; currDepth < depth; currDepth++) { + boolean deepest = ((currDepth + 1) == depth); + + // Swap the two lists. + prevFolders = currFolders; + currFolders = new ArrayList(prevFolders.size()); + + // prepare sub objects at current depth. + ObjectClusterDescription[] arDesc = arDescription[currDepth]; + for (int i = 0; i < prevFolders.size(); i++) { + // prepare sub objects of current parent folder. + prepareSonObjectsOfOneFolder(arDesc, prevFolders.get(i), + deepest, leaves, currFolders); + } + + if (currFolders.size() == 0) { + // Current depth has no folder, so it must be deepest one. + break; + } + } + assertEquals(leavesNum, leaves.size()); + + // Create all leaf objects. + createLeafObjects(leaves); + + // Check that each node in each path of leaves should be existed + // after preparation. + assertPathStatus(parent, leaves, true); + + return (info.filesNum + info.foldersNum); + } + + private void assertDepth(FileStatus[] arFileStatus) { + int currDepth = Integer.MAX_VALUE; + for (int i = arFileStatus.length - 1; i >= 0; i--) { + int prevDepth = currDepth; + currDepth = arFileStatus[i].getPath().depth(); + if (currDepth > prevDepth) { + assertTrue(false); + } + } + } + + private void assertObjectStatus(Path f, boolean isExisted) { + try { + fs.getFileStatus(f); + assertTrue(isExisted); + } catch (FileNotFoundException e) { + assertFalse(isExisted); + } catch (IOException e) { + fail(e.toString()); + } + } + + private void assertPathStatus(String parent, String key, + boolean isExisted) { + assertTrue(key.startsWith(parent)); + Path parentPath = path(parent); + Path currPath = path(key); + while (!currPath.equals(parentPath)) { + assertObjectStatus(currPath, isExisted); + currPath = currPath.getParent(); + assertFalse(currPath.isRoot()); + } + } + + private void assertPathStatus(String parent, List leaves, + boolean isExisted) { + for (int i = 0; i < leaves.size(); i++) { + String key = leaves.get(i); + assertPathStatus(parent, key, isExisted); + } + assertObjectStatus(path(parent), isExisted); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetAndSetWorkingDirectory.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetAndSetWorkingDirectory.java new file mode 100644 index 0000000..59afdd3 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetAndSetWorkingDirectory.java @@ -0,0 +1,93 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import static org.junit.Assert.*; + +public class ITestOBSGetAndSetWorkingDirectory { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + @Test + // 设置WorkingDirectory时相对路径转化为绝对路径赋值给当前工作目录workingDir + public void testWorkingDirectory01() throws Exception { + String path = "a001/b001"; + Path workPath = new Path(path); + fs.setWorkingDirectory(workPath); + assertTrue(fs.getWorkingDirectory() + .toString() + .startsWith(fs.getUri().toString())); + } + + @Test + // 设置WorkingDirectory时绝对路径赋值给当前工作目录workingDir + public void testWorkingDirectory02() throws Exception { + String path = fs.getUri().toString() + "/" + "a001/b001"; + Path workPath = new Path(path); + fs.setWorkingDirectory(workPath); + assertEquals(workPath, fs.getWorkingDirectory()); + } + + @Test + // 设置WorkingDirectory时,需检查路径有效性,不包含"."、":"、"//"等不合法的字符 + public void testWorkingDirectory03() throws Exception { + // 实际不满足,需整改 + Path workPath = new Path("a001/./b001"); + boolean hasException = false; + try { + fs.setWorkingDirectory(workPath); + } catch (IllegalArgumentException e) { + hasException = true; + } + assertFalse(hasException); + + try { + workPath = new Path("a:a001"); + hasException = false; + fs.setWorkingDirectory(workPath); + } catch (IllegalArgumentException e) { + hasException = true; + } + assertTrue(hasException); + + workPath = new Path("//a001"); + hasException = false; + try { + fs.setWorkingDirectory(workPath); + } catch (IllegalArgumentException e) { + hasException = true; + } + assertTrue(hasException); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummary.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummary.java new file mode 100644 index 0000000..ef087c3 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummary.java @@ -0,0 +1,367 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.obs.OBSConstants.FAST_UPLOAD_BYTEBUFFER; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.HashMap; +import java.util.Map; + +/** + * OBS tests for configuring block size. + */ +public class ITestOBSGetContentSummary { + private OBSFileSystem fs; + + private Configuration conf; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static final String TEST_PREFIX = "test_du/"; + + private static final byte[] dataSet = ContractTestUtils.dataset(16, 0, 10); + + private static final Logger LOG = + LoggerFactory.getLogger(ITestOBSGetContentSummary.class); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + } + + private void init(boolean contentEnable, boolean fsDFSListEnable) + throws IOException { + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.setBoolean(OBSConstants.OBS_CONTENT_SUMMARY_ENABLE, contentEnable); + conf.setBoolean(OBSConstants.OBS_CLIENT_DFS_LIST_ENABLE, fsDFSListEnable); + conf.set(OBSConstants.FAST_UPLOAD_BUFFER, FAST_UPLOAD_BYTEBUFFER); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String testPath) { + return new Path(testRootPath + "/" + testPath); + } + + @Test + // 计算对象桶一级目录的ContentSummary,校验正确性 + public void testObjectBucketOnePath() throws Exception { + testGetContentSummaryOnePath(TEST_PREFIX, false); + } + + @Test + // 计算对象桶子目录的ContentSummary,校验正确性 + public void testObjectBucketSubPath() throws Exception { + testGetContentSummarySubPath(TEST_PREFIX, false); + } + + @Test + // 计算文件桶一级目录的ContentSummary,校验正确性 + public void testPosixBucketOnePath() throws Exception { + testGetContentSummaryOnePath(TEST_PREFIX, true); + } + + @Test + // 计算文件桶子目录的ContentSummary,校验正确性 + public void testPosixBucketSubPath() throws Exception { + testGetContentSummarySubPath(TEST_PREFIX, true); + } + + @Test + // 测试空文件的ContentSummary,DirectoryCount为0,FileCount为1,Length为0 + public void testGetContentSummaryOfFile01() throws Exception { + init(true, true); + Path testFile = getTestPath("test_file"); + fs.delete(testFile, true); + + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + + ContentSummary summary = fs.getContentSummary(testFile); + assertTrue(summary.getDirectoryCount() == 0); + assertTrue(summary.getFileCount() == 1); + assertTrue(summary.getLength() == 0); + fs.delete(testFile, true); + } + + @Test + // 测试非空文件的ContentSummary,DirectoryCount为0,FileCount为1,Length为文件长度 + public void testGetContentSummaryOfFile02() throws Exception { + init(true, true); + Path testFile = getTestPath("test_file"); + fs.delete(testFile, true); + + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {1, 2, 3, 4, 5}; + outputStream.write(data); + outputStream.close(); + + ContentSummary summary = fs.getContentSummary(testFile); + assertTrue(summary.getDirectoryCount() == 0); + assertTrue(summary.getFileCount() == 1); + assertTrue(summary.getLength() == data.length); + fs.delete(testFile, true); + } + + @Test + // 路径不存在,抛FileNotFoundException + public void testGetContentSummaryAbnormal01() throws Exception { + init(true, true); + Path testFile = getTestPath("test_file"); + fs.delete(testFile, true); + + boolean hasException = false; + try { + fs.getContentSummary(testFile); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 路径的父目录及上级目录不存在,抛FileNotFoundException + public void testGetContentSummaryAbnormal02() throws Exception { + init(true, true); + Path testFile = getTestPath("a001/b001/test_file"); + fs.delete(testFile.getParent().getParent(), true); + + assertFalse(fs.exists(testFile.getParent())); + + boolean hasException = false; + try { + fs.getContentSummary(testFile); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 路径的父目录及上级目录非目录,抛AccessControlException + public void testGetContentSummaryAbnormal03() throws Exception { + init(true, true); + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + fs.delete(testFile.getParent().getParent(), true); + + FSDataOutputStream outputStream = fs.create(testFile.getParent(), + false); + outputStream.close(); + + boolean hasException = false; + try { + fs.getContentSummary(testFile); + } catch (AccessControlException e) { + hasException = true; + } + assertTrue(hasException); + } + + private void constructTestPaths(String testDir, int dirDeep, int subDirNum, + int subFileNum) throws Exception { + if (dirDeep <= 0) { + return; + } + if (!testDir.endsWith("/")) { + testDir = testDir + "/"; + } + boolean rslt = fs.mkdirs(getTestPath(testDir)); + Assert.assertTrue(rslt); + + for (int i = 0; i < subFileNum; i++) { + FSDataOutputStream outputStream = fs.create( + getTestPath(testDir + "-subFile-" + i), true); + outputStream.write(dataSet); + outputStream.close(); + } + + for (int i = 0; i < subDirNum; i++) { + constructTestPaths(testDir + "-" + dirDeep + "-" + i + "/", + dirDeep - 1, subDirNum, subFileNum); + } + } + + private void testGetContentSummaryOnePath(String prefix, + boolean isPosixTest) throws Exception { + // test primitive + ContentSummary primitive, optimization, optimizationWithDFS = null; + long delay1, delay2, delay3; + try { + init(false, false); + if (isPosixTest != fs.isFsBucket()) { + return; + } + fs.delete(getTestPath(prefix), true); + constructTestPaths(prefix, 3, 5, 5); + long startTime = System.nanoTime(); + primitive = fs.getContentSummary(getTestPath(prefix)); + delay1 = System.nanoTime() - startTime; + } finally { + fs.close(); + fs = null; + } + + try { + init(true, false); + long startTime = System.nanoTime(); + optimization = fs.getContentSummary(getTestPath(prefix)); + delay2 = System.nanoTime() - startTime; + } finally { + fs.close(); + fs = null; + } + + // test optimization + try { + init(true, true); + long startTime = System.nanoTime(); + optimizationWithDFS = fs.getContentSummary(getTestPath(prefix)); + delay3 = System.nanoTime() - startTime; + assertTrue(fs.delete(getTestPath(prefix), true)); + } finally { + if (fs != null) { + fs.close(); + fs = null; + } + } + + LOG.info("delay (ns): {} vs. {} vs. {}", delay1, delay2, delay3); + + // compare + assertTrue(compareContentSummary(primitive, optimization)); + assertTrue(compareContentSummary(primitive, optimizationWithDFS)); + } + + private void testGetContentSummarySubPath(String prefix, + boolean isPosixTest) throws Exception { + // test primitive + ContentSummary primitive = null; + Map summarys = new HashMap<>(); + try { + init(false, false); + if (isPosixTest != fs.isFsBucket()) { + return; + } + fs.delete(getTestPath(prefix), true); + constructTestPaths(prefix, 3, 5, 5); + FileStatus[] status = fs.listStatus(getTestPath(prefix)); + long length = 0; + long fileCount = 0; + long dirCount = 0; + for (FileStatus s : status) { + ContentSummary summary = fs.getContentSummary(s.getPath()); + summarys.put(s.getPath().toString(), summary); + length += summary.getLength(); + fileCount += summary.getFileCount(); + dirCount += summary.getDirectoryCount(); + } + primitive = new ContentSummary(length, fileCount, dirCount); + } finally { + if (fs != null) { + fs.close(); + fs = null; + } + } + + // test optimization + ContentSummary optimization = null; + Map optSummarys = new HashMap<>(); + try { + init(true, true); + FileStatus[] status = fs.listStatus(getTestPath(prefix)); + long length = 0; + long fileCount = 0; + long dirCount = 0; + for (FileStatus s : status) { + ContentSummary summary = fs.getContentSummary(s.getPath()); + optSummarys.put(s.getPath().toString(), summary); + length += summary.getLength(); + fileCount += summary.getFileCount(); + dirCount += summary.getDirectoryCount(); + } + optimization = new ContentSummary(length, fileCount, dirCount); + assertTrue(fs.delete(getTestPath(prefix), true)); + } finally { + if (fs != null) { + fs.close(); + fs = null; + } + } + + // compare + boolean isSameResult = compareContentSummary(primitive, optimization); + assertTrue(isSameResult); + } + + private boolean compareContentSummary(ContentSummary s1, + ContentSummary s2) { + LOG.info(String.format("s1{len[%s], fileCount[%s], dirCount[%s]", + s1.getLength(), s1.getFileCount(), s1.getDirectoryCount())); + + LOG.info(String.format("s2{len[%s], fileCount[%s], dirCount[%s]", + s2.getLength(), s2.getFileCount(), s2.getDirectoryCount())); + + if (s1.getLength() != s2.getLength() + || s1.getDirectoryCount() != s2.getDirectoryCount() + || s1.getFileCount() != s2.getFileCount()) { + return false; + } + return true; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummaryV2.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummaryV2.java new file mode 100644 index 0000000..f19c6cd --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetContentSummaryV2.java @@ -0,0 +1,518 @@ +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.obs.OBSTestConstants.TEST_FS_OBS_NAME; + +import com.obs.services.IObsCredentialsProvider; +import com.obs.services.ObsConfiguration; +import com.obs.services.exception.ObsException; +import com.obs.services.model.fs.ContentSummaryFsRequest; +import com.obs.services.model.fs.ContentSummaryFsResult; +import com.obs.services.model.fs.DirContentSummary; +import com.obs.services.model.fs.DirSummary; +import com.obs.services.model.fs.ListContentSummaryFsRequest; +import com.obs.services.model.fs.ListContentSummaryFsResult; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.obs.mock.MockObsClient; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.lang.reflect.Field; +import java.net.URI; +import java.net.URISyntaxException; +import java.util.ArrayList; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; +import java.util.function.Function; + +public class ITestOBSGetContentSummaryV2 { + + private static boolean failure; + + private static final Logger LOG = LoggerFactory.getLogger(ITestOBSGetContentSummaryV2.class); + + private OBSFileSystem fs; + + private Configuration conf; + + private MockObsClient mockObsClient; + + private OBSFileSystem ofs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private boolean useMock = true; + + @Before + public void setUp() throws IOException, URISyntaxException, NoSuchFieldException, IllegalAccessException { + if (failure) return; + conf = OBSContract.getConfiguration(null); + conf.setInt(OBSConstants.CORE_LIST_THREADS, 100); + conf.setInt(OBSConstants.MAX_LIST_THREADS, 100); + conf.setInt(OBSConstants.RETRY_MAXTIME, 5000); + fs = OBSTestUtils.createTestFileSystem(conf); + ofs = OBSTestUtils.createTestFileSystem(conf); + if (useMock) { + mock(); + } + } + + private void mock() throws NoSuchFieldException, IOException, IllegalAccessException, URISyntaxException { + // create mock client + String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, ""); + URI uri = new URI(fsname); + ObsConfiguration obsConf = new ObsConfiguration(); + String endPoint = conf.getTrimmed(OBSConstants.ENDPOINT, ""); + obsConf.setEndPoint(endPoint); + IObsCredentialsProvider securityProvider = OBSSecurityProviderUtil.createObsSecurityProvider(conf, uri); + mockObsClient = new MockObsClient(securityProvider, obsConf); + + // mock + Field obsFiled = OBSFileSystem.class.getDeclaredField("obs"); + obsFiled.setAccessible(true); + obsFiled.set(this.fs, mockObsClient); + } + + @After + public void cleanUp() throws IOException { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private void createFile(Path path, long fileSize) throws IOException { + StringBuilder stringBuilder = new StringBuilder(); + for (long i = 0; i < fileSize; i ++) { + stringBuilder.append((char)1); + } + FSDataOutputStream outputStream = fs.create(path); + outputStream.write(stringBuilder.toString().getBytes()); + outputStream.close(); + } + + private void createFiles(String basePath, int fileNum, int parDirLevel, long fileSize) throws IOException { + if (parDirLevel < 1) { + throw new IllegalArgumentException("parDirLevel should >= 1"); + } + for (int i = 0; i < fileNum; i ++) { + String path = String.format("%s/sub-dir%d", basePath, i); + for (int j = 0; j < parDirLevel - 1; j ++) { + path = String.format("%s/sub-dir-level%d", path, j); + } + path = path + "/file"; + createFile(new Path(path), fileSize); + } + } + + @Test + public void testFsContentSummaryV2_1() throws IOException { + fs.mkdirs(new Path(testRootPath)); + FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath)); + if (!useMock) { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 0 + 1 && + contentSummary.getFileCount() == 0 && + contentSummary.getLength() == 0); + } + + @Test + public void testFsContentSummaryV2_2() throws IOException { + Path path = new Path(testRootPath + "/dir1"); + fs.mkdirs(path); + FileStatus fileStatus = fs.getFileStatus(path); + if (!useMock) { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 0 + 1 && + contentSummary.getFileCount() == 0 && + contentSummary.getLength() == 0); + } + + @Test + public void testFsContentSummaryV2_3() throws IOException { + Path path = new Path(testRootPath + "/dir1/dir2"); + fs.mkdirs(path); + FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1")); + if (!useMock) { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 1 + 1 && + contentSummary.getFileCount() == 0 && + contentSummary.getLength() == 0); + } + + @Test + public void testFsContentSummaryV2_4() throws IOException { + Path path = new Path(testRootPath + "/dir1/dir2"); + fs.mkdirs(path); + FileStatus fileStatus = fs.getFileStatus(path); + if (!useMock) { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 0 + 1 && + contentSummary.getFileCount() == 0 && + contentSummary.getLength() == 0); + } + + @Test + public void testFsContentSummaryV2_5() throws IOException { + Path path = new Path(testRootPath + "/dir1/dir2/file1"); + createFile(path, 1); + FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1")); + if (!useMock) { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 1 + 1 && + contentSummary.getFileCount() == 1 && + contentSummary.getLength() == 1); + } + + @Test + public void testFsContentSummaryV2_6() throws IOException { + Path path = new Path(testRootPath + "/dir1/dir2/file1"); + createFile(path, 1); + FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1/dir2")); + if (!useMock) { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 0 + 1 && + contentSummary.getFileCount() == 1 && + contentSummary.getLength() == 1); + } + + @Test + public void testFsContentSummaryV2_7() throws IOException { + createFile(new Path(testRootPath + "/dir1/file1"), 1); + createFile(new Path(testRootPath + "/dir1/dir2/file2"), 2); + FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1")); + if (!useMock) { + try { + Thread.sleep(30 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 1 + 1 && + contentSummary.getFileCount() == 2 && + contentSummary.getLength() == 3); + } + + private void testFsContentSummaryV2MultiDirsAndFiles(int fileNum, int parDirLevel, + Function listCSResultCallback) throws IOException { + long fileSize = 3; + createFiles(testRootPath + "/dir1", fileNum, parDirLevel, fileSize); + FileStatus fileStatus = fs.getFileStatus(new Path(testRootPath + "/dir1")); + if (!useMock) { + try { + Thread.sleep(300 * 1000); + } catch (InterruptedException e) { + throw new RuntimeException(e); + } + } + this.mockObsClient.setListCSResultCallback(listCSResultCallback); + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + LOG.info("V2: dirNum: {}, fileNum: {}, fileSize: {}, should be: ({}, {}, {})", + contentSummary.getDirectoryCount(), contentSummary.getFileCount(), contentSummary.getLength(), + fileNum * parDirLevel + 1, fileNum, fileNum * fileSize); + boolean pass = contentSummary.getDirectoryCount() == (long) fileNum * parDirLevel + 1 && + contentSummary.getFileCount() == fileNum && + contentSummary.getLength() == fileNum * fileSize; + if (!pass) { + failure = true; + } + + assertTrue("contentSummary correctness", pass); + + // compare with v1 + ContentSummary contentSummary1 = OBSPosixBucketUtils.fsGetDirectoryContentSummary(fs, + OBSCommonUtils.pathToKey(fs, fileStatus.getPath())); + LOG.info("V1: dirNum: {}, fileNum: {}, fileSize: {}", + contentSummary1.getDirectoryCount(), contentSummary1.getFileCount(), contentSummary1.getLength()); + assertTrue("v2 should align to v1", + contentSummary.getDirectoryCount() == contentSummary1.getDirectoryCount() && + contentSummary.getFileCount() == contentSummary1.getFileCount() && + contentSummary.getLength() == contentSummary1.getLength()); + } + + @Test + public void testFsContentSummaryV2_8() throws IOException { + testFsContentSummaryV2MultiDirsAndFiles(1100, 1, null); + } + + @Test + public void testFsContentSummaryV2_9() throws IOException { + testFsContentSummaryV2MultiDirsAndFiles(900, 1, null); + } + + @Test + public void testFsContentSummaryV2_10() throws IOException { + testFsContentSummaryV2MultiDirsAndFiles(1000, 1, null); + } + + @Test + public void testFsContentSummaryV2_11() throws IOException { + testFsContentSummaryV2MultiDirsAndFiles(1100, 2, null); + } + + @Test + public void testFsContentSummaryV2_12() throws IOException { + testFsContentSummaryV2MultiDirsAndFiles(900, 2, null); + } + + @Test + public void testFsContentSummaryV2_13() throws IOException { + testFsContentSummaryV2MultiDirsAndFiles(1000, 2, null); + } + + @Test + public void testFsContentSummaryV2_14() throws IOException { + Path root = new Path("/"); + FileStatus fileStatus = fs.getFileStatus(root); + try { + OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(fs, fileStatus); + } catch (Exception e) { + assertTrue(e.getMessage(), false); + } + assertTrue(true); + } + + private static class ErrorPattern1 { + private volatile AtomicInteger count; + + private int[] errorPoint; + + private int errorDirNum; + + public ErrorPattern1(int[] errorPoint, int errorDirNum) { + this.errorPoint = errorPoint; + this.errorDirNum = errorDirNum; + this.count = new AtomicInteger(); + } + + public ListContentSummaryFsResult callbackFunc(ListContentSummaryFsResult listCSResult) { + this.count.incrementAndGet(); + boolean needInject = false; + for (int i : errorPoint) { + if (this.count.get() == i) { + needInject = true; + break; + } + } + if (!needInject) { + return listCSResult; + } + List retErrors = new ArrayList<>(); + List retDirs = new ArrayList<>(); + List dirs = listCSResult.getDirContentSummaries(); + int nNum = Math.min(dirs.size(), errorDirNum); + for (int i = 0; i < dirs.size() - nNum; i ++) { + retDirs.add(dirs.get(i)); + } + for (int i = dirs.size() - nNum; i < dirs.size(); i ++) { + DirContentSummary dir = dirs.get(i); + ListContentSummaryFsResult.ErrorResult err = new ListContentSummaryFsResult.ErrorResult(); + err.setStatusCode("500"); + err.setErrorCode("TestErrorCode"); + err.setKey(dir.getKey()); + err.setInode(dir.getInode()); + err.setMessage("InjectError"); + retErrors.add(err); + } + listCSResult.setDirContentSummaries(retDirs); + listCSResult.setErrorResults(retErrors); + return listCSResult; + } + } + + @Test + public void testFsContentSummaryV2_15() throws IOException { + if (!useMock) { + return; + } + ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3); + testFsContentSummaryV2MultiDirsAndFiles(1100, 1, + ep::callbackFunc); + } + + @Test + public void testFsContentSummaryV2_16() throws IOException { + if (!useMock) { + return; + } + ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3); + testFsContentSummaryV2MultiDirsAndFiles(900, 1, + ep::callbackFunc); + } + + @Test + public void testFsContentSummaryV2_17() throws IOException { + if (!useMock) { + return; + } + ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3); + testFsContentSummaryV2MultiDirsAndFiles(1000, 1, + ep::callbackFunc); + } + + @Test + public void testFsContentSummaryV2_18() throws IOException { + if (!useMock) { + return; + } + ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3); + testFsContentSummaryV2MultiDirsAndFiles(1100, 2, + ep::callbackFunc); + } + + @Test + public void testFsContentSummaryV2_19() throws IOException { + if (!useMock) { + return; + } + ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3); + testFsContentSummaryV2MultiDirsAndFiles(900, 2, + ep::callbackFunc); + } + + @Test + public void testFsContentSummaryV2_20() throws IOException { + if (!useMock) { + return; + } + ErrorPattern1 ep = new ErrorPattern1(new int[]{1, 3, 5}, 3); + testFsContentSummaryV2MultiDirsAndFiles(1000, 2, + ep::callbackFunc); + } + + @Test + public void test405FallbackWithoutRetry() throws IOException { + if (useMock) { + mockObsClient.setGetCSUnsupported(true); + mockObsClient.setGetCSNum(0); + mockObsClient.setResponseCode(405); + mockObsClient.setErrorMsg("mock unsupported"); + createFile(new Path(testRootPath + "/dir1/file1"), 1); + createFile(new Path(testRootPath + "/dir1/dir2/file2"), 2); + Path path = new Path(testRootPath + "/dir1"); + ContentSummary contentSummary = fs.getContentSummary(path); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 1 + 1 && + contentSummary.getFileCount() == 2 && + contentSummary.getLength() == 3); + assertEquals("escape without retry", 1, mockObsClient.getGetCSNum()); + mockObsClient.setGetCSUnsupported(false); + mockObsClient.setGetCSNum(0); + } + } + + @Test + public void test503FallbackShouldRetry() throws IOException { + if (useMock) { + mockObsClient.setGetCSUnsupported(true); + mockObsClient.setGetCSNum(0); + mockObsClient.setResponseCode(503); + mockObsClient.setErrorMsg("mock service unavailable"); + createFile(new Path(testRootPath + "/dir1/file1"), 1); + createFile(new Path(testRootPath + "/dir1/dir2/file2"), 2); + Path path = new Path(testRootPath + "/dir1"); + ContentSummary contentSummary = fs.getContentSummary(path); + assertTrue("contentSummary correctness", + contentSummary.getDirectoryCount() == 1 + 1 && + contentSummary.getFileCount() == 2 && + contentSummary.getLength() == 3); + assertTrue("escape with retry", mockObsClient.getGetCSNum() > 1); + mockObsClient.setGetCSUnsupported(false); + mockObsClient.setGetCSNum(0); + } + } + + // @Test + public void testFsContentSummaryV2() throws IOException { + FileStatus fileStatus = ofs.getFileStatus(new Path("/")); + ContentSummary contentSummary = OBSPosixBucketUtils.fsGetDirectoryContentSummaryV2(ofs, fileStatus); + long dirNum = contentSummary.getDirectoryCount(); + long fileNum = contentSummary.getFileCount(); + long fileSize = contentSummary.getLength(); + System.out.printf("%d %d %d", dirNum, fileNum, fileSize); + } + + // @Test + public void testSDK1() { + List dirs = new ArrayList<>(); + ListContentSummaryFsRequest.DirLayer dir = new ListContentSummaryFsRequest.DirLayer(); + dir.setKey("test/dir1/"); + // dir.setInode(4611752345613959168L); + dirs.add(dir); + + ListContentSummaryFsRequest req = new ListContentSummaryFsRequest(); + req.setBucketName("sanfangliantiao-0218"); + req.setMaxKeys(1000); + req.setDirLayers(dirs); + + ListContentSummaryFsResult res = ofs.getObsClient().listContentSummaryFs(req); + System.out.println(res); + } + + // @Test + public void testSDK2() { + ContentSummaryFsRequest req = new ContentSummaryFsRequest(); + req.setBucketName("jianantest02"); + req.setDirName("test/dir1/sub-dir908/"); + + ContentSummaryFsResult res = ofs.getObsClient().getContentSummaryFs(req); + DirSummary summary = res.getContentSummary(); + System.out.printf("%d %d %d", summary.getDirCount(), summary.getFileCount(), summary.getFileSize()); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetFileStatusAndExist.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetFileStatusAndExist.java new file mode 100644 index 0000000..302b26f --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSGetFileStatusAndExist.java @@ -0,0 +1,164 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import static org.junit.Assert.*; + +public class ITestOBSGetFileStatusAndExist { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 获取根目录的信息,exist返回true + public void testGetFileStatusAndExistNormal001() throws IOException { + FileStatus status = fs.getFileStatus(new Path("/")); + System.out.println(status.toString()); + assertTrue("Root path should be directory.", status.isDirectory()); + + assertEquals(fs.exists(new Path("/")), status != null); + } + + @Test + // 获取文件的元数据信息,exist返回true + public void testGetFileStatusAndExistNormal002() throws IOException { + Path testPath = getTestPath("a001/test_file"); + FSDataOutputStream outputStream = null; + try { + outputStream = fs.create(testPath, false); + byte[] data = ContractTestUtils.dataset(8, 'a', 26); + outputStream.write(data); + outputStream.close(); + + FileStatus status = fs.getFileStatus(testPath); + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc.getMr().getKind()); + } + System.out.println(status.toString()); + assertTrue("test path should be a file.", status.isFile()); + assertEquals(fs.exists(testPath), status != null); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + OBSFSTestUtil.deletePathRecursive(fs, testPath); + } + + @Test + // 获取目录的元数据信息, exist返回true + public void testGetFileStatusAndExistNormal003() throws IOException { + Path testPath = getTestPath("a001/b001"); + fs.mkdirs(testPath); + + FileStatus status = fs.getFileStatus(testPath); + System.out.println(status.toString()); + assertTrue("test path should be directory.", status.isDirectory()); + assertEquals(fs.exists(testPath), status != null); + OBSFSTestUtil.deletePathRecursive(fs, testPath); + } + + @Test + // 路径不存在,getFileStatus抛FileNotFoundException, exist返回false + public void testGetFileStatusAndExistAbnormal001() throws IOException { + Path testPath = getTestPath("a001/test_file"); + if (fs.exists(testPath)) { + fs.delete(testPath, true); + } + + boolean hasException = false; + try { + fs.getFileStatus(testPath); + } catch (FileNotFoundException fnfe) { + hasException = true; + } + assertTrue(hasException); + assertFalse(fs.exists(testPath)); + } + + @Test + // 路径的父目录及上级目录不存在,getFileStatus抛FileNotFoundException, exist返回false + public void testGetFileStatusAndExistAbnormal002() throws IOException { + Path testPath = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testPath.getParent()); + + boolean hasException = false; + try { + fs.getFileStatus(testPath); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + assertFalse(fs.exists(testPath)); + } + + @Test + // 路径的父目录及上级目录非目录,getFileStatus抛FileNotFoundException,exist返false + public void testGetFileStatusAndExistAbnormal003() throws IOException { + Path testPath = getTestPath("a001/b001/test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testPath.getParent()); + + FSDataOutputStream outputStream = fs.create(testPath.getParent(), + false); + outputStream.close(); + + boolean hasException = false; + try { + fs.getFileStatus(testPath); + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc.getMr().getKind()); + } + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + assertFalse(fs.exists(testPath)); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapper.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapper.java new file mode 100644 index 0000000..6d1d6f9 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapper.java @@ -0,0 +1,79 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; + +public class ITestOBSHDFSWrapper { + + private Configuration conf; + private MiniDFSCluster dfsCluster; + + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Before + public void setUp() throws Exception { + conf = new Configuration(); + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + } + + @After + public void tearDown() throws Exception { + if (dfsCluster != null) { + dfsCluster.close(); + } + } + + @Test + public void testStatus() throws IOException { + String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + // hdfs native file + DistributedFileSystem rawHdfsFS = dfsCluster.getFileSystem(); + rawHdfsFS.create(new Path("/hdfsfile1")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + // If not set abfs, will still use Hdfs to resole path which will cause FileNotFoundException on hdfs. + Path wrapperFile1 = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"); + FileContext fc = FileContext.getFileContext(wrapperFile1.toUri(), conf); + try { + fc.rename(wrapperFile1, new Path("hdfs://" + fsAuthority + "/wrapper/newFile1_r1")); + } catch (Exception e) { + Assert.assertTrue(e instanceof FileNotFoundException); + } + + // set abfs to MRSHDFSWrapper impl hdfs. + conf.set("fs.AbstractFileSystem.hdfs.impl", "org.apache.hadoop.fs.obs.OBSHDFSWrapper"); + Path wrapperFile1New = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"); + Path newPath = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1_r1"); + FileContext fcNew = FileContext.getFileContext(wrapperFile1New.toUri(), conf); + fcNew.rename(wrapperFile1, newPath); + fcNew.open(newPath).close(); + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapperFileSystem.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapperFileSystem.java new file mode 100644 index 0000000..b2ed19d --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHDFSWrapperFileSystem.java @@ -0,0 +1,986 @@ +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.junit.Assert.fail; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FSMainOperationsBaseTest; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Options; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.TrashPolicy; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +import java.io.File; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.lang.reflect.InvocationTargetException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.HashSet; +import java.util.List; +import java.util.Random; +import java.util.Set; +import java.util.stream.Collectors; + +public class ITestOBSHDFSWrapperFileSystem extends FSMainOperationsBaseTest { + + static Configuration conf = OBSContract.getConfiguration(null); + static String fsAuthority; + static MiniDFSCluster dfsCluster; + + @BeforeClass + public static void setUpper() throws Exception { + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.setLong("dfs.namenode.fs-limits.min-block-size",1000); + dfsCluster = + new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + } + + @Override + protected FileSystem createFileSystem() throws Exception { + + // conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + // conf.setLong("dfs.namenode.fs-limits.min-block-size",1000); + // dfsCluster = + // new MiniDFSCluster.Builder(conf).numDataNodes(1).build(); + // fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + // conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./test/", + // folder.getRoot().toURI().toString()); + + // // hdfs native file + DistributedFileSystem fSys = dfsCluster.getFileSystem(); + String testRootDir = getTestRootPath(fSys).toUri().getPath(); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link."+testRootDir+ + "/test/", conf.get(OBSTestConstants.TEST_FS_OBS_NAME) + "/test/"); + return fSys; + } + + + @AfterClass + public static void tearDowner() { + if (dfsCluster != null) { + dfsCluster.close(); + } + } + + @Rule + public TemporaryFolder folder = new TemporaryFolder(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Test + public void testTrash() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + // + // // hdfs native file + // DistributedFileSystem fSys = fSys; + fSys.create(new Path("/hdfsfile1")).close(); + fSys.mkdirs(new Path("/hdfsdir"), FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + + + conf.set("fs.trash.interval", "30"); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/", + folder.getRoot().toURI() + "/folder1"); + conf.set("fs.hdfs.impl.disable.cache", "true"); + //file:C:\Users\L30002~1\AppData\Local\Temp\junit1625000454693617450/folder1 + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + TrashPolicy trashPolicy = TrashPolicy.getInstance(conf, fs); + + // trash wrapper file + { + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"))); + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/abc/subFile1"))); + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsfile1"))); + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsdir/subfile1"))); + } + + } + + // trash target folder is wrapped to another filesystem + @Test + public void testTrashUser() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + // + // // hdfs native file + // DistributedFileSystem fSys = fSys; + fSys.create(new Path("/hdfsfile1")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.trash.interval", "30"); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./user/", + folder.getRoot().toURI().toString() + "/folder1"); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + TrashPolicy trashPolicy = TrashPolicy.getInstance(conf, fs); + Path hdfsTrashRoot = fs.getTrashRoot(new Path("hdfs://" + fsAuthority + "/hdfsfile1")); + Path localFSTrashRoot = fs.getTrashRoot(new Path("hdfs://" + fsAuthority + "/wrapper/newFile1")); + fs.delete(localFSTrashRoot, true); + fs.delete(hdfsTrashRoot, true); + + // trash wrapper file + { + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"))); + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/user/subFile1"))); + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsfile1"))); + Assert.assertTrue(trashPolicy.moveToTrash(new Path("hdfs://" + fsAuthority + "/hdfsdir/subfile1"))); + } + // list trash root + { + RemoteIterator hdfsTrashItr = fs.listFiles(hdfsTrashRoot, true); + int countHdfs = 0; + while (hdfsTrashItr.hasNext()) { + System.out.println(hdfsTrashItr.next()); + countHdfs++; + } + Assert.assertEquals(2, countHdfs); + + RemoteIterator localTrashItr = fs.listFiles(localFSTrashRoot, true); + int countLocal = 0; + while (localTrashItr.hasNext()) { + System.out.println(localTrashItr.next()); + countLocal++; + } + Assert.assertEquals(2, countLocal); + } + // } + } + + @Test + public void testRename() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + // + // // hdfs native file + // DistributedFileSystem fSys = fSys; + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + File subfolder = folder.newFolder("folder1"); + System.out.println("---"+subfolder.toPath()); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/", + folder.getRoot().toURI().toString() + "/folder1"); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + + // cross fs rename + { + Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1"); + Path to = new Path("hdfs://" + fsAuthority + "/wrapper/hdfsfile1"); + try { + fs.rename(from, to); + fail("cross fs rename should fail."); + } catch (IOException e) { + // 20210729: [BUG2021070600630] change UnsupportedOperationException to IOException + if (!(e.getCause() instanceof UnsupportedOperationException)) { + throw e; + } + // ignore UnsupportedOperationException cause + } + } + + // same fs rename in hdfs + { + Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1"); + Path to = new Path("hdfs://" + fsAuthority + "/hdfsfile1"); + Assert.assertTrue(fs.rename(from, to)); + Assert.assertTrue(fSys.exists(to)); + } + + // same fs rename in wrapper fs + { + Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"); + Path to = new Path("hdfs://" + fsAuthority + "/wrapper/newfile11"); + Assert.assertTrue(fs.rename(from, to)); + Assert.assertTrue(folder.getRoot().toPath().resolve("newfile11").toFile().exists()); + } + + // same fs rename in wrapper fs to different mount point + { + Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile2"); + Path to = new Path("hdfs://" + fsAuthority + "/abc/newFile2"); + Assert.assertTrue(fs.rename(from, to)); + Assert.assertTrue(folder.getRoot().toPath().resolve("folder1/newFile2").toFile().exists()); + } + // } + } + + @Test + public void testRename1() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + // + // // hdfs native file + // DistributedFileSystem fSys = fSys; + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + folder.newFile("newFile4"); + folder.newFile("newFile5"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + subfolder.toPath().resolve("newFile5").toFile().createNewFile(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/", + folder.getRoot().toURI().toString() + "/folder1"); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + + // cross fs rename + { + Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1"); + Path to = new Path("hdfs://" + fsAuthority + "/wrapper/hdfsfile1"); + try { + if (fs instanceof OBSHDFSFileSystem) { + ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE); + } else { + fs.delete(to, false); + fs.rename(from, to); + } + fail("cross fs rename should fail."); + } catch (IOException e) { + // 20210729: [BUG2021070600630] change UnsupportedOperationException to IOException + if (!(e.getCause() instanceof UnsupportedOperationException)) { + throw e; + } + // ignore UnsupportedOperationException cause + } + } + + // same fs rename in hdfs + { + Path from = new Path("hdfs://" + fsAuthority + "/hdfsfile1"); + Path to = new Path("hdfs://" + fsAuthority + "/hdfsfile11"); + boolean success = false; + if (fs instanceof OBSHDFSFileSystem) { + ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE); + success = true; + } else { + fs.delete(to, false); + success = fs.rename(from, to); + } + Assert.assertTrue(success); + Assert.assertTrue(fSys.exists(to)); + } + + // same fs rename in wrapper fs + { + Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"); + Path to = new Path("hdfs://" + fsAuthority + "/wrapper/newfile11"); + boolean success = false; + if (fs instanceof OBSHDFSFileSystem) { + ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE); + success = true; + } else { + fs.delete(to, false); + success = fs.rename(from, to); + } + Assert.assertTrue(success); + Assert.assertTrue(folder.getRoot().toPath().resolve("newfile11").toFile().exists()); + } + + // same fs rename in wrapper fs, to path is exist + { + Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile4"); + Path to = new Path("hdfs://" + fsAuthority + "/wrapper/newFile4"); + boolean success = false; + if (fs instanceof OBSHDFSFileSystem) { + ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE); + success = true; + } else { + fs.delete(to, false); + success = fs.rename(from, to); + } + Assert.assertTrue(success); + Assert.assertTrue(folder.getRoot().toPath().resolve("newFile4").toFile().exists()); + } + + // same fs rename in wrapper fs to different mount point + { + Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile2"); + Path to = new Path("hdfs://" + fsAuthority + "/abc/newFile2"); + boolean success = false; + if (fs instanceof OBSHDFSFileSystem) { + ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE); + success = true; + } else { + fs.delete(to, false); + success = fs.rename(from, to); + } + Assert.assertTrue(success); + Assert.assertTrue(folder.getRoot().toPath().resolve("folder1/newFile2").toFile().exists()); + } + + // same fs rename in wrapper fs to different mount point, to path is exist + { + Path from = new Path("hdfs://" + fsAuthority + "/wrapper/newFile5"); + Path to = new Path("hdfs://" + fsAuthority + "/abc/newFile5"); + boolean success = false; + if (fs instanceof OBSHDFSFileSystem) { + ((OBSHDFSFileSystem) fs).rename(from, to, Options.Rename.OVERWRITE); + success = true; + } else { + fs.delete(to, false); + success = fs.rename(from, to); + } + Assert.assertTrue(success); + Assert.assertTrue(folder.getRoot().toPath().resolve("folder1/newFile5").toFile().exists()); + } + + + // } + } + + @Test + public void testDelete() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + + File dir = folder.newFolder("toBeDelete"); + dir.toPath().resolve("subFile1").toFile().createNewFile(); + dir.toPath().resolve("subFile2").toFile().createNewFile(); + + Path deletePath = new Path("hdfs://" + fsAuthority + "/wrapper/toBeDelete"); + Assert.assertTrue(fs.delete(deletePath, true)); + + try { + fs.listStatus(deletePath); + fail("the path " + deletePath + " should not exists"); + } catch (FileNotFoundException e) { + // success + } + + FileStatus[] status = fs.listStatus(deletePath.getParent()); + Assert.assertEquals(0, status.length); + // } + } + + @Test + public void testReadWriteFiles() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + // conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + // conf.set("fs.hdfs.impl.disable.cache", "true"); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + + Path fileInWrapper = new Path("/wrapper/file1"); + Path fileInHDFS = new Path("/hdfsfile1"); + + byte[] bytes = generateByteArry(4096); + + try (FSDataOutputStream out = fs.create(fileInWrapper)) { + out.write(bytes); + } catch (IOException e) { + fail("write file failed"); + } + + try (FSDataOutputStream out = fs.create(fileInHDFS)) { + out.write(bytes); + } catch (IOException e) { + fail("write file failed"); + } + + byte[] resultWrapper = new byte[4096]; + try (FSDataInputStream in = fs.open(fileInHDFS)) { + in.read(resultWrapper); + } + + byte[] resultHdfs = new byte[4096]; + try (FSDataInputStream in = fs.open(fileInHDFS)) { + in.read(resultHdfs); + } + + Assert.assertArrayEquals(bytes, resultHdfs); + Assert.assertArrayEquals(bytes, resultWrapper); + // } + } + + + + @Test + public void testListStatus() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + // hdfs native file + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + Path listPath = new Path("hdfs://" + fsAuthority + "/wrapper"); + Path listPathWithTailSlash = new Path("hdfs://" + fsAuthority + "/wrapper/"); + FileStatus[] files = fs.listStatus(listPath); + Set filesSet = Arrays.stream(files).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(4, files.length); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "folder1").toString())); + + FileStatus[] files2 = fs.listStatus(listPathWithTailSlash); + Set filesSet2 = Arrays.stream(files2).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(4, files2.length); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "folder1").toString())); + + + RemoteIterator recursivelistIrt = fs.listFiles(listPath, true); + Set recursivelistPaths = new HashSet<>(); + while (recursivelistIrt.hasNext()) { + recursivelistPaths.add(recursivelistIrt.next().getPath().toString()); + } + + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile1").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile2").toString())); + + RemoteIterator nonRecursivelistIrt = fs.listFiles(listPath, false); + Set nonRecursivelistPaths = new HashSet<>(); + while (nonRecursivelistIrt.hasNext()) { + nonRecursivelistPaths.add(nonRecursivelistIrt.next().getPath().toString()); + } + + Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile3").toString())); + + Path listHDFSPath = new Path("hdfs://" + fsAuthority + "/"); + FileStatus[] hdfsFiles = fs.listStatus(listHDFSPath); + Set filesSetHdfs = Arrays.stream(hdfsFiles).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(5, filesSetHdfs.size()); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsfile1").toString())); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsfile2").toString())); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsfile3").toString())); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "hdfsdir").toString())); + + RemoteIterator recursivelistHDFSIrt = fs.listFiles(listHDFSPath, true); + Set recursivelistHDFSPaths = new HashSet<>(); + while (recursivelistHDFSIrt.hasNext()) { + recursivelistHDFSPaths.add(recursivelistHDFSIrt.next().getPath().toString()); + } + Assert.assertEquals(7, recursivelistHDFSPaths.size()); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsfile1").toString())); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsfile2").toString())); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsfile3").toString() )); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsdir/subfile1").toString())); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "hdfsdir/subfile2").toString())); + fSys.deleteOnExit(new Path("/wrapper")); + // } + } + + @Test + public void testListStatus1() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link" + + "./wrapper1234/", folder.getRoot().toURI().toString() + "/test1234"); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper", + folder.getRoot().toURI().toString() + "/test1234"); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper" + + "/1234/", folder.getRoot().toURI().toString() + "/test12345"); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./yy/", + folder.getRoot().toURI().toString() + "/folder1"); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + // hdfs native file + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + Path listPath = new Path("hdfs://" + fsAuthority + "/yy"); + + FileStatus[] status = fs.listStatus(listPath); + Assert.assertEquals(2, status.length); + fSys.deleteOnExit(new Path("/wrapper")); + // } + } + + @Test + public void testListLocatedStatus() throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + // hdfs native file + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + Path listPath = new Path("hdfs://" + fsAuthority + "/wrapper"); + Path listPathWithTailSlash = new Path("hdfs://" + fsAuthority + "/wrapper/"); + RemoteIterator files = fs.listLocatedStatus(listPath); + List locatedStatusList = new ArrayList<>(); + while (files.hasNext()) { + locatedStatusList.add(files.next()); + } + + Set filesSet = locatedStatusList.stream().map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(4, locatedStatusList.size()); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "folder1").toString())); + + RemoteIterator files2 = fs.listLocatedStatus(listPathWithTailSlash); + List locatedStatusList2 = new ArrayList<>(); + while (files2.hasNext()) { + locatedStatusList2.add(files2.next()); + } + Set filesSet2 = locatedStatusList2.stream().map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(4, locatedStatusList2.size()); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "folder1").toString())); + fSys.deleteOnExit(new Path("/wrapper")); + // } + } + + @Test + public void testListStatus2() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./ ", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + // hdfs native file + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + Path listPath = new Path("hdfs://" + fsAuthority + "/"); + Path listPathWithTailSlash = new Path("hdfs://" + fsAuthority + "/"); + FileStatus[] files = fs.listStatus(listPath); + Set filesSet = Arrays.stream(files).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(4, files.length); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(filesSet.contains(new Path(listPath, "folder1").toString())); + + FileStatus[] files2 = fs.listStatus(listPathWithTailSlash); + Set filesSet2 = Arrays.stream(files2).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(4, files2.length); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(filesSet2.contains(new Path(listPath, "folder1").toString())); + + + RemoteIterator recursivelistIrt = fs.listFiles(listPath, true); + Set recursivelistPaths = new HashSet<>(); + while (recursivelistIrt.hasNext()) { + recursivelistPaths.add(recursivelistIrt.next().getPath().toString()); + } + + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "newFile3").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile1").toString())); + Assert.assertTrue(recursivelistPaths.contains(new Path(listPath, "folder1/subFile2").toString())); + + RemoteIterator nonRecursivelistIrt = fs.listFiles(listPath, false); + Set nonRecursivelistPaths = new HashSet<>(); + while (nonRecursivelistIrt.hasNext()) { + nonRecursivelistPaths.add(nonRecursivelistIrt.next().getPath().toString()); + } + + Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile1").toString())); + Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile2").toString())); + Assert.assertTrue(nonRecursivelistPaths.contains(new Path(listPath, "newFile3").toString())); + + Path listHDFSPath = new Path("hdfs://" + fsAuthority + "/"); + FileStatus[] hdfsFiles = fs.listStatus(listHDFSPath); + Set filesSetHdfs = Arrays.stream(hdfsFiles).map(FileStatus::getPath).map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(4, filesSetHdfs.size()); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "folder1").toString())); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "newFile3").toString())); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "newFile2").toString())); + Assert.assertTrue(filesSetHdfs.contains(new Path(listHDFSPath, "newFile1").toString())); + + RemoteIterator recursivelistHDFSIrt = fs.listFiles(listHDFSPath, true); + Set recursivelistHDFSPaths = new HashSet<>(); + while (recursivelistHDFSIrt.hasNext()) { + recursivelistHDFSPaths.add(recursivelistHDFSIrt.next().getPath().toString()); + } + Assert.assertEquals(5, recursivelistHDFSPaths.size()); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "newFile1").toString())); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "newFile3").toString())); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "newFile2").toString())); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "folder1/subFile1").toString())); + Assert.assertTrue(recursivelistHDFSPaths.contains(new Path(listHDFSPath, "folder1/subFile2").toString())); + + fSys.deleteOnExit(new Path("/wrapper")); + // } + } + + @Test + public void testListStatus3() throws IOException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/", + folder.getRoot().toURI().toString() + "/folder1"); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + // hdfs native file + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + // Path listPath = new Path("hdfs://" + fsAuthority + "/"); + Path listPathLocal = new Path("hdfs://" + fsAuthority + "/wrapper/"); + Path listPathLocal2 = new Path("hdfs://" + fsAuthority + "/abc/"); + + RemoteIterator filesLocalItr2 = fs.listStatusIterator(listPathLocal2); + Set filesLocal2Set = remoteItrToList(filesLocalItr2).stream().map(FileStatus::getPath) + .map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(2, filesLocal2Set.size()); + Assert.assertTrue(filesLocal2Set.contains(new Path(listPathLocal2, "subFile1").toString())); + Assert.assertTrue(filesLocal2Set.contains(new Path(listPathLocal2, "subFile2").toString())); + + RemoteIterator locatedFilesLocalItr2 = fs.listLocatedStatus(listPathLocal2); + Set locatedFilesLocal2Set = remoteItrToList(locatedFilesLocalItr2).stream().map(FileStatus::getPath) + .map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(2, locatedFilesLocal2Set.size()); + Assert.assertTrue(locatedFilesLocal2Set.contains(new Path(listPathLocal2, "subFile1").toString())); + Assert.assertTrue(locatedFilesLocal2Set.contains(new Path(listPathLocal2, "subFile2").toString())); + + RemoteIterator recursiveFilesLocalItr2 = fs.listFiles(listPathLocal2, true); + Set recursiveFilesLocal2Set = remoteItrToList(recursiveFilesLocalItr2).stream().map(FileStatus::getPath) + .map(Path::toString).collect(Collectors.toSet()); + Assert.assertEquals(2, recursiveFilesLocal2Set.size()); + Assert.assertTrue(recursiveFilesLocal2Set.contains(new Path(listPathLocal2, "subFile1").toString())); + Assert.assertTrue(recursiveFilesLocal2Set.contains(new Path(listPathLocal2, "subFile2").toString())); + fSys.deleteOnExit(new Path("/wrapper")); + // } + } + + @Test + public void testDeleteFile() throws IOException, NoSuchMethodException, InvocationTargetException, IllegalAccessException { + // Configuration conf = new HdfsConfiguration(); + // try (MiniDFSCluster dfsCluster = new MiniDFSCluster.Builder(conf).numDataNodes(1).build()) { + // String fsAuthority = dfsCluster.getNameNode().getClientNamenodeAddress(); + + conf.set("fs.hdfs.impl", OBSHDFSFileSystem.class.getName()); + conf.set("fs.trash.interval", "30"); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./wrapper/", + folder.getRoot().toURI().toString()); + conf.set("fs.hdfs.mounttable." + fsAuthority + ".link./abc/", + folder.getRoot().toURI().toString() + "/folder1"); + conf.set("fs.hdfs.impl.disable.cache", "true"); + + // hdfs native file + fSys.create(new Path("/hdfsfile1")).close(); + fSys.create(new Path("/hdfsfile2")).close(); + fSys.create(new Path("/hdfsfile3")).close(); + fSys.mkdirs(new Path("/hdfsdir"), + FsPermission.createImmutable((short) 777)); + fSys.create(new Path("/hdfsdir/subfile1")).close(); + fSys.create(new Path("/hdfsdir/subfile2")).close(); + + // wrapped local file system files + folder.newFile("newFile1"); + folder.newFile("newFile2"); + folder.newFile("newFile3"); + File subfolder = folder.newFolder("folder1"); + subfolder.toPath().resolve("subFile1").toFile().createNewFile(); + subfolder.toPath().resolve("subFile2").toFile().createNewFile(); + + FileSystem fs = new Path("hdfs://" + fsAuthority + "/").getFileSystem(conf); + Path file1 = new Path("hdfs://" + fsAuthority + "/hdfsfile1"); + Assert.assertTrue(fs.delete(file1, false)); + try { + fs.getFileStatus(file1); + fail("should throw FileNotFoundException"); + } catch (IOException e) { + Assert.assertTrue(e instanceof FileNotFoundException); + Assert.assertFalse(fs.exists(file1)); + } + + Path fileFolder = new Path("hdfs://" + fsAuthority + "/hdfsdir"); + Assert.assertTrue(fs.delete(fileFolder, true)); + Assert.assertFalse(fs.exists(fileFolder)); + + Path fileLocal1 = new Path("hdfs://" + fsAuthority + "/wrapper/newFile1"); + Assert.assertTrue(fs.delete(fileLocal1, false)); + + try { + fs.getFileStatus(fileLocal1); + fail("should throw FileNotFoundException"); + } catch (IOException e) { + Assert.assertTrue(e instanceof FileNotFoundException); + Assert.assertFalse(fs.exists(fileLocal1)); + } + + Path fileLocalFolder = new Path("hdfs://" + fsAuthority + "/wrapper/folder1"); + Assert.assertTrue(fs.delete(fileLocalFolder, true)); + Assert.assertFalse(fs.exists(fileLocalFolder)); + // } + } + + private List remoteItrToList(RemoteIterator itr) throws IOException { + List list = new ArrayList<>(); + if (itr != null) { + while (itr.hasNext()) { + list.add(itr.next()); + } + } + return list; + } + + Random r = new Random(); + private byte[] generateByteArry(int length) { + byte[] arr = new byte[length]; + r.nextBytes(arr); + return arr; + } + + @Override + public void testFsStatus() { + skip("Unsupport."); + } + + @Override + public void testWorkingDirectory() { + skip("Unsupport."); + } + + @Override + public void testListStatusThrowsExceptionForUnreadableDir() { + skip("Unsupport."); + } + + @Override + public void testRenameDirectoryAsEmptyDirectory() { + skip("Unsupport."); + } + + @Override + public void testRenameDirectoryAsFile() { + skip("Unsupport."); + } + + @Override + public void testRenameDirectoryAsNonEmptyDirectory() { + skip("Unsupport."); + } + + @Override + public void testRenameDirectoryToItself(){ + skip("Unsupport."); + } + + @Override + public void testRenameDirectoryToNonExistentParent(){ + skip("Unsupport."); + } + + @Override + public void testRenameFileAsExistingDirectory(){ + skip("Unsupport."); + } + + @Override + public void testRenameFileAsExistingFile(){ + skip("Unsupport."); + } + + @Override + public void testRenameFileToItself(){ + skip("Unsupport."); + } + + @Override + public void testRenameFileToNonExistentDirectory(){ + skip("Unsupport."); + } + + @Override + public void testRenameNonExistentPath(){ + skip("Unsupport."); + } + + @Override + public void testWDAbsolute(){ + skip("Unsupport."); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHFlush.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHFlush.java new file mode 100644 index 0000000..01ae3fc --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSHFlush.java @@ -0,0 +1,249 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertNull; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.hdfs.client.HdfsDataOutputStream; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.util.Arrays; +import java.util.Collection; +import java.util.EnumSet; + +/** + * OBS tests for configuring block size. + */ +@RunWith(Parameterized.class) +public class ITestOBSHFlush { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static final Logger LOG = LoggerFactory.getLogger( + ITestOBSHFlush.class); + + @Parameterized.Parameters + public static Collection inputStreams() { + return Arrays.asList( + OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_SYNC, + OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_EMPTY, + OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_FLUSH + ); + } + + private boolean downgrade; + + public ITestOBSHFlush(String downgradePolicy) throws IOException { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024)); + conf.set(OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY, downgradePolicy); + fs = OBSTestUtils.createTestFileSystem(conf); + if (fs.exists(getTestPath("testFlush"))) { + fs.delete(getTestPath("testFlush"), true); + } + downgrade = (!downgradePolicy.equals(OBSConstants.OUTPUT_STREAM_HFLUSH_POLICY_SYNC)); + } + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + fs = null; + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 200M数据,分10次写,每次写完hflush,校验文件大小 + public void testFlush01() { + if (!fs.isFsBucket()) { + return; + } + try { + doTheJob("testFlush", OBSAppendTestUtil.BLOCK_SIZE, (short) 2, false, downgrade, + EnumSet.noneOf(HdfsDataOutputStream.SyncFlag.class), + 1024 * 1024 * 200); + } catch (Exception e) { + LOG.error(e.getMessage()); + assertNull("cache exception", e); + } + } + + @Test + // 20M数据,分10次写,每次写完hflush,校验文件大小 + public void testFlush02() { + if (!fs.isFsBucket()) { + return; + } + try { + doTheJob("testFlush", OBSAppendTestUtil.BLOCK_SIZE, (short) 2, false, downgrade, + EnumSet.noneOf(HdfsDataOutputStream.SyncFlag.class), + 1024 * 1024 * 20); + } catch (Exception e) { + LOG.error(e.getMessage()); + assertNull("cache exception", e); + } + } + + @Test + // 51M数据,分10次写,每次写完hflush,校验文件大小 + public void testFlush03() { + if (!fs.isFsBucket()) { + return; + } + try { + doTheJob("testFlush", OBSAppendTestUtil.BLOCK_SIZE, (short) 2, false, downgrade, + EnumSet.noneOf(HdfsDataOutputStream.SyncFlag.class), + 1024 * 1024 * 51); + } catch (Exception e) { + LOG.error(e.getMessage()); + assertNull("cache exception", e); + } + } + + /** + * The method starts new cluster with defined Configuration; creates a file + * with specified block_size and writes 10 equal sections in it; it also + * calls hflush/hsync after each write and throws an IOException in case of + * an error. + * + * @param fileName of the file to be created and processed as required + * @param block_size value to be used for the file's creation + * @param replicas is the number of replicas + * @param isSync hsync or hflush + * @param syncFlags specify the semantic of the sync/flush + * @throws IOException in case of any errors + */ + public void doTheJob(final String fileName, + long block_size, short replicas, boolean isSync, boolean downgrade, + EnumSet syncFlags, int size) + throws IOException { + byte[] fileContent; + final int SECTIONS = 10; + + fileContent = OBSAppendTestUtil.initBuffer(size); + + FSDataInputStream is; + try { + Path path = getTestPath(fileName); + FSDataOutputStream stm = fs.create(path, false, 4096, replicas, + block_size); + + int tenth = size / SECTIONS; + int rounding = size - tenth * SECTIONS; + for (int i = 0; i < SECTIONS; i++) { + // write to the file + stm.write(fileContent, tenth * i, tenth); + + // Wait while hflush/hsync pushes all packets through built pipeline + if (isSync) { + ((OBSBlockOutputStream) stm.getWrappedStream()).hsync(); + } else { + ((OBSBlockOutputStream) stm.getWrappedStream()).hflush(); + } + + if (downgrade) { + // skip check flushed block + continue; + } + + // Check file length if update length is required + if (isSync && syncFlags.contains(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH)) { + long currentFileLength = fs.getFileStatus(path).getLen(); + assertEquals("File size doesn't match for hsync/hflush with updating the length", tenth * (i + 1), currentFileLength); + } else if (!isSync || (!syncFlags.contains(HdfsDataOutputStream.SyncFlag.END_BLOCK) + && !syncFlags.contains(HdfsDataOutputStream.SyncFlag.UPDATE_LENGTH))) { + long currentFileLength = fs.getFileStatus(path).getLen(); + assertEquals("File size doesn't match for hsync/hflush with updating the length", tenth * (i + 1), currentFileLength); + } + + byte[] toRead = new byte[tenth]; + byte[] expected = new byte[tenth]; + System.arraycopy(fileContent, tenth * i, expected, 0, tenth); + // Open the same file for read. Need to create new reader after every write operation(!) + is = fs.open(path); + is.seek(tenth * i); + int readBytes = is.read(toRead, 0, tenth); + System.out.println("Has read " + readBytes); + Assert.assertTrue("Should've get more bytes", + (readBytes > 0) && (readBytes <= tenth)); + is.close(); + checkData(toRead, 0, readBytes, expected, + "Partial verification"); + } + + stm.write(fileContent, tenth * SECTIONS, rounding); + stm.close(); + + OBSAppendTestUtil.checkFullFile(fs, path, fileContent.length, + fileContent, "hflush()"); + } finally { + fs.delete(new Path(testRootPath), true); + fs.close(); + fs = null; + } + } + + void checkData(final byte[] actual, int from, int len, + final byte[] expected, String message) { + for (int idx = 0; idx < len; idx++) { + assertEquals( + message + " byte " + (from + idx) + " differs. expected " + + expected[from + idx] + " actual " + actual[idx], + expected[from + idx], actual[idx]); + actual[idx] = 0; + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStream.java new file mode 100644 index 0000000..8d880c9 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStream.java @@ -0,0 +1,669 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.obs.input.InputPolicyFactory; +import org.apache.hadoop.fs.obs.input.InputPolicys; +import org.apache.hadoop.fs.obs.input.ReadAheadBuffer; +import org.apache.hadoop.fs.obs.input.ReadAheadTask; +import org.apache.hadoop.fs.obs.mock.MockMemArtsCCClient; +import org.apache.hadoop.io.IOUtils; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.junit.runner.RunWith; +import org.junit.runners.Parameterized; +import org.mockito.Mockito; +import org.mockito.stubbing.Answer; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.IOException; +import java.lang.reflect.Field; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Collection; +import java.util.Random; + +/** + * Tests basic functionality for OBSInputStream, including seeking and + * reading files. + */ +@RunWith(Parameterized.class) +public class ITestOBSInputStream { + + private OBSFileSystem fs; + + private static final Logger LOG = + LoggerFactory.getLogger(ITestOBSInputStream.class); + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private String inputPolicy; + + @Parameterized.Parameters + public static Collection inputStreams() { + return Arrays.asList( + OBSConstants.READAHEAD_POLICY_PRIMARY, + // OBSConstants.READAHEAD_POLICY_ADVANCE, + OBSConstants.READAHEAD_POLICY_MEMARTSCC + ); + } + + public ITestOBSInputStream(String inputPolicy) throws IOException, NoSuchFieldException, IllegalAccessException { + this.inputPolicy = inputPolicy; + if (inputPolicy.equals(OBSConstants.READAHEAD_POLICY_PRIMARY)) { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.set(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_PRIMARY); + fs = OBSTestUtils.createTestFileSystem(conf); + } + if (inputPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.set(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_MEMARTSCC); + conf.set(OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE, OBSConstants.MEMARTSCC_INPUTSTREAM_BUFFER_TYPE_BIND); + this.fs = OBSTestUtils.createTestFileSystem(conf); + + MockMemArtsCCClient mockMemArtsCCClient; + mockMemArtsCCClient = new MockMemArtsCCClient(fs, false, this.fs.getBucket()); + mockMemArtsCCClient.init("", ""); + + InputPolicyFactory inputPolicyFactory = InputPolicys.createFactory(OBSConstants.READAHEAD_POLICY_MEMARTSCC); + + // mock memartscc client + Field ccClient = OBSFileSystem.class.getDeclaredField("memArtsCCClient"); + ccClient.setAccessible(true); + ccClient.set(this.fs, mockMemArtsCCClient); + + // mock input policy factory + Field fInputPolicyFactory = OBSFileSystem.class.getDeclaredField("inputPolicyFactory"); + fInputPolicyFactory.setAccessible(true); + fInputPolicyFactory.set(this.fs, inputPolicyFactory); + } + } + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout(30 * 60 * 1000); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path setPath(String path) { + if (path.startsWith("/")) { + return new Path(testRootPath + path); + } else { + return new Path(testRootPath + "/" + path); + } + } + + private static void genTestBuffer(byte[] buffer, long lenth) { + for (int i = 0; i < lenth; i++) { + buffer[i] = (byte) (i % 255); + } + } + + @Test + // 校验inputstream默认readahead大小为1M + public void testDefaultReadAheadSize() { + long readAHeadSize = fs.getReadAheadRange(); + assertTrue("Default read ahead size must 1MB", + readAHeadSize == 1 * 1024 * 1024); + } + + @Test + public void testFileReaderTask() throws Exception { + Path smallSeekFile = setPath("/test/testFileReaderTask.txt"); + long size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, 255); + LOG.info("5MB file created: smallSeekFileOSSFileReader.txt"); + ReadAheadBuffer readBuffer = new ReadAheadBuffer(12, 24); + ReadAheadTask task = new ReadAheadTask(fs.getBucket(),"1", + fs.getObsClient(), readBuffer); + //NullPointerException, fail + task.run(); + assertEquals(readBuffer.getStatus(), ReadAheadBuffer.STATUS.ERROR); + //OK + task = new ReadAheadTask(fs.getBucket(),"test/test/testFileReaderTask.txt", + fs.getObsClient(), readBuffer); + task.run(); + assertEquals(readBuffer.getStatus(), ReadAheadBuffer.STATUS.SUCCESS); + } + + @Test + // 打开文件流,按固定长度和随机长度分别向前seek 5次 + public void testInSeekFile() throws Exception { + Path smallSeekFile = setPath("/test/smallSeekFile.txt"); + long size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("5MB file created: smallSeekFile.txt"); + + FSDataInputStream instream = this.fs.open(smallSeekFile); + int seekTimes = 5; + LOG.info("multiple fold position seeking test...:"); + for (int i = 0; i < seekTimes; i++) { + long pos = size / (seekTimes - i) - 1; + LOG.info("begin seeking for pos: " + pos); + instream.seek(pos); + assertTrue("expected position at:" + pos + ", but got:" + + instream.getPos(), instream.getPos() == pos); + LOG.info("completed seeking at pos: " + instream.getPos()); + } + LOG.info("random position seeking test...:"); + Random rand = new Random(); + for (int i = 0; i < seekTimes; i++) { + long pos = Math.abs(rand.nextLong()) % size; + LOG.info("begin seeking for pos: " + pos); + instream.seek(pos); + assertTrue("expected position at:" + pos + ", but got:" + + instream.getPos(), instream.getPos() == pos); + LOG.info("completed seeking at pos: " + instream.getPos()); + } + IOUtils.closeStream(instream); + } + + @Test + // 打开文件输入流,分别测试随机和顺序读取功能正确性 + public void testSequentialAndRandomRead() throws Exception { + Path smallSeekFile = setPath("/test/smallSeekFile.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("5MB file created: smallSeekFile.txt"); + + long now = System.currentTimeMillis(); + FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile); + // OBSInputStream in = + // (OBSInputStream) fsDataInputStream.getWrappedStream(); + assertTrue("expected position at:" + 0 + ", but got:" + + fsDataInputStream.getPos(), fsDataInputStream.getPos() == 0); + + byte[] bytes = new byte[(int) size]; + int bytesRead = 0; + int bufOffset = 0; + int bytesRemaining = size; + int ret = 0; + while (bytesRead < size) { + ret = fsDataInputStream.read(bytes, bufOffset, bytesRemaining); + // System.out.println(ret); + + if (ret < 0) { + break; + } + + bufOffset += ret; + bytesRemaining -= ret; + bytesRead += ret; + } + + LOG.warn("time consumed mills:" + String.valueOf( + System.currentTimeMillis() - now)); + + byte[] testBuffer = new byte[256]; + genTestBuffer(testBuffer, 256); + + byte[] equalSizeBuffer = new byte[(int) size]; + + for (int i = 0; i < equalSizeBuffer.length; i++) { + equalSizeBuffer[i] = testBuffer[(i) % 256]; + } + assertTrue(Arrays.equals(bytes, equalSizeBuffer)); + + fsDataInputStream.close(); + //fsDataInputStream.seek(4 * 1024 * 1024); + + } + + @Test + // 打开文件输入流,测试read byteBuffer功能正确性 + public void testReadWithByteBuffer() throws Exception { + final int bufLen = 256 * 1024; + final int sizeMB = 100; + String filename = "readTestFile_" + sizeMB + ".txt"; + Path readTestFile = setPath("/test/" + filename); + long size = sizeMB * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256, + 255); + LOG.info(sizeMB + "MB file created: /test/" + filename); + + FSDataInputStream instream = this.fs.open(readTestFile); + ByteBuffer byteBuffer = ByteBuffer.allocateDirect(bufLen); + long expectedRound = size / bufLen + (size % bufLen == 0 ? 0 : 1); + long bytesRead = 0; + long round = 0; + while (bytesRead < size) { + byteBuffer.clear(); + round++; + int bytes = instream.read(byteBuffer); + if (bytes == -1) { + break; + } + + if (round < expectedRound) { + assertEquals(bufLen, bytes); + } else { + assertEquals(size - bytesRead, bytes); + } + + bytesRead += bytes; + + if (bytesRead % (1024 * 1024) == 0) { + int available = instream.available(); + int remaining = (int) (size - bytesRead); + assertTrue("expected remaining:" + remaining + ", but got:" + + available, + remaining == available); + LOG.info("Bytes read: " + Math.round( + (double) bytesRead / (1024 * 1024)) + + " MB"); + } + + byteBuffer.flip(); + for (int i = 0; i < bytes; i++) { + byteBuffer.get(); + } + assertEquals(0, byteBuffer.remaining()); + } + assertEquals(0, instream.available()); + assertEquals(expectedRound, round); + assertEquals(size, bytesRead); + IOUtils.closeStream(instream); + } + + @Test + // 打开文件流,测试read(byte[] buf, final int off, final int len)接口正确性 + public void testReadFile() throws Exception { + final int bufLen = 256; + final int sizeFlag = 5; + String filename = "readTestFile_" + sizeFlag + ".txt"; + Path readTestFile = setPath("/test/" + filename); + long size = sizeFlag * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256, + 255); + LOG.info(sizeFlag + "MB file created: /test/" + filename); + + FSDataInputStream instream = this.fs.open(readTestFile); + byte[] buf = new byte[bufLen]; + long bytesRead = 0; + while (bytesRead < size) { + int bytes; + if (size - bytesRead < bufLen) { + int remaining = (int) (size - bytesRead); + bytes = instream.read(buf, 0, remaining); + } else { + bytes = instream.read(buf, 0, bufLen); + } + bytesRead += bytes; + + if (bytesRead % (1024 * 1024) == 0) { + int available = instream.available(); + int remaining = (int) (size - bytesRead); + assertTrue("expected remaining:" + remaining + ", but got:" + + available, + remaining == available); + LOG.info("Bytes read: " + Math.round( + (double) bytesRead / (1024 * 1024)) + + " MB"); + } + } + assertTrue(instream.available() == 0); + IOUtils.closeStream(instream); + } + + @Test + // 打开文件输入流,测试read(long position, byte[] buffer, int offset, int length)接口正确性 + public void testReadByPosistion() throws IOException { + Path smallSeekFile = setPath("/test/smallSeekFile.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("5MB file created: smallSeekFile.txt"); + + long now = System.currentTimeMillis(); + FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile); + // OBSInputStream in + // = (OBSInputStream) fsDataInputStream.getWrappedStream(); + assertTrue("expected position at:" + 0 + ", but got:" + + fsDataInputStream.getPos(), + fsDataInputStream.getPos() == 0); + + byte[] testBuffer = new byte[256]; + genTestBuffer(testBuffer, 256); + + int length = 1024 * 1024; + int pos = 0; + byte[] randomBytes = new byte[length]; + int len = fsDataInputStream.read(pos, randomBytes, 0, length); + // if (fs.getMetricSwitch()) { + // MockMetricsConsumer mmc + // = (MockMetricsConsumer) fs.getMetricsConsumer(); + // assertEquals("read", mmc.getOpName()); + // assertTrue(mmc.isSuccess()); + // assertEquals("random", mmc.getOpType()); + // + // assertTrue(length == len); + // LOG.warn("random read len: " + len); + // } + byte[] equalsRandomBuffer = new byte[length]; + for (int i = 0; i < equalsRandomBuffer.length; i++) { + equalsRandomBuffer[i] = testBuffer[(i + pos) % 256]; + } + assertTrue(Arrays.equals(randomBytes, equalsRandomBuffer)); + + int overlapSize = 1024; + len = fsDataInputStream.read(size - overlapSize, randomBytes, 0, length); + assertEquals(overlapSize, len); + + len = fsDataInputStream.read(size, randomBytes, 0, length); + assertEquals(-1, len); + + LOG.warn("time consumed mills:" + String.valueOf( + System.currentTimeMillis() - now)); + + fsDataInputStream.close(); + } + + @Test + // 读取位置超过文件大小时,校验是否返回-1 + public void testReadPositionExceedFileLength() throws IOException { + Path smallSeekFile = setPath("/test/smallSeekFile.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("5MB file created: smallSeekFile.txt"); + + long now = System.currentTimeMillis(); + FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile); + // OBSInputStream in + // = (OBSInputStream) fsDataInputStream.getWrappedStream(); + assertTrue("expected position at:" + 0 + ", but got:" + + fsDataInputStream.getPos(), + fsDataInputStream.getPos() == 0); + + byte[] testBuffer = new byte[256]; + genTestBuffer(testBuffer, 256); + + int length = 10 * 1024 * 1024; + int pos = size; + byte[] randomBytes = new byte[length]; + int len = fsDataInputStream.read(pos, randomBytes, 0, length); + assertTrue(len == -1); + + pos = size + 1; + len = fsDataInputStream.read(pos, randomBytes, 0, length); + assertTrue(len == -1); + fsDataInputStream.close(); + + } + + @Test + // 读取长度超过文件大小时,校验是否成功读到文件末尾 + public void testReadLengthExceedFileLength() throws IOException { + if (inputPolicy.equals(OBSConstants.READAHEAD_POLICY_MEMARTSCC)) { + return; + /** + * 争议用例:hadoop PositionedReadable接口中的 + * int read(long position, byte[] buffer, int offset, int length)方法 + * 没有规定:返回值必需要等于min(buffer.size, length),即必需要读满buffer + * 而该用例假设:读取5M数据,返回读取成功大小一定等于5M + */ + } + Path smallSeekFile = setPath("/test/smallSeekFile.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("5MB file created: smallSeekFile.txt"); + + long now = System.currentTimeMillis(); + FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile); + // OBSInputStream in + // = (OBSInputStream) fsDataInputStream.getWrappedStream(); + assertTrue("expected position at:" + 0 + ", but got:" + + fsDataInputStream.getPos(), + fsDataInputStream.getPos() == 0); + + byte[] testBuffer = new byte[256]; + genTestBuffer(testBuffer, 256); + + int length = 10 * 1024 * 1024; + int pos = 0; + byte[] randomBytes = new byte[length]; + int len = fsDataInputStream.read(pos, randomBytes, 0, length); + assertTrue(size == len); + fsDataInputStream.close(); + } + + @Test + // 读取长度超过文件大小时,校验是否成功读到文件末尾 + public void testReadLengthExceedFileLength1() throws IOException { + Path smallSeekFile = setPath("/test/smallSeekFile1.txt"); + int size = 5 * 1024; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("1MB file created: smallSeekFile1.txt"); + + long now = System.currentTimeMillis(); + FSDataInputStream fsDataInputStream = this.fs.open(smallSeekFile); + // OBSInputStream in + // = (OBSInputStream) fsDataInputStream.getWrappedStream(); + assertTrue("expected position at:" + 0 + ", but got:" + + fsDataInputStream.getPos(), + fsDataInputStream.getPos() == 0); + + byte[] testBuffer = new byte[256]; + genTestBuffer(testBuffer, 256); + + int length = 1 * 1024 * 1024; + int pos = 0; + byte[] randomBytes = new byte[length]; + int len = fsDataInputStream.read(pos, randomBytes, 0, length); + assertTrue(size == len); + fsDataInputStream.close(); + } + + @Test + // 校验四个参数read转三个参数read开关是否能正确配置生效 + public void testReadTransformSwitch() throws Exception { + final int sizeFlag = 5; + String filename = "readTestFile_" + sizeFlag + ".txt"; + Path readTestFile = setPath("/test/" + filename); + long size = sizeFlag * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, readTestFile, size, 256, + 255); + LOG.info(sizeFlag + "MB file created: /test/" + filename); + + byte[] buffer = new byte[32]; + FSDataInputStream fsDataInputStream = this.fs.open(readTestFile); + FSInputStream obsInputStream + = (FSInputStream) fsDataInputStream.getWrappedStream(); + FSInputStream mockInputStream = Mockito.spy(obsInputStream); + System.out.println(mockInputStream); + Mockito.doReturn(-100) + .when((FSInputStream) mockInputStream) + .read(buffer, 0, 10); + int readLen = mockInputStream.read(0, buffer, 0, 10); + assertTrue(readLen == -100); + IOUtils.closeStream(fsDataInputStream); + fs.close(); + fs = null; + + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.READAHEAD_TRANSFORM_ENABLE, false); + OBSFileSystem obsFs = OBSTestUtils.createTestFileSystem(conf); + fsDataInputStream = obsFs.open(readTestFile); + obsInputStream = (FSInputStream) fsDataInputStream.getWrappedStream(); + mockInputStream = Mockito.spy(obsInputStream); + Mockito.when(mockInputStream.read(buffer, 0, 10)) + .then((Answer) invocationOnMock -> -100); + readLen = mockInputStream.read(0, buffer, 0, 10); + assertTrue(readLen == 10); + IOUtils.closeStream(fsDataInputStream); + obsFs.close(); + } + + @Test + public void testSeekOutOfRange1() throws IOException { + Path smallSeekFile = setPath("/test/seekOutOfRangeFile1.txt"); + long size = 100; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("5MB file created: smallSeekFile.txt"); + + FSDataInputStream instream = this.fs.open(smallSeekFile); + instream.seek(size); + } + + @Test(expected = EOFException.class) + public void testSeekOutOfRange2() throws IOException { + Path smallSeekFile = setPath("/test/seekOutOfRangeFile2.txt"); + long size = 100; + + ContractTestUtils.generateTestFile(this.fs, smallSeekFile, size, 256, + 255); + LOG.info("5MB file created: smallSeekFile.txt"); + + FSDataInputStream instream = this.fs.open(smallSeekFile); + instream.seek(size + 1); + } + + + // @Test + // public void testRead() throws Exception { + // if (fs.getMetricSwitch()) { + // if (!fs.isFsBucket()) { + // return; + // } + // Path testFile = setPath("test_file"); + // if (fs.exists(testFile)) { + // fs.delete(testFile); + // } + // final int fileSize = 1024; + // + // FSDataOutputStream outputStream = fs.create(testFile); + // byte[] data5 = ContractTestUtils.dataset(fileSize, 'a', + // 26); + // outputStream.write(data5); + // outputStream.hsync(); + // FSDataInputStream inputStream; + // inputStream = fs.open(testFile); + // int res = inputStream.read(); + // + // MockMetricsConsumer mmc + // = (MockMetricsConsumer) fs.getMetricsConsumer(); + // assertEquals("read", mmc.getOpName()); + // assertEquals("1byte", mmc.getOpType()); + // assertTrue(mmc.isSuccess()); + // inputStream.close(); + // + // inputStream = fs.open(testFile); + // ByteBuffer byteBuffer = ByteBuffer.allocateDirect(fileSize); + // int res1 = inputStream.read(byteBuffer); + // MockMetricsConsumer mmc1 = + // (MockMetricsConsumer) fs.getMetricsConsumer(); + // assertEquals("read", mmc1.getOpName()); + // assertEquals("byteBuf", mmc1.getOpType()); + // assertTrue(mmc1.isSuccess()); + // inputStream.close(); + // + // inputStream = fs.open(testFile); + // byte[] bytes = new byte[fileSize]; + // int res2 = inputStream.read(bytes, 0, 1); + // MockMetricsConsumer mmc2 = + // (MockMetricsConsumer) fs.getMetricsConsumer(); + // assertEquals("read", mmc2.getOpName()); + // assertEquals("seq", mmc2.getOpType()); + // assertTrue(mmc2.isSuccess()); + // inputStream.close(); + // + // inputStream = fs.open(testFile); + // byte[] bytes2 = new byte[fileSize]; + // long position = 2; + // int offset = 1; + // int length = 4; + // inputStream.readFully(position, bytes2, offset, length); + // MockMetricsConsumer mmc3 = + // (MockMetricsConsumer) fs.getMetricsConsumer(); + // assertEquals("readFully", mmc3.getOpName()); + // assertTrue(mmc3.isSuccess()); + // inputStream.close(); + // + // inputStream = fs.open(testFile); + // byte[] bytes3 = new byte[fileSize]; + // long position1 = 3; + // int offset1 = 4; + // int length1 = 5; + // int readSize = inputStream.read(position1, bytes3, offset1, + // length1); + // MockMetricsConsumer mmc4 = + // (MockMetricsConsumer) fs.getMetricsConsumer(); + // assertEquals("read", mmc4.getOpName()); + // assertEquals("random", mmc4.getOpType()); + // assertTrue(mmc4.isSuccess()); + // inputStream.close(); + // + // fs.delete(testFile, false); + // } + // } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStreamSimulator.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStreamSimulator.java new file mode 100644 index 0000000..bf94b99 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInputStreamSimulator.java @@ -0,0 +1,45 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.fs.obs.memartscc.OBSInputStreamSimulator; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class ITestOBSInputStreamSimulator { + + @Before + public void setUp() throws Exception { + + } + + @After + public void tearDown() throws Exception { + + } + + @Test + public void testRead() throws Exception{ + long fileStatusLength = 100 * 1024 * 1024; + long readAheadRangeValue = 1024 * 1024; + OBSInputStreamSimulator obsInputStreamSimulator = + new OBSInputStreamSimulator(fileStatusLength, readAheadRangeValue); + int willReadBytes; + long readBytesFromOBS; + + willReadBytes = 5 *1024 * 1024; + readBytesFromOBS = obsInputStreamSimulator.read(willReadBytes); + assertEquals("First read", willReadBytes, readBytesFromOBS); + + willReadBytes = 512 * 1024; + readBytesFromOBS = obsInputStreamSimulator.read(willReadBytes); + assertEquals("Second read", readBytesFromOBS, readAheadRangeValue); + + readBytesFromOBS = obsInputStreamSimulator.read(willReadBytes); + assertEquals("Second read", readBytesFromOBS, 0); + + obsInputStreamSimulator.close(); + } +} + diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInvoker.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInvoker.java new file mode 100644 index 0000000..9c8c744 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSInvoker.java @@ -0,0 +1,244 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import com.obs.services.exception.ObsException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.security.AccessControlException; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.runners.MockitoJUnitRunner; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.net.SocketTimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +@RunWith(MockitoJUnitRunner.class) +public class ITestOBSInvoker { + private static final int COMMON_RETRY_LIMIT = 5; + + private static final int QOS_RETRY_LIMIT = 7; + + private int retryCount; + + private OBSInvoker invoker; + private OBSInvoker invoker_fail; + private OBSFileSystem fs; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + + @Before + public void setup() { + retryCount = 0; + Configuration conf = new Configuration(); + conf.setInt(OBSConstants.RETRY_LIMIT, COMMON_RETRY_LIMIT); + conf.setInt(OBSConstants.RETRY_QOS_LIMIT, QOS_RETRY_LIMIT); + + conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 5); + conf.setLong(OBSConstants.RETRY_SLEEP_MAXTIME, 10); + conf.setLong(OBSConstants.RETRY_QOS_SLEEP_BASETIME, 5); + conf.setLong(OBSConstants.RETRY_QOS_SLEEP_MAXTIME, 10); + + try { + fs = OBSTestUtils.createTestFileSystem(conf); + } catch (IOException e) { + e.printStackTrace(); + } + + OBSRetryPolicy RETRY_POLICY = new OBSRetryPolicy(conf); + invoker = new OBSInvoker(fs, RETRY_POLICY, (text, e, retries, idempotent) -> retryCount++); + + conf.setLong(OBSConstants.RETRY_MAXTIME, 10); + conf.setLong(OBSConstants.RETRY_QOS_MAXTIME, 10); + OBSRetryPolicy RETRY_POLICY1 = new OBSRetryPolicy(conf); + invoker_fail = new OBSInvoker(fs, RETRY_POLICY1, (text, e, retries, idempotent) -> retryCount++); + } + + @Test + public void testNonIOException() { + //can not process non IOException + } + + @Test + public void testAccessControlException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(403); + try { + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw obsException; + }, true); + } catch (AccessControlException e) { + Assert.assertEquals("AccessControlException",0,retryCount); + } + } + + @Test + public void testFileNotFoundException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(404); + try { + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw obsException; + }, true); + } catch (FileNotFoundException e) { + Assert.assertEquals("FileNotFoundException",0,retryCount); + } + } + + @Test + public void testOBSFileConflictException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(409); + try { + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw obsException; + }, true); + } catch (OBSFileConflictException e) { + Assert.assertEquals("OBSFileConflictException",0,retryCount); + } + } + + @Test + public void testOBSIllegalArgumentException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(400); + try { + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw obsException; + }, true); + } catch (OBSIllegalArgumentException e) { + Assert.assertEquals("OBSIllegalArgumentException",0,retryCount); + } + } + + @Test(expected = OBSIOException.class) + public void testOBSIOException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(500); + invoker_fail.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw obsException; + }, true); + + } + + @Test + public void testOBSIOExceptionSuccess() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(500); + final AtomicInteger counter = new AtomicInteger(0); + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + if (counter.incrementAndGet() < COMMON_RETRY_LIMIT) { + throw obsException; + } + return null; + }, true); + assertEquals(COMMON_RETRY_LIMIT-1, retryCount); + } + + @Test(expected = SocketTimeoutException.class) + public void testIOException() throws IOException { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(500); + invoker_fail.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw new SocketTimeoutException("test"); + }, true); + } + + @Test + public void testIOExceptionSuccess() throws IOException { + final AtomicInteger counter = new AtomicInteger(0); + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + if (counter.incrementAndGet() < QOS_RETRY_LIMIT) { + throw new SocketTimeoutException("test"); + } + return null; + }, true); + assertEquals( QOS_RETRY_LIMIT-1, retryCount); + } + + @Test(expected = OBSQosException.class) + public void testOBSQosException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(503); + obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE); + invoker_fail.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw obsException; + }, true); + + } + + @Test + public void testOBSQosExceptionSuccess() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(503); + obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE); + final AtomicInteger counter = new AtomicInteger(0); + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + if (counter.incrementAndGet() < QOS_RETRY_LIMIT) { + throw obsException; + } + return null; + }, true); + assertEquals(QOS_RETRY_LIMIT-1, retryCount); + } + + //translate exception + @Test(expected = OBSQosException.class) + public void testTranslateOBSException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(503); + OBSQosException qosException = new OBSQosException("test", obsException); + obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE); + invoker_fail.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw qosException; + }, true); + } + + //translate exception + @Test + public void testTranslateOBSExceptionSuccess() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(503); + OBSQosException qosException = new OBSQosException("test", obsException); + obsException.setErrorCode(OBSCommonUtils.DETAIL_QOS_CODE); + final AtomicInteger counter = new AtomicInteger(0); + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + if (counter.incrementAndGet() < QOS_RETRY_LIMIT) { + throw qosException; + } + return null; + }, true); + assertEquals(QOS_RETRY_LIMIT-1, retryCount); + } + + //nonIdempotent exception + @Test(expected = OBSIOException.class) + public void testNonIdempotentOBSException() throws Exception { + ObsException obsException = new ObsException("test"); + obsException.setResponseCode(500); + invoker.retryByMaxTime(OBSOperateAction.write,"test", + () -> { + throw obsException; + }, false); + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListFiles.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListFiles.java new file mode 100644 index 0000000..e5bb26f --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListFiles.java @@ -0,0 +1,237 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ITestOBSListFiles { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static final String SUB_DIR_PREFIX = "sub_dir-"; + + private static final String SUB_DIR_FILE_SURFIX = "sub_file-"; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.MULTIPART_SIZE, + String.valueOf(5 * 1024 * 1024)); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 路径是根目录,结果数组大小大于等于0 + public void testListFiles001() throws Exception { + RemoteIterator iterator = fs.listFiles(new Path("/"), + false); + long entries = 0; + while (iterator.hasNext()) { + entries++; + iterator.next(); + } + assertTrue(entries >= 0); + } + + @Test + // 路径是一个文件,recursive为true或false,结果数组大小都为1 + public void testListFiles002() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + + RemoteIterator iterator = fs.listFiles(testFile, + false); + long count = 0; + while (iterator.hasNext()) { + count++; + LocatedFileStatus status = iterator.next(); + assertTrue(status.isFile()); + } + assertEquals(1, count); + + iterator = fs.listFiles(testFile, true); + + count = 0; + while (iterator.hasNext()) { + count++; + LocatedFileStatus status = iterator.next(); + assertTrue(status.isFile()); + } + assertEquals(1, count); + + fs.delete(testFile, false); + } + + @Test + // 路径是一个目录,每级目录下都有多个目录和文件,recursive为false + public void testListFiles003() throws Exception { + Path testPath = getTestPath("test_folder"); + fs.mkdirs(testPath); + + StringBuilder sb = new StringBuilder("test_folder"); + for (int i = 1; i <= 2; i++) { + sb.append("/"); + sb.append(SUB_DIR_PREFIX + i); + } + String lastDir = sb.toString(); + + Path lastDirPath = getTestPath(lastDir); + Path parent = lastDirPath.getParent(); + System.out.println("a " + parent); + + while (!parent.equals(testPath)) { + addSubDirUnderSpecificDir(parent, 2); + addFileUnderSpecificDir(parent, 3); + parent = parent.getParent(); + System.out.println("b " + parent); + } + + RemoteIterator iterator = fs.listFiles(testPath, + false); + + long count = 0; + while (iterator.hasNext()) { + count++; + LocatedFileStatus status = iterator.next(); + System.out.println(status.toString()); + System.out.println("count: " + count); + } + + // assertEquals(1, count); + } + + @Test + // 路径是一个目录,每级目录下都有多个目录和文件,recursive为true + public void testListFiles004() throws Exception { + Path testPath = getTestPath("test_folder"); + fs.mkdirs(testPath); + + StringBuilder sb = new StringBuilder("test_folder"); + for (int i = 1; i <= 2; i++) { + sb.append("/"); + sb.append(SUB_DIR_PREFIX + i); + } + String lastDir = sb.toString(); + + Path lastDirPath = getTestPath(lastDir); + Path parent = lastDirPath.getParent(); + System.out.println("a " + parent); + + while (!parent.equals(testPath)) { + addSubDirUnderSpecificDir(parent, 2); + addFileUnderSpecificDir(parent, 3); + parent = parent.getParent(); + System.out.println("b " + parent); + } + + RemoteIterator iterator = fs.listFiles(testPath, + true); + long count = 0; + while (iterator.hasNext()) { + count++; + LocatedFileStatus status = iterator.next(); + System.out.println(status.toString()); + System.out.println("count: " + count); + } + } + + @Test + // 路径或父目录及上级目录不存在,抛FileNotFoundException + public void testListFilesAbnormal01() throws Exception { + Path testFolder = getTestPath("test_dir"); + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + + boolean hasException = false; + try { + fs.listFiles(testFolder, false); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 路径的父目录及上级目录不是目录,抛AccessControlException + public void testListFilesAbnormal02() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFolder = getTestPath("test_dir"); + FSDataOutputStream outputStream = fs.create(testFolder, false); + outputStream.close(); + + Path testFile = getTestPath("test_dir/test_file"); + + boolean hasException = false; + try { + fs.listFiles(testFile, false); + } catch (AccessControlException e) { + hasException = true; + } + assertTrue(hasException); + } + + private void addSubDirUnderSpecificDir(Path dir, int subDirNum) + throws IOException { + for (int i = 0; i < subDirNum; i++) { + Path subDir = getTestPath(dir + "/" + SUB_DIR_PREFIX + i); + fs.mkdirs(subDir); + } + } + + private void addFileUnderSpecificDir(Path dir, int fileNum) + throws IOException { + byte[] data = ContractTestUtils.dataset(8, 'a', 26); + for (int i = 0; i < fileNum; i++) { + Path file = getTestPath(dir + "/" + SUB_DIR_FILE_SURFIX + i); + FSDataOutputStream outputStream = fs.create(file, false); + outputStream.write(data); + outputStream.close(); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListLocatedStatus.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListLocatedStatus.java new file mode 100644 index 0000000..99c661d --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListLocatedStatus.java @@ -0,0 +1,269 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.LocatedFileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.RemoteIterator; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.util.ArrayList; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ITestOBSListLocatedStatus { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024)); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 列举根目录,不带filter,结果列表大小大于等于0 + public void testListLocatedStatus01() throws Exception { + RemoteIterator iterator = fs.listLocatedStatus( + new Path("/")); + List files = new ArrayList<>(); + while (iterator.hasNext()) { + LocatedFileStatus status = iterator.next(); + files.add(status); + } + assertTrue(files.size() >= 0); + } + + @Test + // 列举文件,不带filter,结果列表大小等于1,并且状态为文件 + public void testListLocatedStatus02() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, true); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + outputStream.write(data); + outputStream.close(); + + RemoteIterator iterator = fs.listLocatedStatus( + testFile); + int entries = 0; + while (iterator.hasNext()) { + entries++; + LocatedFileStatus status = iterator.next(); + assertTrue(status.isFile()); + } + assertEquals(1, entries); + if (fs.exists(testFile)) { + fs.delete(testFile, true); + } + } + + @Test + // 列举文件,带filter,fileter指定文件path,结果列表大小等于1,并且状态为文件 + public void testListLocatedStatus03() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + outputStream.close(); + + PathFilter filter = path -> { + String relativePath = path.toString().split(fs.getBucket())[1]; + return relativePath.equals(testFile.toString()); + }; + RemoteIterator iterator = fs.listLocatedStatus( + testFile, filter); + int entries = 0; + while (iterator.hasNext()) { + entries++; + LocatedFileStatus status = iterator.next(); + assertTrue(status.isFile()); + } + assertEquals(1, entries); + } + + @Test + // 列举一个目录,目录下存在多个文件和目录,结果列表大小为目录下文件和子目录个数之和 + public void testListLocatedStatus04() throws Exception { + Path testFolder = getTestPath("test_folder"); + fs.mkdirs(testFolder); + + for (int i = 0; i < 2; i++) { + Path subFolder = getTestPath("test_folder/sub_dir-" + i); + fs.mkdirs(subFolder); + } + + byte[] data = ContractTestUtils.dataset(8, 'a', 26); + for (int i = 0; i < 3; i++) { + Path file = getTestPath("test_folder/sub_file-" + i); + FSDataOutputStream outputStream = fs.create(file, true); + outputStream.write(data); + outputStream.close(); + } + + RemoteIterator iterator = fs.listLocatedStatus( + testFolder); + int entries = 0; + while (iterator.hasNext()) { + entries++; + iterator.next(); + } + assertEquals(5, entries); + + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + } + + @Test + // 列举一个目录,目录下存在多个文件和目录,带filter只过滤出文件,结果列表大小为目录下文件数之和 + public void testListLocatedStatus05() throws Exception { + Path testFolder = getTestPath("test_folder/"); + fs.mkdirs(testFolder); + + for (int i = 0; i < 2; i++) { + Path subFolder = getTestPath("test_folder/sub_dir-" + i); + fs.mkdirs(subFolder); + } + + byte[] data = ContractTestUtils.dataset(8, 'a', 26); + for (int i = 0; i < 3; i++) { + Path file = getTestPath("test_folder/sub_file-" + i); + FSDataOutputStream outputStream = fs.create(file, true); + outputStream.write(data); + outputStream.close(); + } + + PathFilter filter = path -> !path.toString().contains("dir"); + RemoteIterator iterator = fs.listLocatedStatus( + testFolder, filter); + int entries = 0; + while (iterator.hasNext()) { + entries++; + LocatedFileStatus status = iterator.next(); + assertTrue(status.isFile()); + } + + assertEquals(3, entries); + + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + } + + @Test + // 如果路径不存在或父目录及上级目录不存在,抛FileNotFoundException + public void testListLocatedStatusAbnormal01() throws Exception { + Path testFolder = getTestPath("test_dir"); + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + + boolean hasException = false; + try { + fs.listLocatedStatus(testFolder); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 带filter,如果路径不存在或父目录及上级目录不存在,抛FileNotFoundException + public void testListLocatedStatusAbnormal02() throws Exception { + Path testFolder = getTestPath("test_dir"); + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + + PathFilter filter = new PathFilter() { + @Override + public boolean accept(Path path) { + return false; + } + }; + boolean hasException = false; + try { + fs.listLocatedStatus(testFolder); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 如果路径的父目录及上级目录不是一个文件,抛AccessControlException + public void testListLocatedStatusAbnormal03() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFolder = getTestPath("a001"); + FSDataOutputStream outputStream = fs.create(testFolder, false); + outputStream.close(); + + Path testFile = getTestPath("a001/b001"); + boolean hasException = false; + try { + fs.listLocatedStatus(testFile); + } catch (AccessControlException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 带filter,如果路径的父目录及上级目录不是一个文件,抛AccessControlException + public void testListLocatedStatusAbnormal04() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testFolder = getTestPath("a001"); + FSDataOutputStream outputStream = fs.create(testFolder, false); + outputStream.close(); + + Path testFile = getTestPath("a001/b001"); + PathFilter filter = path -> true; + boolean hasException = false; + try { + fs.listLocatedStatus(testFile, filter); + } catch (AccessControlException e) { + hasException = true; + } + assertTrue(hasException); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListStatus.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListStatus.java new file mode 100644 index 0000000..c5930af --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSListStatus.java @@ -0,0 +1,311 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.PathFilter; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; + +public class ITestOBSListStatus { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024)); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 路径是根目录,FileStatus结果数组长度大于等于0 + public void testListStatus001() throws IOException { + FileStatus[] summaries = fs.listStatus(new Path("/")); + assertTrue(summaries.length >= 0); + } + + @Test + // 路径是一个文件,FileStatus结果数组长度等于1,且为文件 + public void testListStatus002() throws IOException { + Path testFile = getTestPath("test_file"); + fs.delete(testFile, true); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + outputStream.write(data); + outputStream.close(); + + FileStatus[] status = fs.listStatus(testFile); + assertEquals(1, status.length); + assertTrue(status[0].isFile()); + if (fs.exists(testFile)) { + fs.delete(testFile, true); + } + } + + @Test + // 路径是一个空目录,FileStatus结果数组长度等于0 + public void testListStatus003() throws Exception { + Path testFolder = getTestPath("test_dir"); + fs.delete(testFolder, true); + fs.mkdirs(testFolder); + FileStatus[] status = fs.listStatus(testFolder); + assertEquals(0, status.length); + } + + @Test + // 路径是一个非空目录,FileStatus结果数组长度等于子目录数和子文件数之和 + public void testListStatus004() throws IOException { + Path testFolder = getTestPath("test_dir"); + fs.mkdirs(testFolder); + + for (int i = 0; i < 2; i++) { + Path subFolder = getTestPath("test_dir" + "/sub_folder" + i); + fs.mkdirs(subFolder); + } + + byte[] data = ContractTestUtils.dataset(8, 'a', 26); + for (int i = 0; i < 3; i++) { + Path file = getTestPath("test_dir" + "/sub_file" + i); + FSDataOutputStream outputStream = fs.create(file, true); + outputStream.write(data); + outputStream.close(); + } + + FileStatus[] status = fs.listStatus(testFolder); + System.out.println( + "folder " + testFolder + " size: " + status.length); + + assertEquals(2 + 3, status.length); + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + } + + @Test + // 文件不存在,抛FileNotFoundException + public void testListStatus005() throws IOException { + Path testFolder = getTestPath("test_file"); + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + + boolean hasException = false; + try { + fs.listStatus(testFolder); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 文件的父目录及上级目录不存在,抛FileNotFoundException + public void testListStatus006() throws IOException { + Path testFolder = getTestPath("test_file"); + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + + boolean hasException = false; + try { + fs.listStatus(testFolder); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 路径的父目录及上级目录不是一个目录,抛AccessControlException + public void testListStatus007() throws IOException { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + fs.delete(testFile.getParent().getParent(), true); + + FSDataOutputStream outputStream = fs.create(testFile.getParent(), + false); + outputStream.close(); + + boolean hasException = false; + try { + fs.listStatus(testFile); + } catch (AccessControlException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 路径为null, 抛NullPointerException + public void testListStatus008() throws IOException { + Path testFolder = null; + + boolean hasException = false; + try { + fs.listStatus(testFolder); + } catch (NullPointerException e) { + hasException = true; + } + assertTrue(hasException); + } + + @Test + // 测试基类方法listStatus(Path f, PathFilter filter),用filter对返回结果列表进行过滤 + public void testParentMethod01() throws Exception { + Path testPath = getTestPath("test_file"); + final Path qualifiedPath = testPath.makeQualified(fs.getUri(), + fs.getWorkingDirectory()); + FSDataOutputStream outputStream = fs.create(qualifiedPath, false); + outputStream.close(); + + PathFilter filter = path -> { + if (path.equals(qualifiedPath)) { + return true; + } + return false; + }; + + FileStatus[] files = fs.listStatus(new Path(testRootPath), filter); + assertEquals(1, files.length); + assertTrue(files[0].getPath().equals(qualifiedPath)); + + fs.delete(qualifiedPath, true); + } + + @Test + // 测试基类方法listStatus(Path[] files, PathFilter filter),列举多个路径,用filter对返回结果列表进行过滤 + public void testParentMethod02() throws Exception { + Path dir1 = getTestPath("dir1"); + fs.mkdirs(dir1); + final Path file1 = + getTestPath("dir1/test_file1").makeQualified(fs.getUri(), + fs.getWorkingDirectory()); + FSDataOutputStream outputStream = fs.create(file1, false); + outputStream.close(); + + Path dir2 = getTestPath("dir2"); + fs.mkdirs(dir2); + final Path file2 = + getTestPath("dir2/test_file2").makeQualified(fs.getUri(), + fs.getWorkingDirectory()); + ; + outputStream = fs.create(file2, false); + outputStream.close(); + + PathFilter filter = path -> { + if (path.equals(file1) || path.equals(file2)) { + return true; + } + return false; + }; + + Path[] dirs = {dir1, dir2}; + FileStatus[] files = fs.listStatus(dirs, filter); + assertEquals(2, files.length); + assertTrue(files[0].getPath().equals(file1)); + assertTrue(files[1].getPath().equals(file2)); + + fs.delete(dir1, true); + fs.delete(dir2, true); + } + + @Test + // 测试基类方法listStatus(Path[] files),列举多个路径,不对返回结果列表进行过滤 + public void testParentMethod03() throws Exception { + Path dir1 = getTestPath("dir1"); + fs.mkdirs(dir1); + final Path file1 = + getTestPath("dir1/test_file1").makeQualified(fs.getUri(), + fs.getWorkingDirectory()); + FSDataOutputStream outputStream = fs.create(file1, false); + outputStream.close(); + + Path dir2 = getTestPath("dir2"); + fs.mkdirs(dir2); + final Path file2 = + getTestPath("dir2/test_file2").makeQualified(fs.getUri(), + fs.getWorkingDirectory()); + outputStream = fs.create(file2, false); + outputStream.close(); + + Path[] dirs = {dir1, dir2}; + FileStatus[] files = fs.listStatus(dirs); + assertEquals(2, files.length); + assertTrue(files[0].getPath().equals(file1)); + assertTrue(files[1].getPath().equals(file2)); + + fs.delete(dir1, true); + fs.delete(dir2, true); + } + + @Test + // 测试基类方法listStatusIterator(final Path p),列举多个路径,不对返回结果列表进行过滤 + public void testParentMethod04() throws Exception { + Path testFolder = getTestPath("test_dir"); + fs.mkdirs(testFolder); + + for (int i = 0; i < 2; i++) { + Path subFolder = getTestPath("test_dir" + "/sub_folder" + i); + fs.mkdirs(subFolder); + } + + byte[] data = ContractTestUtils.dataset(8, 'a', 26); + for (int i = 0; i < 3; i++) { + Path file = getTestPath("test_dir" + "/sub_file" + i); + FSDataOutputStream outputStream = fs.create(file, true); + outputStream.write(data); + outputStream.close(); + } + + FileStatus[] status = fs.listStatus(testFolder); + System.out.println( + "folder " + testFolder + " size: " + status.length); + + assertEquals(2 + 3, status.length); + + if (fs.exists(testFolder)) { + fs.delete(testFolder, true); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSLoginHelper.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSLoginHelper.java new file mode 100644 index 0000000..b6f684c --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSLoginHelper.java @@ -0,0 +1,257 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.OBSLoginHelper; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; +import java.net.URI; +import java.net.URISyntaxException; + +/** + * Test how URIs and login details are extracted from URIs. + */ +public class ITestOBSLoginHelper extends Assert { + public static final String BUCKET = "obs://bucket"; + + private static final URI ENDPOINT = uri(BUCKET); + + public static final String S = "%2f"; + + public static final String P = "%2b"; + + public static final String P_RAW = "+"; + + public static final String USER = "user"; + + public static final String PASS = "pass"; + + public static final String PASLASHSLASH = "pa" + S + S; + + public static final String PAPLUS = "pa" + P; + + public static final String PAPLUS_RAW = "pa" + P_RAW; + + public static final URI WITH_USER_AND_PASS = uri("obs://user:pass@bucket"); + + public static final Path PATH_WITH_LOGIN = + new Path(uri("obs://user:pass@bucket/dest")); + + public static final URI WITH_SLASH_IN_PASS = uri( + "obs://user:" + PASLASHSLASH + "@bucket"); + + public static final URI WITH_PLUS_IN_PASS = uri( + "obs://user:" + PAPLUS + "@bucket"); + + public static final URI WITH_PLUS_RAW_IN_PASS = uri( + "obs://user:" + PAPLUS_RAW + "@bucket"); + + public static final URI USER_NO_PASS = uri("obs://user@bucket"); + + public static final URI WITH_USER_AND_COLON = uri("obs://user:@bucket"); + + public static final URI NO_USER = uri("obs://:pass@bucket"); + + public static final URI NO_USER_NO_PASS = uri("obs://:@bucket"); + + public static final URI NO_USER_NO_PASS_TWO_COLON = uri("obs://::@bucket"); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + /** + * Construct a URI; raises an RTE if it won't parse. This allows it to be + * used in static constructors. + * + * @param s URI string + * @return the URI + * @throws RuntimeException on a URI syntax problem + */ + private static URI uri(String s) { + try { + return new URI(s); + } catch (URISyntaxException e) { + throw new RuntimeException(e.toString(), e); + } + } + + /** + * Assert that a built up FS URI matches the endpoint. + * + * @param uri URI to build the FS UIR from + */ + private void assertMatchesEndpoint(URI uri) { + assertEquals("Source " + uri, + ENDPOINT, OBSLoginHelper.buildFSURI(uri)); + } + + /** + * Assert that the login/pass details from a URI match that expected. + * + * @param user username + * @param pass password + * @param uri URI to build login details from + * @return the login tuple + */ + private OBSLoginHelper.Login assertMatchesLogin(String user, + String pass, URI uri) { + OBSLoginHelper.Login expected = new OBSLoginHelper.Login(user, + pass); + OBSLoginHelper.Login actual = OBSLoginHelper.extractLoginDetails( + uri); + if (!expected.equals(actual)) { + Assert.fail("Source " + uri + + " login expected=:" + toString(expected) + + " actual=" + toString(actual)); + } + return actual; + } + + @Test + public void testSimpleFSURI() throws Throwable { + assertMatchesEndpoint(ENDPOINT); + } + + @Test + public void testLoginSimple() throws Throwable { + OBSLoginHelper.Login login = assertMatchesLogin("", "", ENDPOINT); + assertFalse("Login of " + login, login.hasLogin()); + } + + @Test + public void testLoginWithUserAndPass() throws Throwable { + OBSLoginHelper.Login login = assertMatchesLogin(USER, PASS, + WITH_USER_AND_PASS); + assertTrue("Login of " + login, login.hasLogin()); + } + + @Test + public void testLoginWithSlashInPass() throws Throwable { + assertMatchesLogin(USER, "pa//", WITH_SLASH_IN_PASS); + } + + @Test + public void testLoginWithPlusInPass() throws Throwable { + assertMatchesLogin(USER, "pa+", WITH_PLUS_IN_PASS); + } + + @Test + public void testLoginWithPlusRawInPass() throws Throwable { + assertMatchesLogin(USER, "pa+", WITH_PLUS_RAW_IN_PASS); + } + + @Test + public void testLoginWithUser() throws Throwable { + assertMatchesLogin(USER, "", USER_NO_PASS); + } + + @Test + public void testLoginWithUserAndColon() throws Throwable { + assertMatchesLogin(USER, "", WITH_USER_AND_COLON); + } + + @Test + public void testLoginNoUser() throws Throwable { + assertMatchesLogin("", "", NO_USER); + } + + @Test + public void testLoginNoUserNoPass() throws Throwable { + assertMatchesLogin("", "", NO_USER_NO_PASS); + } + + @Test + public void testLoginNoUserNoPassTwoColon() throws Throwable { + assertMatchesLogin("", "", NO_USER_NO_PASS_TWO_COLON); + } + + @Test + public void testFsUriWithUserAndPass() throws Throwable { + assertMatchesEndpoint(WITH_USER_AND_PASS); + } + + @Test + public void testFsUriWithSlashInPass() throws Throwable { + assertMatchesEndpoint(WITH_SLASH_IN_PASS); + } + + @Test + public void testFsUriWithPlusInPass() throws Throwable { + assertMatchesEndpoint(WITH_PLUS_IN_PASS); + } + + @Test + public void testFsUriWithPlusRawInPass() throws Throwable { + assertMatchesEndpoint(WITH_PLUS_RAW_IN_PASS); + } + + @Test + public void testFsUriWithUser() throws Throwable { + assertMatchesEndpoint(USER_NO_PASS); + } + + @Test + public void testFsUriWithUserAndColon() throws Throwable { + assertMatchesEndpoint(WITH_USER_AND_COLON); + } + + @Test + public void testFsiNoUser() throws Throwable { + assertMatchesEndpoint(NO_USER); + } + + @Test + public void testFsUriNoUserNoPass() throws Throwable { + assertMatchesEndpoint(NO_USER_NO_PASS); + } + + @Test + public void testFsUriNoUserNoPassTwoColon() throws Throwable { + assertMatchesEndpoint(NO_USER_NO_PASS_TWO_COLON); + } + + @Test + public void testPathURIFixup() throws Throwable { + + } + + /** + * Stringifier. Kept in the code to avoid accidental logging in production + * code. + * + * @return login details for assertions. + */ + public String toString(OBSLoginHelper.Login login) { + final StringBuilder sb = new StringBuilder("LoginTuple{"); + sb.append("<'").append(login.getUser()).append('\''); + sb.append(", '").append(login.getPassword()).append('\''); + sb.append('>'); + return sb.toString(); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStream.java new file mode 100644 index 0000000..b7d2abb --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStream.java @@ -0,0 +1,456 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.obs.input.InputPolicyFactory; +import org.apache.hadoop.fs.obs.input.InputPolicys; +import org.apache.hadoop.fs.obs.input.OBSMemArtsCCInputStream; +import org.apache.hadoop.fs.obs.mock.MockMemArtsCCClient; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Field; +import java.util.Arrays; + +public class ITestOBSMemArtsCCInputStream { + private OBSFileSystem fs; + + private static final Logger LOG = LoggerFactory.getLogger(ITestOBSMemArtsCCInputStream.class); + + private static String testRootPath = OBSTestUtils.generateUniqueTestPath(); + + private MockMemArtsCCClient mockMemArtsCCClient; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout(30 * 60 * 1000); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.set(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_MEMARTSCC); + conf.setInt("fs.obs.memartscc.buffer.size", 1); + conf.set("fs.obs.memartscc.inputstream.buffer.type", "bind"); + this.fs = OBSTestUtils.createTestFileSystem(conf); + + mockMemArtsCCClient = new MockMemArtsCCClient(fs, false, this.fs.getBucket()); + mockMemArtsCCClient.init("", ""); + + InputPolicyFactory inputPolicyFactory = InputPolicys.createFactory(OBSConstants.READAHEAD_POLICY_MEMARTSCC); + + // mock memartscc client + Field ccClient = OBSFileSystem.class.getDeclaredField("memArtsCCClient"); + ccClient.setAccessible(true); + ccClient.set(this.fs, mockMemArtsCCClient); + + // mock input policy factory + Field fInputPolicyFactory = OBSFileSystem.class.getDeclaredField("inputPolicyFactory"); + fInputPolicyFactory.setAccessible(true); + fInputPolicyFactory.set(this.fs, inputPolicyFactory); + } + + private void changeReadAheadRange(long readAheadRange) throws NoSuchFieldException, IllegalAccessException { + Field rRange = OBSFileSystem.class.getDeclaredField("readAheadRange"); + rRange.setAccessible(true); + rRange.set(this.fs, readAheadRange); + return; + } + + private void changeState(OBSMemArtsCCInputStream is, OBSMemArtsCCInputStream.State state) + throws NoSuchFieldException, IllegalAccessException { + Field fstate = OBSMemArtsCCInputStream.class.getDeclaredField("state"); + fstate.setAccessible(true); + fstate.set(is, state); + return; + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path setPath(String path) { + if (path.startsWith("/")) { + return new Path(testRootPath + path); + } else { + return new Path(testRootPath + "/" + path); + } + } + + private static void genTestBuffer(byte[] buffer, long lenth) { + for (int i = 0; i < lenth; i++) { + buffer[i] = (byte) (i % 255); + } + } + + private int fullReadAndCheck(InputStream is, int offset, int size, boolean onebyteRead) throws IOException { + byte[] buf = new byte[size]; + int bytesRead = 0; + int off = 0; + int _byte = 0; + if (onebyteRead) { + while (off < size) { + _byte = is.read(); + if (_byte == -1) { + break; + } + if (_byte >= 0) { + buf[off] = (byte) _byte; + off ++; + } + } + } else { + do { + off += bytesRead; + bytesRead = is.read(buf, off, size - off); + } while (bytesRead > 0); + } + + byte[] testBuffer = new byte[256]; + genTestBuffer(testBuffer, 256); + + byte[] equalSizeBuffer = new byte[(int) size]; + + int start = (offset % 256); + for (int i = 0; i < equalSizeBuffer.length; i++) { + equalSizeBuffer[i] = testBuffer[start % 256]; + start ++; + } + assertTrue(Arrays.equals(buf, equalSizeBuffer)); + return off; + } + + @Test + public void testNew2ORead() throws NoSuchFieldException, IllegalAccessException, IOException { + Path mread2OReadFile = setPath("/test/testNew2ORead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertTrue("input state should be 'New'", mis.getState().equals(OBSMemArtsCCInputStream.State.NEW)); + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + } + + @Test + public void testStayInORead() throws NoSuchFieldException, IllegalAccessException, IOException { + Path mread2OReadFile = setPath("/test/testStayInORead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertTrue("input state should be 'New'", mis.getState().equals(OBSMemArtsCCInputStream.State.NEW)); + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + + // read 256K to stay in ORead + readsize = 256 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should stay in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + } + + @Test + public void testNew2ORead2MRead2ORead() throws NoSuchFieldException, IllegalAccessException, IOException { + Path mread2OReadFile = setPath("/test/testNew2ORead2MRead2ORead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertTrue("input state should be 'New'", mis.getState().equals(OBSMemArtsCCInputStream.State.NEW)); + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + // read 1M to transfer to MRead + readsize = 1 * 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + + // read 1M to stay in MRead + readsize = 1 * 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + + // set cache miss + this.mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + + // read 512KB to transfer to ORead + readsize = 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + + } + + @Test + public void testMRead2ORead() throws NoSuchFieldException, IllegalAccessException, IOException { + Path mread2OReadFile = setPath("/test/testMRead2ORead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.MREAD); + + int readsize = 512 * 1024; + int offset = 0; + + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + + // set cache miss + this.mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + + // read 512KB to transfer to ORead + readsize = 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + } + + @Test + public void testLazySeek2MRead() throws NoSuchFieldException, IllegalAccessException, IOException { + Path mread2OReadFile = setPath("/test/testLazySeek2MRead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertTrue("input state should be 'New'", mis.getState().equals(OBSMemArtsCCInputStream.State.NEW)); + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + // ORead 100 one byte read + readsize = 256; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, true); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + + offset += 2 * 1024 * 1024; + fsDataInputStream.seek(offset); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + + // read 0.5M to transfer to MRead + readsize = 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + + // MRead 100 one byte read + readsize = 256; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, true); + assertTrue(bytesRead == readsize); + offset += bytesRead; + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + } + + @Test + public void testSmallRandomRead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path mread2OReadFile = setPath("/test/testSmallRandomRead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertTrue("input state should be 'New'", mis.getState().equals(OBSMemArtsCCInputStream.State.NEW)); + int offset = 0; + int readsize = 1; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + readsize = 10; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + readsize = 100; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + readsize = 1000; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + readsize = 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + // seek and reopen to mread + offset = 35; + fsDataInputStream.seek(offset); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + + readsize = 100; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + readsize = 1000; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + readsize = 1; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + readsize = 5; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + // another seek + offset = 37 * 1024; + fsDataInputStream.seek(offset); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + + readsize = 5; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + readsize = 1; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + readsize = 29; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + // set cache miss + this.mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + readsize = 10; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + this.mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + readsize = 100; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + } + + @Test + public void testReadBackward() throws IOException, NoSuchFieldException, IllegalAccessException { + Path mread2OReadFile = setPath("/test/testReadBackward.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertTrue("input state should be 'New'", mis.getState().equals(OBSMemArtsCCInputStream.State.NEW)); + int offset = 0; + int readsize = 1 * 1024 * 1024; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + readsize = 1 * 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + this.mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + readsize = 1 * 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + offset = 0; + fsDataInputStream.seek(offset); + readsize = 1 * 1024 * 1024 + 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamBufIO.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamBufIO.java new file mode 100644 index 0000000..9637c40 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamBufIO.java @@ -0,0 +1,401 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.obs.input.InputPolicyFactory; +import org.apache.hadoop.fs.obs.input.InputPolicys; +import org.apache.hadoop.fs.obs.input.OBSMemArtsCCInputStream; +import org.apache.hadoop.fs.obs.mock.MockMemArtsCCClient; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.Timeout; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Field; +import java.util.Arrays; + +public class ITestOBSMemArtsCCInputStreamBufIO { + private OBSFileSystem fs; + + private static final Logger LOG = LoggerFactory.getLogger(ITestOBSMemArtsCCInputStreamBufIO.class); + + private static String testRootPath = OBSTestUtils.generateUniqueTestPath(); + + private MockMemArtsCCClient mockMemArtsCCClient; + + private int bufSize; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout(30 * 60 * 1000); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.set(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_MEMARTSCC); + bufSize = 8192; + conf.setInt("fs.obs.memartscc.buffer.size", bufSize); + conf.set("fs.obs.memartscc.inputstream.buffer.type", "bind"); + this.fs = OBSTestUtils.createTestFileSystem(conf); + + mockMemArtsCCClient = new MockMemArtsCCClient(fs, false, this.fs.getBucket()); + mockMemArtsCCClient.init("", ""); + + InputPolicyFactory inputPolicyFactory = InputPolicys.createFactory(OBSConstants.READAHEAD_POLICY_MEMARTSCC); + + // mock memartscc client + Field ccClient = OBSFileSystem.class.getDeclaredField("memArtsCCClient"); + ccClient.setAccessible(true); + ccClient.set(this.fs, mockMemArtsCCClient); + + // mock input policy factory + Field fInputPolicyFactory = OBSFileSystem.class.getDeclaredField("inputPolicyFactory"); + fInputPolicyFactory.setAccessible(true); + fInputPolicyFactory.set(this.fs, inputPolicyFactory); + } + + private void changeReadAheadRange(long readAheadRange) throws NoSuchFieldException, IllegalAccessException { + Field rRange = OBSFileSystem.class.getDeclaredField("readAheadRange"); + rRange.setAccessible(true); + rRange.set(this.fs, readAheadRange); + return; + } + + private void changeState(OBSMemArtsCCInputStream is, OBSMemArtsCCInputStream.State state) + throws NoSuchFieldException, IllegalAccessException { + Field fstate = OBSMemArtsCCInputStream.class.getDeclaredField("state"); + fstate.setAccessible(true); + fstate.set(is, state); + return; + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private Path setPath(String path) { + if (path.startsWith("/")) { + return new Path(testRootPath + path); + } else { + return new Path(testRootPath + "/" + path); + } + } + + private static void genTestBuffer(byte[] buffer, long lenth) { + for (int i = 0; i < lenth; i++) { + buffer[i] = (byte) (i % 255); + } + } + + private int fullReadAndCheck(InputStream is, int offset, int size, boolean onebyteRead) throws IOException { + byte[] buf = new byte[size]; + int bytesRead = 0; + int off = 0; + int _byte = 0; + if (onebyteRead) { + while (off < size) { + _byte = is.read(); + if (_byte == -1) { + break; + } + if (_byte >= 0) { + buf[off] = (byte) _byte; + off ++; + } + } + } else { + do { + off += bytesRead; + bytesRead = is.read(buf, off, size - off); + } while (bytesRead > 0); + } + + byte[] testBuffer = new byte[256]; + genTestBuffer(testBuffer, 256); + + byte[] equalSizeBuffer = new byte[(int) size]; + + int start = (offset % 256); + for (int i = 0; i < equalSizeBuffer.length; i++) { + equalSizeBuffer[i] = testBuffer[start % 256]; + start ++; + } + assertTrue(Arrays.equals(buf, equalSizeBuffer)); + return off; + } + + @Test + public void testRead1() throws NoSuchFieldException, IllegalAccessException, IOException { + Path mread2OReadFile = setPath("/test/testRead1.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testRead1.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + + readsize = 1 * 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + offset += bytesRead; + + + readsize = 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, true); + assertTrue(bytesRead == readsize); + offset += bytesRead; + + mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + readsize = 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, true); + assertTrue(bytesRead == readsize); + offset += bytesRead; + } + + private void doOnceMixReadTest(Path testFile, int oneByteReadSize) + throws IOException, NoSuchFieldException, IllegalAccessException { + FSDataInputStream fsDataInputStream = this.fs.open(testFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + // one byte read + int readsize = oneByteReadSize; + int offset = 0; + // read 3 byte + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + if (bufSize <= 1024) { + throw new IllegalArgumentException("buffer size should larger than 1K"); + } + // read less than bufferSize + readsize = bufSize - 1024; + + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + // read larger than bufferSize + readsize = bufSize + 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + } + + @Test + public void testMixRead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testMixReadFile = setPath("/test/testMixRead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, testMixReadFile, size, 256, 255); + LOG.info("5MB file created: testMixRead.txt"); + + for (int i = 1; i < 20; i++) { + doOnceMixReadTest(testMixReadFile, i); + } + } + + @Test + public void testMixRead1() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testMixReadFile = setPath("/test/testMixRead1.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, testMixReadFile, size, 256, 255); + LOG.info("5MB file created: testMixRead1.txt"); + + FSDataInputStream fsDataInputStream = this.fs.open(testMixReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + // one byte read + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + if (bufSize <= 1024) { + throw new IllegalArgumentException("buffer size should larger than 1K"); + } + // read less than bufferSize + readsize = bufSize - 1024; + + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + // read larger than bufferSize + readsize = bufSize + 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + } + + @Test + public void testReadTail() throws NoSuchFieldException, IllegalAccessException, IOException { + Path mread2OReadFile = setPath("/test/testReadTail.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, + mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testReadTail.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + + int offset = (size - 8); + fsDataInputStream.seek(offset); + int readsize = 8; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue(bytesRead == readsize); + + offset = (size - 6024); + fsDataInputStream.seek(offset); + readsize = 6024 - 8; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,true); + assertTrue(bytesRead == readsize); + + offset = (size - 6024); + fsDataInputStream.seek(offset); + readsize = 6024 - 8; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertTrue(bytesRead == readsize); + } + + @Test + public void testSeekEscapeSeekRead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testSeekEscapeSeekRead = setPath("/test/testSeekEscapeSeekRead.txt"); + int size = 5 * 1024 * 1024; + ContractTestUtils.generateTestFile(this.fs, testSeekEscapeSeekRead, size, 256, 255); + LOG.info("sim parquet file testSeekEscapeRead.txt in size {} created", size); + FSDataInputStream fsDataInputStream = this.fs.open(testSeekEscapeSeekRead); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.MREAD); + + int offset = 0; + int readsize = 4096; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + readsize = 1 * 1024 * 1024; + offset = 1 * 1024 * 1024; + fsDataInputStream.seek(offset); + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + + offset = 0; + readsize = 512 * 1024; + fsDataInputStream.seek(offset); + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + } + + @Test + public void testSimParquet() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testSimParquetFile = setPath("/test/testSimParquet.txt"); + int size = 137051553; + ContractTestUtils.generateTestFile(this.fs, testSimParquetFile, size, 256, 255); + LOG.info("sim parquet file testSimParquet.txt in size {} created", size); + FSDataInputStream fsDataInputStream = this.fs.open(testSimParquetFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(8 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.MREAD); + + int offset = 27895882; + fsDataInputStream.seek(offset); + int readsize = 22; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, true); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + readsize = 2016; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + offset = 2235733; + readsize = 8388608; + fsDataInputStream.seek(offset); + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == 8 * 1024 * 1024); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + readsize = 3005571; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in ORead State", mis.getState().equals(OBSMemArtsCCInputStream.State.OREAD)); + offset += bytesRead; + + offset = 27895882; + readsize = 2328035; + fsDataInputStream.seek(offset); + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + + offset = 127584000; + readsize = 9467553; + fsDataInputStream.seek(offset); + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertTrue(bytesRead == readsize); + assertTrue("should in MRead State", mis.getState().equals(OBSMemArtsCCInputStream.State.MREAD)); + offset += bytesRead; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamStatistics.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamStatistics.java new file mode 100644 index 0000000..d053100 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamStatistics.java @@ -0,0 +1,44 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; + +import org.junit.Test; + +public class ITestOBSMemArtsCCInputStreamStatistics extends ITestOBSMemArtsCCInputStreamStatisticsTestBase { + + @Test + public void testNew2ORead2MRead2ORead() throws Exception { + runNew2ORead2MRead2ORead(); + long bytesRead = this.fs.getSchemeStatistics().getBytesRead(); + assertEquals(3145728, bytesRead); + + tearDownFS(); + } + + @Test + public void testLazySeek2MRead() throws Exception { + runLazySeek2MRead(); + long bytesRead = this.fs.getSchemeStatistics().getBytesRead(); + assertEquals(1049088, bytesRead); + + tearDownFS(); + } + + @Test + public void testBackSeekAndRead() throws Exception { + runBackSeekAndRead(); + long bytesRead = this.fs.getSchemeStatistics().getBytesRead(); + assertEquals(1048576, bytesRead); + + tearDownFS(); + } + + @Test + public void testSeekAndRead() throws Exception { + runSeekAndRead(); + long bytesRead = this.fs.getSchemeStatistics().getBytesRead(); + assertEquals(1048576, bytesRead); + + tearDownFS(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamStatisticsTestBase.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamStatisticsTestBase.java new file mode 100644 index 0000000..48d2144 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamStatisticsTestBase.java @@ -0,0 +1,389 @@ +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.obs.OBSTestUtils.disableFilesystemCaching; +import static org.apache.hadoop.fs.obs.TrafficStatistics.TrafficType.Q; +import static org.apache.hadoop.fs.obs.TrafficStatistics.TrafficType.Q1; +import static org.apache.hadoop.fs.obs.TrafficStatistics.TrafficType.Q2; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.obs.input.InputPolicyFactory; +import org.apache.hadoop.fs.obs.input.InputPolicys; +import org.apache.hadoop.fs.obs.input.OBSMemArtsCCInputStream; +import org.apache.hadoop.fs.obs.mock.MockMemArtsCCClient; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.rules.Timeout; +import org.mockito.Mockito; +import org.powermock.reflect.Whitebox; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.ByteArrayOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.PrintStream; +import java.lang.reflect.Field; +import java.lang.reflect.Method; + +public class ITestOBSMemArtsCCInputStreamStatisticsTestBase { + protected static final Logger LOG = LoggerFactory.getLogger(ITestOBSMemArtsCCInputStreamTrafficReport.class); + + private static final String testRootPath = OBSTestUtils.generateUniqueTestPath(); + + // redirect System.out for testing + protected final ByteArrayOutputStream outContent = new ByteArrayOutputStream(); + + private final PrintStream originalOut = System.out; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Rule + public Timeout testTimeout = new Timeout(30 * 60 * 1000); + + protected OBSFileSystem fs; + + protected MockMemArtsCCClient mockMemArtsCCClient; + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + private static void genTestBuffer(byte[] buffer, long length) { + for (int i = 0; i < length; i++) { + buffer[i] = (byte) (i % 255); + } + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.set(OBSConstants.READAHEAD_POLICY, OBSConstants.READAHEAD_POLICY_MEMARTSCC); + conf.set("fs.obs.memartscc.inputstream.buffer.type", "pool"); + conf.setBoolean(OBSConstants.MEMARTSCC_TRAFFIC_REPORT_ENABLE, true); + fs = OBSTestUtils.createTestFileSystem(conf); + + mockMemArtsCCClient = new MockMemArtsCCClient(fs, false, this.fs.getBucket()); + mockMemArtsCCClient.init("", ""); + + InputPolicyFactory inputPolicyFactory = InputPolicys.createFactory(OBSConstants.READAHEAD_POLICY_MEMARTSCC); + + // mock memartscc client + Field ccClient = OBSFileSystem.class.getDeclaredField("memArtsCCClient"); + ccClient.setAccessible(true); + ccClient.set(this.fs, mockMemArtsCCClient); + + Method method = OBSFileSystem.class.getDeclaredMethod("initTrafficReport", Configuration.class); + method.setAccessible(true); + method.invoke(this.fs, conf); + + // mock input policy factory + Field fInputPolicyFactory = OBSFileSystem.class.getDeclaredField("inputPolicyFactory"); + fInputPolicyFactory.setAccessible(true); + fInputPolicyFactory.set(this.fs, inputPolicyFactory); + + // redirect System.out + System.setOut(new PrintStream(outContent)); + } + + protected void changeReadAheadRange(long readAheadRange) throws NoSuchFieldException, IllegalAccessException { + Field rRange = OBSFileSystem.class.getDeclaredField("readAheadRange"); + rRange.setAccessible(true); + rRange.set(this.fs, readAheadRange); + } + + protected void changeState(OBSMemArtsCCInputStream is, OBSMemArtsCCInputStream.State state) + throws NoSuchFieldException, IllegalAccessException { + Field fstate = OBSMemArtsCCInputStream.class.getDeclaredField("state"); + fstate.setAccessible(true); + fstate.set(is, state); + } + + public void tearDownFS() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.getSchemeStatistics().reset(); + fs.close(); + } + } + + @After + public void tearDown() throws Exception { + // restore System.out + System.setOut(originalOut); + System.out.println(outContent); + try { + tearDownFS(); + } catch (IOException ignored) { + } + } + + protected Path setPath(String path) { + if (path.startsWith("/")) { + return new Path(testRootPath + path); + } else { + return new Path(testRootPath + "/" + path); + } + } + + protected int fullReadAndCheck(InputStream is, int offset, int size, boolean onebyteRead) throws IOException { + byte[] buf = new byte[size]; + int bytesRead = 0; + int off = 0; + int _byte = 0; + if (onebyteRead) { + while (off < size) { + _byte = is.read(); + if (_byte == -1) { + break; + } + if (_byte >= 0) { + buf[off] = (byte) _byte; + off++; + } + } + } else { + do { + off += bytesRead; + bytesRead = is.read(buf, off, size - off); + } while (bytesRead > 0); + } + + byte[] testBuffer = new byte[256]; + ITestOBSMemArtsCCInputStreamStatisticsTestBase.genTestBuffer(testBuffer, 256); + + byte[] equalSizeBuffer = new byte[(int) size]; + + int start = (offset % 256); + for (int i = 0; i < equalSizeBuffer.length; i++) { + equalSizeBuffer[i] = testBuffer[start % 256]; + start++; + } + assertArrayEquals(buf, equalSizeBuffer); + return off; + } + + protected void runNew2ORead2MRead2ORead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path mread2OReadFile = setPath("/test/testNew2ORead2MRead2ORead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertEquals("input state should be 'New'", mis.getState(), OBSMemArtsCCInputStream.State.NEW); + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + assertEquals("should in ORead State", mis.getState(), OBSMemArtsCCInputStream.State.OREAD); + // read 1M to transfer to MRead + readsize = 1 * 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + assertEquals("should in MRead State", mis.getState(), OBSMemArtsCCInputStream.State.MREAD); + + // read 1M to stay in MRead + readsize = 1 * 1024 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + assertEquals("should in MRead State", mis.getState(), OBSMemArtsCCInputStream.State.MREAD); + + // set cache miss + this.mockMemArtsCCClient.setNextCCReadReturnCacheMiss(); + + // read 512KB to transfer to ORead + readsize = 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + assertEquals("should in ORead State", mis.getState(), OBSMemArtsCCInputStream.State.OREAD); + } + + protected void runLazySeek2MRead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path mread2OReadFile = setPath("/test/testLazySeek2MRead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, mread2OReadFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(mread2OReadFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertEquals("input state should be 'New'", mis.getState(), OBSMemArtsCCInputStream.State.NEW); + int readsize = 512 * 1024; + int offset = 0; + // read 0.5M to open ORead stream + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize,false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + assertEquals("should in ORead State", mis.getState(), OBSMemArtsCCInputStream.State.OREAD); + + readsize = 256; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, true); + assertEquals(bytesRead, readsize); + offset += bytesRead; + assertEquals("should in ORead State", mis.getState(), OBSMemArtsCCInputStream.State.OREAD); + + offset += 2 * 1024 * 1024; + fsDataInputStream.seek(offset); + assertEquals("should in ORead State", mis.getState(), OBSMemArtsCCInputStream.State.OREAD); + + // read 0.5M to transfer to MRead + readsize = 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + assertEquals("should in MRead State", mis.getState(), OBSMemArtsCCInputStream.State.MREAD); + + readsize = 256; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, true); + assertEquals(bytesRead, readsize); + offset += bytesRead; + assertEquals("should in ORead State", mis.getState(), OBSMemArtsCCInputStream.State.MREAD); + } + + protected void runBackSeekAndRead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testFile = setPath("/test/testBackSeekAndRead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, testFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(testFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertEquals("input state should be 'New'", mis.getState(), OBSMemArtsCCInputStream.State.NEW); + int offset = 2 * 1024 * 1024; + fsDataInputStream.seek(offset); + + int readsize = 512 * 1024; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + assertEquals("should in OREAD State", mis.getState(), OBSMemArtsCCInputStream.State.OREAD); + + // Seek to original position and read + fsDataInputStream.seek(offset); + readsize = 512 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + } + + protected void runSeekAndRead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testFile = setPath("/test/testSeekAndRead.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, testFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(testFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertEquals("input state should be 'New'", mis.getState(), OBSMemArtsCCInputStream.State.NEW); + int offset = 0; + + int readsize = 256 * 1024; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + assertEquals("should in OREAD State", mis.getState(), OBSMemArtsCCInputStream.State.OREAD); + + offset += bytesRead + 512 * 1024; + fsDataInputStream.seek(offset); + readsize = 256 * 1024; + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + } + + protected void runTailRead() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testFile = setPath("/test/testSmallFileRead.txt"); + int size = 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, testFile, size, 256, 255); + LOG.info("5MB file created: testMRead2ORead.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(testFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertEquals("input state should be 'New'", mis.getState(), OBSMemArtsCCInputStream.State.NEW); + // Seek to Tail position + int offset = size - 8 * 1024; + fsDataInputStream.seek(offset); + + int readsize = 1 * 1024; + int bytesRead = 0; + + for (int i = 0; i < 8; i++) { + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + offset += bytesRead; + assertEquals(bytesRead, readsize); + } + + offset -= 8 * 1024; + fsDataInputStream.seek(offset); + for (int i = 0; i < 8; i++) { + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + offset += bytesRead; + assertEquals(bytesRead, readsize); + } + } + + protected void runStatisticsOffLimit() throws IOException, NoSuchFieldException, IllegalAccessException { + Path testFile = setPath("/test/testStatisticsOffLimit.txt"); + int size = 5 * 1024 * 1024; + + ContractTestUtils.generateTestFile(this.fs, testFile, size, 256, 255); + LOG.info("5MB file created: testStatisticsOffLimit.txt"); + FSDataInputStream fsDataInputStream = this.fs.open(testFile); + OBSMemArtsCCInputStream mis = (OBSMemArtsCCInputStream) fsDataInputStream.getWrappedStream(); + + // Change statistics to its max limit + this.fs.getTrafficStatistics().increase(Long.MAX_VALUE, Q); + this.fs.getTrafficStatistics().increase(Long.MAX_VALUE, Q1); + this.fs.getTrafficStatistics().increase(Long.MAX_VALUE, Q2); + + changeReadAheadRange(1 * 1024 * 1024); + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + assertEquals("input state should be 'New'", mis.getState(), OBSMemArtsCCInputStream.State.NEW); + int offset = 0; + + int readsize = 1 * 1024 * 1024; + int bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + + changeState(mis, OBSMemArtsCCInputStream.State.NEW); + bytesRead = fullReadAndCheck(fsDataInputStream, offset, readsize, false); + assertEquals(bytesRead, readsize); + offset += bytesRead; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamTrafficReport.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamTrafficReport.java new file mode 100644 index 0000000..53671f5 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMemArtsCCInputStreamTrafficReport.java @@ -0,0 +1,75 @@ +package org.apache.hadoop.fs.obs; + +import org.junit.Test; + +import static org.junit.Assert.assertEquals; + +public class ITestOBSMemArtsCCInputStreamTrafficReport extends ITestOBSMemArtsCCInputStreamStatisticsTestBase { + + @Test + public void testNew2ORead2MRead2ORead() throws Exception { + runNew2ORead2MRead2ORead(); + + tearDownFS(); + mockMemArtsCCClient.printTotalStatistics(); + String[] outputArray = outContent.toString().split("\n"); + assertEquals("Total: Q:3145728 Q`:3670015 Q2:1572863 Q1:2097152", + outputArray[outputArray.length - 1]); + } + + @Test + public void testLazySeek2MRead() throws Exception { + runLazySeek2MRead(); + + tearDownFS(); + mockMemArtsCCClient.printTotalStatistics(); + String[] outputArray = outContent.toString().split("\n"); + assertEquals("Total: Q:2097152 Q`:1573120 Q2:524544 Q1:1048576", + outputArray[outputArray.length - 1]); + } + + @Test + public void testBackSeekAndRead() throws Exception { + runBackSeekAndRead(); + + tearDownFS(); + mockMemArtsCCClient.printTotalStatistics(); + String[] outputArray = outContent.toString().split("\n"); + assertEquals("Total: Q:2097152 Q`:1572864 Q2:524288 Q1:1048576", + outputArray[outputArray.length - 1]); + } + + @Test + public void testSeekAndRead() throws Exception { + runSeekAndRead(); + + tearDownFS(); + mockMemArtsCCClient.printTotalStatistics(); + String[] outputArray = outContent.toString().split("\n"); + assertEquals("Total: Q:1048576 Q`:1048576 Q2:0 Q1:1048576", + outputArray[outputArray.length - 1]); + } + + @Test + public void testTailRead() throws Exception { + runTailRead(); + + tearDownFS(); + mockMemArtsCCClient.printTotalStatistics(); + String[] outputArray = outContent.toString().split("\n"); + assertEquals("Total: Q:8192 Q`:8192 Q2:0 Q1:8192", + outputArray[outputArray.length - 1]); + } + + @Test + public void testStatisticsOffLimit() throws Exception { + runStatisticsOffLimit(); + + tearDownFS(); + mockMemArtsCCClient.printTotalStatistics(); + String[] outputArray = outContent.toString().split("\n"); + assertEquals("Total: Q:3145728 Q`:2097152 Q2:1048576 Q1:1048576", + outputArray[outputArray.length - 1]); + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMetricInfo.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMetricInfo.java new file mode 100644 index 0000000..b03ffd0 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMetricInfo.java @@ -0,0 +1,320 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.obs.OBSConstants.FAST_UPLOAD_BYTEBUFFER; +import static org.junit.Assert.*; +import static org.mockito.Matchers.anyObject; + +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.ContentSummary; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.security.AccessControlException; +import org.junit.*; +import org.mockito.Mockito; +import org.powermock.reflect.Whitebox; + +import java.io.FileNotFoundException; +import java.io.IOException; +import java.util.EnumSet; + +public class ITestOBSMetricInfo { + private OBSFileSystem fs; + + private Configuration conf; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static final String TEST_FILE = "dest_file"; + + private OBSFileSystem mockFs; + + private ObsClient obsClient; + + private ObsClient mockObsClient; + + private Path testFile = getTestPath("testFile"); + + private Path testDir = getTestPath("testDir"); + + private static byte[] dataSet = ContractTestUtils.dataset(16, 0, 10); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + conf.setBoolean(OBSConstants.OBS_CONTENT_SUMMARY_ENABLE, true); + conf.setBoolean(OBSConstants.OBS_CLIENT_DFS_LIST_ENABLE, true); + conf.set(OBSConstants.FAST_UPLOAD_BUFFER, FAST_UPLOAD_BYTEBUFFER); + fs = OBSTestUtils.createTestFileSystem(conf); + obsClient = fs.getObsClient(); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + + initTestEnv(); + + initMock(); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + } + } + + private void initTestEnv() throws Exception { + FSDataOutputStream outputStream = fs.create(testFile, true); + outputStream.write(dataSet); + outputStream.close(); + assertTrue(fs.mkdirs(testDir)); + if (!fs.exists(testDir)) { + throw new IOException("testDir not exist!"); + } + } + + private void initMock() { + mockFs = Mockito.spy(fs); + mockObsClient = Mockito.spy(obsClient); + Whitebox.setInternalState(mockFs, obsClient, mockObsClient); + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // open一个文件,获取fileStatus的len + public void testOpen003() throws Exception { + if (fs.getMetricSwitch()) { + Path testFile = getTestPath(TEST_FILE); + FSDataOutputStream outputStream = fs.create(testFile, true); + OBSFSTestUtil.writeData(outputStream, 1024); + outputStream.close(); + + try { + fs.open(testFile, 2048); + } catch (IOException e) { + e.printStackTrace(); + + } + //getFileStatus正常监控指标 + fs.getFileStatus(testFile); + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc.getMr().getKind()); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + } + } + + @Test + // 测试MetricGetFileStatus + public void testGetFileStatus() throws IOException { + OBSFileConflictException fileConflictException = new OBSFileConflictException( + "mock FileConflictException"); + FileNotFoundException fileNotFoundException = new FileNotFoundException( + "mock FileNotFoundException"); + AccessControlException accessControlException = + new AccessControlException("mock AccessControlException"); + OBSIOException obsioException = new OBSIOException("mock IOException", + new ObsException("mock ObsException")); + + Mockito .doThrow(fileConflictException) + .doCallRealMethod() + .when(mockFs) + .innerGetFileStatus(anyObject()); + try { + mockFs.getFileStatus(testFile); + } + catch (FileNotFoundException e) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + } + } + + @Test + // create 路径的父目录是文件,抛出ParentNotDirectoryException + public void testCreateAbnormal001() throws Exception { + if (fs.getMetricSwitch()) { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + Path parentPath = testFile.getParent(); + + FSDataOutputStream outputStream = null; + boolean hasException = false; + try { + outputStream = OBSFSTestUtil.createStream(fs, parentPath); + outputStream.close(); + fs.create(testFile, new FsPermission((short) 00644), + false, fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(testFile), + fs.getDefaultBlockSize(testFile), null); + + } catch (ParentNotDirectoryException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent()); + } + } + + @Test + // create的文件已存在,抛出FileAlreadyExistsException + public void testCreateAbnormal002() throws Exception { + if (fs.getMetricSwitch()) { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + + FSDataOutputStream outputStream = null; + try { + outputStream = OBSFSTestUtil.createStream(fs, testFile, false); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + boolean hasException = false; + try { + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile.getParent(), + new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + } catch (FileAlreadyExistsException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + assertTrue(hasException); + + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + } + + @Test + // createNonRecursive 路径的父目录是文件,抛出ParentNotDirectoryException + public void testCreateAbnormal003() throws Exception { + if (fs.getMetricSwitch()) { + if (!fs.isFsBucket()) { + return; + } + Path testFile = getTestPath("a001/b001/test_file"); + Path parentPath = testFile.getParent(); + + FSDataOutputStream outputStream = null; + boolean hasException = false; + try { + outputStream = OBSFSTestUtil.createStream(fs, parentPath); + outputStream.close(); + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + outputStream = fs.create(testFile, + new FsPermission((short) 00644), + flags, 4096, (short) 3, + 128 * 1024 * 1024, null, null); + } catch (ParentNotDirectoryException e) { + hasException = true; + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + + assertTrue(hasException); + OBSFSTestUtil.deletePathRecursive(fs, testFile.getParent()); + } + } + + @Test + public void testDeleteNormal001() throws Exception { + if (fs.getMetricSwitch()) { + if (!fs.isFsBucket()) { + return; + } + Path testPath = getTestPath("a001/b001/test_file"); + fs.delete(testPath.getParent(), true); + FSDataOutputStream outputStream = null; + try { + outputStream = fs.create(testPath, + new FsPermission((short) 00644), + false, 4096, + (short) 3, 128 * 1024 * 1024, null); + OBSFSTestUtil.writeData(outputStream, 1 * 1024 * 1024); + outputStream.hsync(); + fs.delete(testPath, true); + String position = String.valueOf(1 * 1024 * 1024); + MockMetricsConsumer mmc2 = + (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc2.getMr().getKind()); + } catch (Exception e) { + e.printStackTrace(); + } finally { + if (outputStream != null) { + outputStream.close(); + } + } + } + + } + + //FileNotFoundException + @Test + public void testDeleteNormal002() { + if (fs.getMetricSwitch()) { + try { + fs.delete(new Path(testRootPath), true); + fs.delete(new Path(testRootPath), true); + } catch (IOException e) { + + } + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc.getMr().getKind()); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMkdirs.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMkdirs.java new file mode 100644 index 0000000..0950aee --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMkdirs.java @@ -0,0 +1,191 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileAlreadyExistsException; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +public class ITestOBSMkdirs { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + + fs.delete(new Path(testRootPath), true); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + fs = null; + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // 创建一级目录,返回true + public void testMkdirNormal001() throws Exception { + Path testDir = new Path("/test_dir"); + fs.delete(testDir, true); + + boolean res = fs.mkdirs(testDir, new FsPermission((short) 00644)); + fs.delete(testDir, true); + } + + @Test + // 创建多级目录,父目录存在,返回true + public void testMkdirNormal002() throws Exception { + Path testDir = getTestPath("test_dir"); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + fs.delete(testDir, true); + } + + @Test + // 创建多级目录,父目录及上级目录不存在,返回true + public void testMkdirNormal003() throws Exception { + Path testDir = getTestPath("a001/b001/test_dir"); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.exists(testDir.getParent())); + assertTrue(fs.exists(testDir.getParent().getParent())); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir.getParent().getParent(), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + fs.delete(testDir.getParent().getParent(), true); + } + + @Test + // 创建多级目录,父目录及上级目录不存在,创建各级父目录,并携带permission,返回true + public void testMkdirNormal004() throws Exception { + Path testDir = getTestPath("a001/b001/test_dir"); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + assertTrue(fs.exists(testDir.getParent())); + assertEquals(fs.getFileStatus(testDir).getPermission(), + fs.getFileStatus(testDir.getParent()).getPermission()); + assertTrue(fs.exists(testDir.getParent().getParent())); + assertEquals(fs.getFileStatus(testDir).getPermission(), + fs.getFileStatus(testDir.getParent().getParent()).getPermission()); + fs.delete(testDir, true); + } + + @Test + // 创建多级目录,父目录及上级目录不存在,返回true + public void testMkdirNormal005() throws Exception { + Path testDir = getTestPath("a001/b001/test_dir"); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + fs.delete(testDir, true); + + assertTrue(fs.mkdirs(testDir.getParent().getParent(), + new FsPermission((short) 00644))); + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + fs.delete(testDir.getParent().getParent(), true); + } + + @Test + // 路径存在并且是一个目录,返回true + public void testMkdirNormal006() throws Exception { + Path testDir = getTestPath("a001/b001/test_dir"); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + + assertTrue(fs.mkdirs(testDir, new FsPermission((short) 00644))); + fs.delete(testDir.getParent().getParent(), true); + } + + @Test + // 路径已存在同名文件,抛FileAlreadyExistException + public void testMkdirAbnormal001() throws Exception { + Path testDir = getTestPath("test_dir"); + + FSDataOutputStream stream = fs.create(testDir, false); + byte[] data = ContractTestUtils.dataset(16, 0, 16); + stream.write(data); + stream.close(); + assertTrue(fs.getFileStatus(testDir).isFile()); + + boolean hasException = false; + try { + fs.mkdirs(testDir, new FsPermission((short) 00644)); + } catch (FileAlreadyExistsException e) { + hasException = true; + } + assertTrue(hasException); + fs.delete(testDir, true); + } + + @Test + // 路径的父目录及上级目录不是一个目录,抛ParentNotDirectoryException + public void testMkdirAbnormal002() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testDir = getTestPath("a001/b001/test_dir"); + + FSDataOutputStream stream = fs.create(testDir.getParent(), false); + byte[] data = ContractTestUtils.dataset(16, 0, 16); + stream.write(data); + stream.close(); + assertTrue(fs.getFileStatus(testDir.getParent()).isFile()); + + boolean hasException = false; + try { + fs.mkdirs(testDir, new FsPermission((short) 00644)); + } catch (ParentNotDirectoryException e) { + hasException = true; + } + assertTrue(hasException); + fs.delete(testDir.getParent().getParent(), true); + + stream = fs.create(testDir.getParent().getParent(), false); + stream.write(data); + stream.close(); + + hasException = false; + try { + fs.mkdirs(testDir, new FsPermission((short) 00644)); + } catch (ParentNotDirectoryException e) { + hasException = true; + } + assertTrue(hasException); + fs.delete(testDir.getParent().getParent(), true); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMultiDeleteObjects.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMultiDeleteObjects.java new file mode 100644 index 0000000..eebba9d --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSMultiDeleteObjects.java @@ -0,0 +1,224 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +public class ITestOBSMultiDeleteObjects { + private OBSFileSystem fs; + + private int testBufferSize; + + private int modulus; + + private byte[] testBuffer; + + private int maxEntriesToDelete; + + private static final int DEFAULT_MULTI_DELETE_THRESHOLD = 3; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setInt(OBSConstants.MULTI_DELETE_MAX_NUMBER, 1000); + maxEntriesToDelete = 1000; + fs = OBSTestUtils.createTestFileSystem(conf); + testBufferSize = fs.getConf().getInt( + ContractTestUtils.IO_CHUNK_BUFFER_SIZE, 128); + modulus = fs.getConf().getInt( + ContractTestUtils.IO_CHUNK_MODULUS_SIZE, 128); + testBuffer = new byte[testBufferSize]; + + for (int i = 0; i < testBuffer.length; ++i) { + testBuffer[i] = (byte) (i % modulus); + } + } + + private Path getTestPath(String testPath) { + return new Path(testRootPath + "/" + testPath); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + private void constructTestData(Path path, int subDirNum, int childFileNum) + throws IOException { + if (fs.exists(path)) { + fs.delete(path, true); + } + ContractTestUtils.assertPathDoesNotExist(fs, + path.toString() + "should not exist.", path); + + fs.mkdirs(path); + String directory = path.toString(); + if (!directory.endsWith("/")) { + directory = directory + "/"; + } + + for (int i = 0; i < subDirNum; i++) { + fs.mkdirs(new Path(directory + "subDir-" + i)); + } + + for (int j = 0; j < childFileNum; j++) { + FSDataOutputStream outputStream = fs.create( + new Path(directory + "childFile-" + j)); + outputStream.write(testBuffer); + outputStream.close(); + } + } + + private long testMultiDelete(String testPathStr, int subDirNum, + int childFileNum) throws IOException { + Path testPath = getTestPath(testPathStr); + constructTestData(testPath, subDirNum, childFileNum); + long startTime = System.nanoTime(); + fs.delete(testPath, true); + long endTime = System.nanoTime(); + ContractTestUtils.assertPathDoesNotExist(fs, + testPath.toString() + " should not exist.", testPath); + + return (endTime - startTime) / 1000; + } + + @Test + // 一级目录包含10个子目录,批删一级目录 + public void testMultiDelete001() throws IOException { + int dirNum = maxEntriesToDelete / 100; + int fileNum = 0; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 一级目录包含10个子目录,每个目录包含10个子文件,批删一级目录 + public void testMultiDelete002() throws IOException { + int dirNum = maxEntriesToDelete / 100; + int fileNum = dirNum; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 一级目录包含100个子目录,批删一级目录 + public void testMultiDelete003() throws IOException { + int dirNum = maxEntriesToDelete / 10; + int fileNum = 0; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 一级目录包含100个子目录,每个目录包含100个子文件,批删一级目录 + public void testMultiDelete004() throws IOException { + int dirNum = maxEntriesToDelete / 10; + int fileNum = dirNum; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 一级目录包含1000个子目录,批删一级目录 + public void testMultiDelete005() throws IOException { + int dirNum = maxEntriesToDelete; + int fileNum = 0; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 一级目录包含1010个子目录,每个目录包含10个子文件,批删一级目录 + public void testMultiDelete006() throws IOException { + int dirNum = maxEntriesToDelete + maxEntriesToDelete / 100; + int fileNum = maxEntriesToDelete / 100; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 一级目录包含2000个子目录,批删一级目录 + public void testMultiDelete007() throws IOException { + int dirNum = 2 * maxEntriesToDelete; + int fileNum = 0; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 一级目录包含2010个子目录,每个目录包含10个子文件,批删一级目录 + public void testMultiDelete008() throws IOException { + int dirNum = 2 * maxEntriesToDelete + maxEntriesToDelete / 100; + int fileNum = maxEntriesToDelete / 100; + testMultiDelete("testMultiDelete-" + dirNum + "-" + fileNum, dirNum, + fileNum); + } + + @Test + // 不同批删门禁时,一级目录包含3个子目录,每个目录包含3个子文件,删除一级目录时间对比 + public void testMultiDeleteThreshold() throws IOException { + Configuration conf = fs.getConf(); + int defaultThreshold = conf.getInt(OBSConstants.MULTI_DELETE_THRESHOLD, + DEFAULT_MULTI_DELETE_THRESHOLD); + conf.setInt(OBSConstants.MULTI_DELETE_THRESHOLD, 2 * defaultThreshold); + + int dirNum = defaultThreshold - 1; + int fileNum = defaultThreshold - 1; + long timeUsedDefaultThreshold = testMultiDelete( + "testMultiDeleteThreshold-" + dirNum + "-" + fileNum, + dirNum, fileNum); + + conf.setInt(OBSConstants.MULTI_DELETE_THRESHOLD, 4 * defaultThreshold); + long timeUsedThreshold = testMultiDelete( + "testMultiDeleteThreshold-" + dirNum + "-" + fileNum, + dirNum, fileNum); + + System.out.println( + String.format( + "double default threshold: %d, timeUsed(us): %s; threshold: %d, timeUsed(us): %s.", + 2 * defaultThreshold, timeUsedDefaultThreshold, + 4 * defaultThreshold, timeUsedThreshold)); + // assertTrue(timeUsedThreshold < timeUsedDefaultThreshold); + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSOpen.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSOpen.java new file mode 100644 index 0000000..5884897 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSOpen.java @@ -0,0 +1,239 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class ITestOBSOpen { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private static final String TEST_FILE = "dest_file"; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + OBSFSTestUtil.deletePathRecursive(fs, new Path(testRootPath)); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + @Test + // open一个文件发起range读,seek到开始读一段 + public void testOpenNormal001() throws Exception { + Path testFile = getTestPath(TEST_FILE); + FSDataOutputStream outputStream = fs.create(testFile, true); + OBSFSTestUtil.writeData(outputStream, 20 * 1024 * 1024); + outputStream.close(); + OBSFSTestUtil.assertFileHasLength(fs, testFile, 20 * 1024 * 1024); + + boolean hasException = false; + try { + OBSFSTestUtil.readFile(fs, testFile, 0, 5 * 1024 * 1024); + } catch (IOException e) { + hasException = true; + } + + assertFalse(hasException); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + } + + @Test + // open一个文件发起range读,seek到开始读全部 + public void testOpenNormal002() throws Exception { + Path testFile = getTestPath(TEST_FILE); + FSDataOutputStream outputStream = fs.create(testFile, true); + OBSFSTestUtil.writeData(outputStream, 20 * 1024 * 1024); + outputStream.close(); + OBSFSTestUtil.assertFileHasLength(fs, testFile, 20 * 1024 * 1024); + + boolean hasException = false; + try { + OBSFSTestUtil.readFile(fs, testFile, 0, 20 * 1024 * 1024); + } catch (IOException e) { + hasException = true; + } + + assertFalse(hasException); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + } + + @Test + // open一个文件发起range读,seek到中间读一段 + public void testOpenNormal003() throws Exception { + Path testFile = getTestPath(TEST_FILE); + FSDataOutputStream outputStream = fs.create(testFile, true); + OBSFSTestUtil.writeData(outputStream, 20 * 1024 * 1024); + outputStream.close(); + OBSFSTestUtil.assertFileHasLength(fs, testFile, 20 * 1024 * 1024); + + boolean hasException = false; + try { + OBSFSTestUtil.readFile(fs, testFile, 10 * 1024 * 1024, + 5 * 1024 * 1024); + } catch (IOException e) { + hasException = true; + } + + assertFalse(hasException); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + } + + @Test + // open一个文件发起range读,seek到中间读至末尾 + public void testOpenNormal004() throws Exception { + Path testFile = getTestPath(TEST_FILE); + FSDataOutputStream outputStream = fs.create(testFile, true); + OBSFSTestUtil.writeData(outputStream, 20 * 1024 * 1024); + outputStream.close(); + OBSFSTestUtil.assertFileHasLength(fs, testFile, 20 * 1024 * 1024); + + boolean hasException = false; + try { + OBSFSTestUtil.readFile(fs, testFile, 10 * 1024 * 1024, + 10 * 1024 * 1024); + } catch (IOException e) { + hasException = true; + } + + assertFalse(hasException); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + } + + @Test + // open的文件不存在,抛FileNotFoundException + public void testOpenAbnormal001() throws Exception { + Path testFile = getTestPath(TEST_FILE); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + OBSFSTestUtil.assertPathExistence(fs, testFile, false); + + boolean hasException = false; + FSDataInputStream inputStream = null; + try { + inputStream = fs.open(testFile, 4096); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + assertTrue(hasException); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath(TEST_FILE)); + } + + @Test + // open的是一个目录,抛FileNotFoundException + public void testOpenAbnormal002() throws Exception { + Path testDir = getTestPath("a/b/c"); + fs.mkdirs(testDir); + OBSFSTestUtil.assertPathExistence(fs, testDir, true); + + boolean hasException = false; + FSDataInputStream inputStream = null; + try { + inputStream = fs.open(testDir, 4096); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + assertTrue(hasException); + OBSFSTestUtil.deletePathRecursive(fs, getTestPath("a/b/c")); + } + + @Test + // open的文件父目录及上级目录不存在,抛FileNotFoundException + public void testOpenAbnormal003() throws Exception { + Path testPath = getTestPath("a/b/c/d"); + OBSFSTestUtil.deletePathRecursive(fs, testPath.getParent().getParent()); + OBSFSTestUtil.assertPathExistence(fs, testPath.getParent().getParent(), + false); + + boolean hasException = false; + FSDataInputStream inputStream = null; + try { + inputStream = fs.open(testPath, 4096); + } catch (FileNotFoundException e) { + hasException = true; + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + assertTrue(hasException); + } + + @Test + // open的文件父目录及上级目录不是一个目录,抛AccessControlException + public void testOpenAbnormal004() throws Exception { + if (!fs.isFsBucket()) { + return; + } + Path testPath = getTestPath("a/b/c/d"); + fs.mkdirs(testPath.getParent().getParent()); + FSDataOutputStream os = OBSFSTestUtil.createStream(fs, + testPath.getParent(), true); + os.close(); + + boolean hasException = false; + FSDataInputStream inputStream = null; + try { + inputStream = fs.open(testPath, 4096); + } catch (AccessControlException e) { + hasException = true; + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + assertTrue(hasException); + OBSFSTestUtil.deletePathRecursive(fs, testPath.getParent().getParent()); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSOutputStream.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSOutputStream.java new file mode 100644 index 0000000..abb7843 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSOutputStream.java @@ -0,0 +1,100 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.io.IOUtils; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +import static org.junit.Assert.assertTrue; + +public class ITestOBSOutputStream { + private OBSFileSystem fs; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setLong(OBSConstants.MULTIPART_SIZE, 5 * 1024 * 1024); + conf.setBoolean(OBSConstants.FAST_UPLOAD, false); + fs = OBSTestUtils.createTestFileSystem(conf); + + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + /** + * @throws IOException + */ + @Test + // 分别上传小于、等于、大于MULTIPART_SIZE大小文件,校验数据正确性 + public void testUpload() throws IOException { + verifyUpload(5 * 1024 * 1024 + 1); + verifyUpload(5 * 1024 * 1024); + verifyUpload(5 * 1024 * 1024 - 1); + } + + /** + * @throws IOException + */ + @Test + // 上传0字节大小文件,校验文件长度 + public void testZeroUpload() throws IOException { + verifyUpload(0); + } + + @Test + // 文件输出流关闭后,继续写数据,抛出IOException + public void testWriteAfterStreamClose() throws IOException { + Path dest = getTestPath("testWriteAfterStreamClose"); + FSDataOutputStream stream = fs.create(dest, true); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + try { + stream.write(data); + stream.close(); + boolean hasException = false; + try { + stream.write(data); + } catch (IOException e) { + hasException = true; + } + assertTrue(hasException); + + } finally { + IOUtils.closeStream(stream); + } + } + + private Path getTestPath(String relativePath) { + return new Path(testRootPath + "/" + relativePath); + } + + private void verifyUpload(long fileSize) throws IOException { + ContractTestUtils.createAndVerifyFile(fs, getTestPath("test_file"), + fileSize); + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRename.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRename.java new file mode 100644 index 0000000..37c71c5 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRename.java @@ -0,0 +1,881 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyString; + +import com.obs.services.ObsClient; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.ParentNotDirectoryException; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.mockito.Mockito; +import org.mockito.internal.util.reflection.Whitebox; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; + +public class ITestOBSRename { + private OBSFileSystem fs; + + private static String testRootPath = OBSTestUtils.generateUniqueTestPath(); + + private static final Logger LOG = LoggerFactory.getLogger( + ITestOBSRename.class); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setClass(OBSConstants.OBS_METRICS_CONSUMER, + MockMetricsConsumer.class, BasicMetricsConsumer.class); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(5 * 1024 * 1024)); + conf.setBoolean(OBSConstants.METRICS_SWITCH, true); + fs = OBSTestUtils.createTestFileSystem(conf); + if (fs.exists(new Path(testRootPath))) { + fs.delete(new Path(testRootPath), true); + } + fs.mkdirs(new Path(testRootPath)); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + @Test + //文件到文件:源文件存在,但目标文件不存在,Rename返回True + public void testFileToFile01() throws IOException { + String srcFile = "srcFile"; + String destFile = "destFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destFilePath = new Path(testRootPath + "/" + destFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destFilePath)) { + fs.delete(destFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + boolean res = fs.rename(srcFilePath, destFilePath); + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.normal, mmc.getMr().getKind()); + + } + assertTrue(res); + assertFalse(fs.exists(srcFilePath)); + assertTrue(fs.exists(destFilePath)); + + fs.delete(destFilePath); + } + + @Test + //文件到文件:源文件和目标文件均不存在,Rename返回False + public void testFileToFile02() throws IOException { + String srcFile = "srcFile"; + String destFile = "destFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destFilePath = new Path(testRootPath + "/" + destFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destFilePath)) { + fs.delete(destFilePath); + } + + assertFalse(fs.rename(srcFilePath, destFilePath)); + assertFalse(fs.exists(srcFilePath)); + assertFalse(fs.exists(destFilePath)); + } + + @Test + //文件到文件:源文件不存在,但目标文件已存在,基于HDFS原生语义,源文件不存在时,Rename返回False + public void testFileToFile03() throws IOException { + String srcFile = "srcFile"; + String destFile = "destFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destFilePath = new Path(testRootPath + "/" + destFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destFilePath)) { + fs.delete(destFilePath); + } + + FSDataOutputStream stream = fs.create(destFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertFalse(fs.rename(srcFilePath, destFilePath)); + assertFalse(fs.exists(srcFilePath)); + assertTrue(fs.exists(destFilePath)); + + fs.delete(destFilePath); + } + + @Test + //文件到文件:源文件存在,但目标文件的父目录不存在,Rename返回False + public void testFileToFile04() throws IOException { + String srcFile = "srcFile"; + String nonexistParent = "parent"; + String destFile = "destFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destParentDirPath = new Path(testRootPath + "/" + nonexistParent); + Path destFilePath = new Path( + testRootPath + "/" + nonexistParent + "/" + destFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destParentDirPath)) { + fs.delete(destParentDirPath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertFalse(fs.rename(srcFilePath, destFilePath)); + assertTrue(fs.exists(srcFilePath)); + assertFalse(fs.exists(destParentDirPath)); + assertFalse(fs.exists(destFilePath)); + + fs.delete(srcFilePath); + } + + @Test + //文件到文件:源文件存在,但目标文件的Parent是一个文件,Rename抛ParentNotDirectoryException + public void testFileToFile05() throws IOException { + String srcFile = "srcFile"; + String nonexistParent = "parent"; + String destFile = "destFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destParentDirPath = new Path(testRootPath + "/" + nonexistParent); + Path destFilePath = new Path( + testRootPath + "/" + nonexistParent + "/" + destFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destParentDirPath)) { + fs.delete(destParentDirPath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + stream = fs.create(destParentDirPath); + data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + boolean expectedException = false; + try { + fs.rename(srcFilePath, destFilePath); + } catch (ParentNotDirectoryException e) { + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + } + expectedException = true; + } + assertTrue(expectedException); + assertTrue(fs.exists(srcFilePath)); + assertTrue(fs.exists(destParentDirPath)); + assertFalse(fs.exists(destFilePath)); + + fs.delete(srcFilePath); + fs.delete(destParentDirPath); + } + + @Test + //文件到文件:源文件存在,目标文件也存在,Rename返回False + public void testFileToFile06() throws IOException { + String srcFile = "srcFile"; + String destFile = "destFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destFilePath = new Path(testRootPath + "/" + destFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destFilePath)) { + fs.delete(destFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + stream = fs.create(destFilePath); + data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertFalse(fs.rename(srcFilePath, destFilePath)); + assertTrue(fs.exists(srcFilePath)); + assertTrue(fs.exists(destFilePath)); + + fs.delete(srcFilePath); + fs.delete(destFilePath); + } + + @Test + //文件到文件:源文件存在,目标文件和源文件相同,Rename返回True + public void testFileToFile07() throws IOException { + String srcFile = "srcFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertTrue(fs.rename(srcFilePath, srcFilePath)); + assertTrue(fs.exists(srcFilePath)); + + fs.delete(srcFilePath); + } + + @Test + // 文件到文件:src的父目录及上级目录不是一个目录,抛AccessControlException + public void testFileToFile08() throws IOException { + if(!fs.isFsBucket()) { + return; + } + String srcFile = "a001/srcFile"; + String dstFile = "b001/dstFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path dstFilePath = new Path(testRootPath + "/" + dstFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(dstFilePath)) { + fs.delete(dstFilePath); + } + fs.mkdirs(dstFilePath.getParent()); + FSDataOutputStream stream = fs.create(srcFilePath.getParent()); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + boolean hasException = false; + try { + fs.rename(srcFilePath, dstFilePath); + } catch (AccessControlException e) { + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns()); + } + hasException = true; + } + assertTrue(hasException); + + fs.delete(dstFilePath); + } + + @Test + // 文件到文件:dst的父目录及上级目录不是一个目录,抛ParentNotDirectoryException + public void testFileToFile09() throws IOException { + String srcFile = "a001/srcFile"; + String dstFile = "b001/dstFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path dstFilePath = new Path(testRootPath + "/" + dstFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(dstFilePath)) { + fs.delete(dstFilePath); + } + + fs.mkdirs(srcFilePath.getParent()); + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + stream = fs.create(dstFilePath.getParent()); + stream.write(data); + stream.close(); + + boolean hasException = false; + try { + fs.rename(srcFilePath, dstFilePath); + } catch (ParentNotDirectoryException e) { + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns()); + } + hasException = true; + } + assertTrue(hasException); + + fs.delete(srcFilePath); + } + + @Test + //文件到文件:src的父目录及上级目录不存在,返回false + public void testFileToFile10() throws IOException { + String srcFile = "a001/srcFile"; + String dstFile = "b001/dstFile"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path dstFilePath = new Path(testRootPath + "/" + dstFile); + fs.delete(srcFilePath.getParent(), true); + fs.delete(dstFilePath.getParent(), true); + fs.mkdirs(dstFilePath.getParent()); + assertFalse(fs.rename(srcFilePath, dstFilePath)); + fs.delete(srcFilePath); + } + + // @Test + // //文件到文件:对象桶rename过程中删除src失败,返回false + // public void testFileToFile11() throws IOException { + // if (fs.isFsBucket()) { + // return; + // } + // String srcFile = "a001/srcFile"; + // String dstFile = "b001/dstFile"; + // Path srcFilePath = new Path(testRootPath + "/" + srcFile); + // Path dstFilePath = new Path(testRootPath + "/" + dstFile); + // fs.delete(srcFilePath.getParent(), true); + // fs.delete(dstFilePath.getParent(), true); + // + // fs.mkdirs(srcFilePath.getParent()); + // fs.mkdirs(dstFilePath.getParent()); + // FSDataOutputStream stream = fs.create(srcFilePath); + // byte[] data = ContractTestUtils.dataset(16, 'a', 26); + // stream.write(data); + // stream.close(); + // + // OBSFileSystem mockFs = Mockito.spy(fs); + // ObsClient client = fs.getObsClient(); + // ObsClient mockClient = Mockito.spy(client); + // Whitebox.setInternalState(mockFs, String.valueOf(client), mockClient); + // IOException ioException = new IOException("mock IOException"); + // Mockito.doThrow(ioException) + // .when(mockClient) + // .deleteObject(anyString(), anyString()); + // + // assertFalse(mockFs.rename(srcFilePath, dstFilePath)); + // + // fs.delete(srcFilePath); + // } + + @Test + //目录到文件:源目录存在,目标文件也存在,Rename返回False + public void testDirToFile01() throws IOException { + String srcDir = "srcDir"; + String destFile = "destFile"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destFilePath = new Path(testRootPath + "/" + destFile); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destFilePath)) { + fs.delete(destFilePath); + } + + fs.mkdirs(srcDirPath); + + FSDataOutputStream stream = fs.create(destFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertFalse(fs.rename(srcDirPath, destFilePath)); + assertTrue(fs.exists(srcDirPath)); + assertTrue(fs.exists(destFilePath)); + + fs.delete(srcDirPath); + fs.delete(destFilePath); + } + + @Test + //目录到文件:src是根目录,返回false + public void testDirToFile02() throws IOException { + if (!fs.isFsBucket()) { + return; + } + Path srcFilePath = new Path("/"); + Path dstFilePath = new Path(testRootPath + "/test_file"); + + assertFalse(fs.rename(srcFilePath, dstFilePath)); + } + + @Test + //目录到目录:源目录存在,但目标目录不存在,Rename返回True + public void testDirToDir01() throws IOException { + String srcDir = "srcDir"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destDirPath = new Path(testRootPath + "/" + destDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + fs.mkdirs(srcDirPath); + + assertTrue(fs.rename(srcDirPath, destDirPath)); + assertFalse(fs.exists(srcDirPath)); + assertTrue(fs.exists(destDirPath)); + + fs.delete(destDirPath); + } + + @Test + //目录到目录:源目录和目标目录均不存在,Rename返回False + public void testDirToDir02() throws IOException { + String srcDir = "srcDir"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destDirPath = new Path(testRootPath + "/" + destDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + assertFalse(fs.rename(srcDirPath, destDirPath)); + assertFalse(fs.exists(srcDirPath)); + assertFalse(fs.exists(destDirPath)); + } + + @Test + //目录到目录:源目录不存在,但目标目录已存在,基于HDFS原生语义,源目录不存在时,Rename返回False + public void testDirToDir03() throws IOException { + String srcDir = "srcDir"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destDirPath = new Path(testRootPath + "/" + destDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + fs.mkdirs(destDirPath); + + assertFalse(fs.rename(srcDirPath, destDirPath)); + assertFalse(fs.exists(srcDirPath)); + assertTrue(fs.exists(destDirPath)); + + fs.delete(destDirPath); + } + + @Test + //目录到目录:源目录存在,但目标目录的父目录不存在,Rename返回False + public void testDirToDir04() throws IOException { + String srcDir = "srcDir"; + String nonexistParent = "parent"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destParentDirPath = new Path(testRootPath + "/" + nonexistParent); + Path destDirPath = new Path( + testRootPath + "/" + nonexistParent + "/" + destDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destParentDirPath)) { + fs.delete(destParentDirPath); + } + + fs.mkdirs(srcDirPath); + + assertFalse(fs.rename(srcDirPath, destDirPath)); + assertTrue(fs.exists(srcDirPath)); + assertFalse(fs.exists(destParentDirPath)); + assertFalse(fs.exists(destDirPath)); + + fs.delete(srcDirPath); + } + + @Test + //目录到目录:目标目录是源目录下面的一个Child,Rename返回False + public void testDirToDir05() throws IOException { + String srcDir = "srcDir"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destDirPath = new Path(testRootPath + "/" + destDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + Path childDestDirPath = new Path( + testRootPath + "/" + srcDir + "/" + destDir); + fs.mkdirs(srcDirPath); + fs.mkdirs(childDestDirPath); + + assertFalse(fs.rename(srcDirPath, childDestDirPath)); + assertTrue(fs.exists(srcDirPath)); + + fs.delete(srcDirPath); + } + + @Test + //目录到目录:源目录存在,目标目录存在,源目录被移到目标目录下作为Child,Rename返回True + public void testDirToDir06() throws IOException { + String srcDir = "srcDir"; + String srcFile = "srcFile"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path srcFilePath = new Path( + testRootPath + "/" + srcDir + "/" + srcFile); + Path destDirPath = new Path(testRootPath + "/" + destDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + fs.mkdirs(srcDirPath); + fs.mkdirs(destDirPath); + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertTrue(fs.rename(srcDirPath, destDirPath)); + assertFalse(fs.exists(srcDirPath)); + assertTrue(fs.exists( + new Path(testRootPath + "/" + destDir + "/" + srcDir + "/"))); + assertTrue(fs.exists(new Path( + testRootPath + "/" + destDir + "/" + srcDir + "/" + srcFile))); + + fs.delete(destDirPath); + } + + @Test + //目录到目录:源目录存在,目标目录存在,源目录被移到目标目录下作为Child,但目标目录下该目录已存在,Rename返回False + public void testDirToDir07() throws IOException { + String srcDir = "srcDir"; + String srcFile = "srcFile"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path srcFilePath = new Path( + testRootPath + "/" + srcDir + "/" + srcFile); + Path destDirPath = new Path(testRootPath + "/" + destDir); + Path destChildDirPath = new Path( + testRootPath + "/" + destDir + "/" + srcDir); + Path destFilePath = new Path( + testRootPath + "/" + destDir + "/" + srcDir + "/" + srcFile); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + fs.mkdirs(srcDirPath); + fs.mkdirs(destDirPath); + fs.mkdirs(destChildDirPath); + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertFalse(fs.rename(srcDirPath, destDirPath)); + assertTrue(fs.exists(srcDirPath)); + assertTrue(fs.exists(destChildDirPath)); + assertFalse(fs.exists(destFilePath)); + + fs.delete(srcDirPath); + fs.delete(destDirPath); + } + + @Test + //目录到目录:源目录存在,目标目录和源目录相同,Rename返回False + public void testDirToDir08() throws IOException { + String srcDir = "srcDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + + fs.mkdirs(srcDirPath); + + assertFalse(fs.rename(srcDirPath, srcDirPath)); + assertTrue(fs.exists(srcDirPath)); + + fs.delete(srcDirPath); + } + + @Test + //目录到目录:src的父目录及上级目录不存在,Rename返回False + public void testDirToDir09() throws IOException { + String srcDir = "a001/a002/srcDir"; + String dstDir = "b001/b002/dstDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path dstDirPath = new Path(testRootPath + "/" + dstDir); + fs.delete(srcDirPath.getParent(), true); + fs.delete(dstDirPath, true); + + fs.mkdirs(dstDirPath.getParent()); + assertFalse(fs.rename(srcDirPath, dstDirPath)); + fs.delete(srcDirPath); + } + + @Test + //目录到目录:src是根目录,返回false + public void testDirToDir10() throws IOException { + if (!fs.isFsBucket()) { + return; + } + String srcDir = "/"; + String dstDir = "dstDir"; + Path srcDirPath = new Path(srcDir); + Path dstDirPath = new Path(testRootPath + "/" + dstDir); + fs.delete(dstDirPath, true); + fs.mkdirs(dstDirPath); + + assertFalse(fs.rename(srcDirPath, dstDirPath)); + fs.delete(dstDirPath); + } + + @Test + //目录到目录:对象桶rename过程中删除src失败,返回false + public void testDirToDir11() throws IOException { + if (!fs.isFsBucket()) { + return; + } + if (fs.isFsBucket()) { + return; + } + String srcDir = "a001/srcDir"; + String dstDir = "b001/dstDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path dstDirPath = new Path(testRootPath + "/" + dstDir); + fs.delete(srcDirPath.getParent(), true); + fs.delete(dstDirPath.getParent(), true); + + fs.mkdirs(srcDirPath); + fs.mkdirs(dstDirPath.getParent()); + + OBSFileSystem mockFs = Mockito.spy(fs); + ObsClient client = fs.getObsClient(); + ObsClient mockClient = Mockito.spy(client); + Whitebox.setInternalState(mockFs, String.valueOf(client), mockClient); + IOException ioException = new IOException("mock IOException"); + Mockito.doThrow(ioException) + .when(mockClient) + .deleteObject(anyString(), anyString()); + + assertFalse(mockFs.rename(srcDirPath, dstDirPath)); + + fs.delete(srcDirPath); + fs.delete(dstDirPath.getParent()); + } + + @Test + //目录到目录:源目录存在,但目标目录的父目录为文件或不存在,Rename返回True + public void testDirToDir12() throws IOException { + if (!fs.isFsBucket()) { + return; + } + String srcDir = "a/b/srcDir"; + String destDir = "e/d/destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destDirPath = new Path(testRootPath + "/" + destDir); + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + fs.mkdirs(srcDirPath); + + FSDataOutputStream stream = fs.create( + destDirPath.getParent().getParent()); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + boolean hasException = false; + try { + fs.rename(srcDirPath, destDirPath); + } catch (ParentNotDirectoryException e) { + if (fs.getMetricSwitch()) { + MockMetricsConsumer mmc + = (MockMetricsConsumer) fs.getMetricsConsumer(); + assertEquals(BasicMetricsConsumer.MetricKind.abnormal, mmc.getMr().getKind()); + LOG.warn("metricInfo:"+mmc.getMr().getExceptionIns()); + } + hasException = true; + } + assertTrue(hasException); + } + + @Test + //目录到目录:源目录中存在超过1000个文件,rename返回True + public void testDirToDir13() throws IOException { + if (fs.isFsBucket()) { + return; + } + + String srcDir = "srcDir"; + String destDir = "destDir"; + Path srcDirPath = new Path(testRootPath + "/" + srcDir); + Path destDirPath = new Path(testRootPath + "/" + destDir); + + if (fs.exists(srcDirPath)) { + fs.delete(srcDirPath,true); + } + fs.mkdirs(srcDirPath); + for (int i=0; i<1001;i++){ + Path srcFilePath = new Path( + testRootPath + "/" + srcDir + "/" + srcDir + i); + FSDataOutputStream stream = fs.create(srcFilePath); + stream.close(); + } + + if (fs.exists(destDirPath)) { + fs.delete(destDirPath,true); + } + fs.mkdirs(destDirPath); +// boolean rename = fs.rename(srcDirPath, destDirPath); + assertTrue(fs.rename(srcDirPath, destDirPath)); + + } + + @Test + //文件到目录:源文件存在,目标目录存在,源文件被移到目标目录下作为Child,Rename返回True + public void testFileToDir01() throws IOException { + String srcFile = "srcFile"; + String destDir = "destDir"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destDirPath = new Path(testRootPath + "/" + destDir); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + fs.mkdirs(destDirPath); + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertTrue(fs.rename(srcFilePath, destDirPath)); + assertFalse(fs.exists(srcFilePath)); + assertTrue( + fs.exists(new Path(testRootPath + "/" + destDir + "/" + srcFile))); + + fs.delete(destDirPath); + } + + @Test + //文件到目录:源文件存在,目标目录存在,源文件被移到目标目录下作为Child,但目标目录下该文件已存在,Rename返回False + public void testFileToDir02() throws IOException { + String srcFile = "srcFile"; + String destDir = "destDir"; + Path srcFilePath = new Path(testRootPath + "/" + srcFile); + Path destDirPath = new Path(testRootPath + "/" + destDir); + Path destFilePath = new Path( + testRootPath + "/" + destDir + "/" + srcFile); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + if (fs.exists(destDirPath)) { + fs.delete(destDirPath); + } + + fs.mkdirs(destDirPath); + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + stream = fs.create(destFilePath); + data = ContractTestUtils.dataset(16, 'a', 26); + stream.write(data); + stream.close(); + + assertFalse(fs.rename(srcFilePath, destDirPath)); + assertTrue(fs.exists(srcFilePath)); + assertTrue(fs.exists(destDirPath)); + assertTrue(fs.exists(destFilePath)); + + fs.delete(srcFilePath); + fs.delete(destDirPath); + } + + void checkData(final byte[] actual, int from, int len, + final byte[] expected, String message) { + for (int idx = 0; idx < len; idx++) { + assertEquals( + message + " byte " + (from + idx) + " differs. expected " + + expected[from + idx] + " actual " + actual[idx], + expected[from + idx], actual[idx]); + actual[idx] = 0; + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRetryMechanism.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRetryMechanism.java new file mode 100644 index 0000000..f20fedf --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRetryMechanism.java @@ -0,0 +1,1401 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.anyString; + +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObsObject; +import com.obs.services.model.fs.FSStatusEnum; +import com.obs.services.model.fs.GetBucketFSStatusResult; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.runners.MockitoJUnitRunner; +import org.powermock.reflect.Whitebox; + +import java.io.EOFException; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.net.SocketException; +import java.nio.ByteBuffer; + +@RunWith(MockitoJUnitRunner.class) +public class ITestOBSRetryMechanism { + private OBSFileSystem fs; + + private OBSFileSystem mockFs; + + private ObsClient obsClient; + + private ObsClient mockObsClient; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private Path testFile = getTestPath("testFile"); + + private Path testDir = getTestPath("testDir"); + + private static byte[] dataSet = ContractTestUtils.dataset(16, 0, 10); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.FAST_UPLOAD, false); + conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 2); + + fs = OBSTestUtils.createTestFileSystem(conf); + obsClient = fs.getObsClient(); + initTestEnv(); + + initMock(); + } + + private Path getTestPath(String testPath) { + return new Path(testRootPath + "/" + testPath); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + } + Mockito.reset(mockFs, mockObsClient); + } + + private void initTestEnv() throws Exception { + FSDataOutputStream outputStream = fs.create(testFile, true); + outputStream.write(dataSet); + outputStream.close(); + assertTrue(fs.mkdirs(testDir)); + if (!fs.exists(testDir)) { + throw new IOException("testDir not exist!"); + } + } + + private void initMock() { + mockFs = Mockito.spy(fs); + mockObsClient = Mockito.spy(obsClient); + Whitebox.setInternalState(mockFs, obsClient, mockObsClient); + } + + @Test + // 测试GetFileStatus重试机制,重试时对于403等特殊异常能正确识别 + public void testGetFileStatus() throws IOException { + OBSFileConflictException fileConflictException = new OBSFileConflictException( + "mock FileConflictException"); + FileNotFoundException fileNotFoundException = new FileNotFoundException( + "mock FileNotFoundException"); + AccessControlException accessControlException = + new AccessControlException("mock AccessControlException"); + OBSIOException obsioException = new OBSIOException("mock IOException", + new ObsException("mock ObsException")); + + Mockito.doThrow(obsioException) + .doThrow(obsioException) + .doThrow(obsioException) + .doThrow(obsioException) + .doThrow(obsioException) + .doThrow(fileNotFoundException) + .doThrow(obsioException) + .doThrow(obsioException) + .doThrow(fileConflictException) + .doThrow(obsioException) + .doThrow(obsioException) + .doThrow(accessControlException) + .doThrow(obsioException) + .doThrow(obsioException) + .doThrow(obsioException) + .doCallRealMethod() + .when(mockFs) + .innerGetFileStatus(anyObject()); + + int i = 0; + while (true) { + try { + mockFs.getFileStatus(testFile); + break; + } catch (FileNotFoundException e) { + assertTrue(i == 0 || i == 1); + } catch (AccessControlException e) { + assertTrue(i == 2); + } + i++; + } + assertTrue(i == 3); + } + + @Test + // 测试IsFolderEmpty重试机制,重试时对于403等特殊异常能正确识别 + public void testIsFolderEmpty() throws Exception { + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + + int i = 0; + while (true) { + try { + OBSCommonUtils.isFolderEmpty(mockFs, + testDir.toString().substring(1)); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试CreateEmptyObject重试机制,重试时对于403等特殊异常能正确识别 + public void testCreateEmptyObject() throws Exception { + if (fs.isFsBucket()) { + return; + } + + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .doCallRealMethod() + .when(mockObsClient) + .putObject(anyObject()); + + int i = 0; + while (true) { + try { + OBSObjectBucketUtils.createEmptyObject(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testEmptyDir"))); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试FsCreateFolder重试机制,重试时对于403等特殊异常能正确识别 + public void testFsCreateFolder() throws IOException { + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .newFolder(anyObject()); + + int i = 0; + while (true) { + try { + OBSPosixBucketUtils.fsCreateFolder(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testFsCreateFolder"))); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试ListObjects重试机制,重试时对于403等特殊异常能正确识别 + public void testListObjects() throws Exception { + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + + int i = 0; + while (true) { + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setDelimiter("/"); + request.setMaxKeys(1000); + + OBSCommonUtils.listObjects(mockFs, request); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试ListObjectsRecursively重试机制,重试时对于403等特殊异常能正确识别 + public void testListObjectsRecursively() throws Exception { + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + + int i = 0; + while (true) { + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setMaxKeys(1000); + + OBSCommonUtils.listObjects(mockFs, request); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试ContinueListObjects重试机制,重试时对于403等特殊异常能正确识别 + public void testContinueListObjects() throws Exception { + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + + int i = 0; + while (true) { + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setDelimiter("/"); + request.setMaxKeys(1000); + + ObjectListing objectListing = new ObjectListing(null, null, + mockFs.getBucket(), true, testRootPath.substring(1), null, + 1000, "/", "/", null); + + OBSCommonUtils.continueListObjects(mockFs, objectListing); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 与testListObjectsRecursively的测试点重复,不用再重复测试 + public void testContinueListObjectsRecursively() { + } + + @Test + // 测试InnerFsTruncateWithRetry重试机制,重试时对于403等特殊异常能正确识别 + public void testInnerFsTruncateWithRetry() throws Exception { + if (!fs.isFsBucket()) { + return; + } + + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .truncateObject(anyObject()); + + int i = 0; + while (true) { + try { + OBSPosixBucketUtils.innerFsTruncateWithRetry(mockFs, testFile, 3); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试FileInnerFsRenameWithRetry重试机制,重试时对于403等特殊异常能正确识别 + public void testFileInnerFsRenameWithRetry() throws Exception { + if (!fs.isFsBucket()) { + return; + } + + Path testFileSrc = getTestPath("test_file_src"); + FSDataOutputStream outputStream = fs.create(testFileSrc, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + Path testFileDst = getTestPath("test_file_dst"); + String srcKey = OBSCommonUtils.pathToKey(fs, testFileSrc); + String dstKey = OBSCommonUtils.pathToKey(fs, testFileDst); + + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .doThrow(notFoundException) + .when(mockObsClient) + .renameFile(anyObject()); + + int i = 0; + boolean success; + while (i < 4) { + try { + success = OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, + testFileSrc, testFileDst, srcKey, dstKey); + assertTrue(success); + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } + i++; + } + } + + @Test + // 测试DirInnerFsRenameWithRetry重试机制,重试时对于403等特殊异常能正确识别 + public void testDirInnerFsRenameWithRetry() throws Exception { + if (!fs.isFsBucket()) { + return; + } + + Path testDirSrc = getTestPath("test_dir_src"); + fs.mkdirs(testDirSrc); + Path testDirDst = getTestPath("test_dir_dst"); + String srcKey = OBSCommonUtils.pathToKey(fs, testDirSrc); + String dstKey = OBSCommonUtils.pathToKey(fs, testDirDst); + + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .doThrow(notFoundException) + .when(mockObsClient) + .renameFile(anyObject()); + + int i = 0; + boolean success; + while (i < 4) { + try { + success = OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, + testDirSrc, testDirDst, srcKey, dstKey); + assertTrue(success); + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } + i++; + } + } + + @Test + // http ResponseCode 400 not retry + public void testNotRetry400() throws IOException { + OBSIllegalArgumentException illegalArgumentException = new OBSIllegalArgumentException( + "mock OBSIllegalArgumentException"); + OBSIOException obsioException = new OBSIOException("mock IOException", + new ObsException("mock ObsException")); + + Mockito.doThrow(obsioException) + .doThrow(illegalArgumentException) + .doThrow(obsioException) + .doThrow(obsioException) + .doThrow(obsioException) + .doCallRealMethod() + .when(mockFs) + .innerGetFileStatus(anyObject()); + //OBSIllegalArgumentException not retry + try { + mockFs.getFileStatus(testFile); + } catch (OBSIllegalArgumentException e) { + assertTrue(true); + } + //OBSIOException retry + try { + mockFs.getFileStatus(testFile); + } catch (IOException e) { + assertTrue(false); + } + } + + @Test + // 测试getBucketFsStatus重试机制 + public void testGetBucketFsStatus() throws Exception { + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doReturn(new GetBucketFSStatusResult(null, null, 0, null, null, + null, null, null, FSStatusEnum.ENABLED)) + .when(mockObsClient) + .getBucketFSStatus(anyObject()); + + int i = 0; + while (true) { + try { + OBSCommonUtils.getBucketFsStatus(mockObsClient, "obsfilesystem-bucket"); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试lazySeek onReadFailure时对reopen的重试机制 + public void testLazySeek() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {1, 2, 3, 4, 5, 6}; + outputStream.write(data); + outputStream.close(); + + ObsException obsException = new ObsException("mock ObsException."); + ObsException eofException = new ObsException( + "mock ObsException of EOF."); + eofException.setResponseCode(OBSCommonUtils.EOF_CODE); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(eofException) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + + FSDataInputStream inputStream = mockFs.open(testFile, 4096); + byte[] buffer = new byte[1024]; + int bytes; + try { + int i = 0; + while (true) { + try { + bytes = inputStream.read(buffer, 0, 5); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } + i++; + } + assertTrue(i == 3); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + assertTrue(bytes == -1); + } + + @Test + // 测试onReadFailure时对reopen的重试机制 + public void testOnReadFailure() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + + ObsException obsException = new ObsException("mock ObsException."); + ObsException eofException = new ObsException( + "mock ObsException of EOF."); + eofException.setResponseCode(OBSCommonUtils.EOF_CODE); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + SocketException connectionResetException = new SocketException( + "mock Connection Reset Exception"); + ObsException socketException = new ObsException("connection reset ", + connectionResetException); + + Mockito.doThrow(eofException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(eofException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(eofException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(eofException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(eofException) + .doThrow(eofException) + .doThrow(socketException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + + FSDataInputStream inputStream = mockFs.open(testFile, 4096); + byte[] buffer = new byte[1024]; + int bytes; + try { + int i = 0; + while (true) { + try { + bytes = inputStream.read(buffer, 0, 5); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (EOFException e) { + assertTrue(i == 3); + } + i++; + } + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + assertTrue(bytes == -1); + } + + @Test + // 测试read(byte b[], int off, int len)接口重试机制,重试时对于EOF等特殊异常能正确识别 + public void testReadRetry() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read() throws IOException { + return 0; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + IOException ioException = new IOException("mock IOException"); + EOFException eofException = new EOFException("mock " + + "EOFException"); + Mockito.doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(eofException) + .doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(); + + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + FSDataInputStream inputStream = mockFs.open(testFile, 4096); + byte[] buffer = new byte[1024]; + int bytes = -1; + try { + int i = 0; + while (bytes < 0) { + bytes = inputStream.read(buffer, 0, 5); + if (bytes == -1) { + assertTrue(i == 0); + } else { + assertTrue(i == 1); + } + i++; + } + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + assertTrue(bytes == 5); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + + } + + //read() + @Test + // 测试read()接口重试机制,重试时对于EOF等特殊异常能正确识别 + public void testReadRetry001() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read() throws IOException { + return 0; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + IOException ioException = new IOException("mock IOException"); + EOFException eofException = new EOFException("mock " + + "EOFException"); + Mockito.doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(eofException) + .doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(); + + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + FSDataInputStream inputStream = mockFs.open(testFile, 4096); + int bytes = -1; + try { + int i = 0; + while (bytes < 0) { + bytes = inputStream.read(); + if (bytes == -1) { + assertTrue(i == 0); + } else { + assertTrue(i == 1); + } + i++; + } + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + assertTrue(bytes == 0); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + + } + + //read(ByteBuffer byteBuffer) + @Test + // 测试read(ByteBuffer byteBuffer)接口重试机制,重试时对于EOF等特殊异常能正确识别 + public void testReadByteBufferRetry() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read() throws IOException { + return 0; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + IOException ioException = new IOException("mock IOException"); + EOFException eofException = new EOFException("mock " + + "EOFException"); + Mockito.doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(eofException) + .doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(); + + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + FSDataInputStream inputStream = mockFs.open(testFile, 4096); + ByteBuffer byteBuffer = ByteBuffer.allocateDirect(1024); + int bytes = -1; + try { + int i = 0; + while (bytes < 0) { + bytes = inputStream.read(byteBuffer); + if (bytes == -1) { + assertTrue(i == 0); + } else { + assertTrue(i == 1); + } + i++; + } + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + assertTrue(bytes == 1024); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + + @Test + //randomReadWithNewInputStream 前半部分重试机制 + public void testReadWithNewInputStreamRetry001() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {1, 2, 3, 4, 5, 6}; + outputStream.write(data); + outputStream.close(); + + ObsException obsException = new ObsException("mock ObsException."); + ObsException eofException = new ObsException( + "mock ObsException of EOF."); + eofException.setResponseCode(OBSCommonUtils.EOF_CODE); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(eofException) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + + FSDataInputStream inputStream = mockFs.open(testFile, 4096); + byte[] buffer = new byte[1024]; + int bytes; + try { + int i = 0; + while (true) { + try { + bytes = inputStream.read(0, buffer, 0, 5); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } + i++; + } + assertTrue(i == 3); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + assertTrue(bytes == -1); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + //randomReadWithNewInputStream 后半部分重试机制 + public void testReadWithNewInputStreamRetry002() throws Exception { + Path testFile = getTestPath("test_file"); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + FSDataOutputStream outputStream = fs.create(testFile, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read() throws IOException { + return 0; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + IOException ioException = new IOException("mock IOException"); + EOFException eofException = new EOFException("mock " + + "EOFException"); + Mockito.doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doThrow(eofException) + .doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(); + + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + FSDataInputStream inputStream = mockFs.open(testFile, 4096); + byte[] buffer = new byte[1024]; + int bytes = -1; + try { + int i = 0; + while (bytes < 0) { + bytes = inputStream.read(0, buffer, 0, 5); + if (bytes == -1) { + assertTrue(i == 0); + } else { + assertTrue(i == 1); + } + i++; + } + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + assertTrue(bytes == 5); + OBSFSTestUtil.deletePathRecursive(fs, testFile); + } + + @Test + // 测试delete接口重试机制,重试时对于404等特殊异常能正确识别 + public void testDeleteObject() throws Exception { + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .deleteObject(anyString(), anyString()); + + int i = 0; + while (true) { + try { + OBSCommonUtils.deleteObject(mockFs, + testFile.toString().substring(1)); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试对象桶CopyFile接口重试机制,重试时对于403等特殊异常能正确识别 + public void testCopyFile() throws Exception { + if (fs.isFsBucket()) { + return; + } + Path srcFile = getTestPath("srcFile"); + Path destFile = getTestPath("destFile"); + FSDataOutputStream os = fs.create(srcFile); + OBSFSTestUtil.writeData(os, 1024 * 1024); + os.close(); + + ObsException obsException = new ObsException("mock ObsException."); + ObsException unauthorizedException = new ObsException("mock " + + "unauthorized exception."); + unauthorizedException.setResponseCode(OBSCommonUtils.UNAUTHORIZED_CODE); + ObsException forbiddenException = new ObsException("mock " + + "forbidden exception."); + forbiddenException.setResponseCode(OBSCommonUtils.FORBIDDEN_CODE); + ObsException notFoundException = new ObsException("mock " + + "not found exception."); + notFoundException.setResponseCode(OBSCommonUtils.NOT_FOUND_CODE); + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(obsException) + .doThrow(obsException) + .doThrow(unauthorizedException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(forbiddenException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(notFoundException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(conflictException) + .doThrow(obsException) + .doThrow(obsException) + .doThrow(obsException) + .doCallRealMethod() + .when(mockObsClient) + .copyObject(anyObject()); + + Mockito.doCallRealMethod() + .when(mockObsClient) + .getObjectMetadata(anyObject()); + + Mockito.doReturn(5 * 1024 * 1024L) + .when(mockFs).getCopyPartSize(); + + int i = 0; + while (true) { + try { + OBSObjectBucketUtils.copyFile(mockFs, + srcFile.toString().substring(1), + destFile.toString().substring(1), 10); + break; + } catch (AccessControlException e) { + assertTrue(i == 0 || i == 1); + } catch (FileNotFoundException e) { + assertTrue(i == 2); + } catch (OBSFileConflictException e) { + assertTrue(i == 3); + } + i++; + } + assertTrue(i == 4); + } + + @Test + // 测试文件桶递归删除时,服务端返回409,在OBSA侧执行重试 + public void testDeleteNotEmptyDirRecursively() throws Exception { + if (!fs.isFsBucket()) { + return; + } + + // 1、构造多级目录结构 + Path dir01 = getTestPath("dir01/"); + Path subDir01 = getTestPath("dir01/subDir01/"); + Path subFile01 = getTestPath("dir01/subFile01"); + fs.mkdirs(dir01); + fs.mkdirs(subDir01); + FSDataOutputStream os = fs.create(subFile01); + OBSFSTestUtil.writeData(os, 1024 * 1024); + os.close(); + + // 2、删除对象注入FileConflictException异常 + ObsException conflictException = new ObsException("mock " + + "conflict exception."); + conflictException.setResponseCode(OBSCommonUtils.CONFLICT_CODE); + + Mockito.doThrow(conflictException) + .doThrow(conflictException) + .doThrow(conflictException) + .doCallRealMethod() + .when(mockObsClient) + .deleteObject(anyString(), anyString()); + + Mockito.doThrow(conflictException) + .doThrow(conflictException) + .doThrow(conflictException) + .doCallRealMethod() + .when(mockObsClient) + .deleteObjects(anyObject()); + + // OBSCommonUtils.RETRY_MAXTIME = 10; + // // 3、递归删除一级目录,重试超时,删除失败 + // boolean hasException = false; + // try { + // mockFs.delete(dir01, true); + // } catch (Exception e) { + // hasException = true; + // } + // assertTrue("delete folder should has exception", hasException); + // + // OBSCommonUtils.RETRY_MAXTIME = 180000; + // 4、递归删除一级目录,重试多次成功 + boolean hasException = false; + try { + mockFs.delete(dir01, true); + } catch (Exception e) { + hasException = true; + } + assertFalse("delete folder should not has exception", hasException); + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRetryMechanism2.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRetryMechanism2.java new file mode 100644 index 0000000..a29acae --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSRetryMechanism2.java @@ -0,0 +1,2134 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; +import static org.mockito.Matchers.anyInt; +import static org.mockito.Matchers.anyObject; +import static org.mockito.Matchers.anyString; + +import com.obs.services.ObsClient; +import com.obs.services.exception.ObsException; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObsObject; +import com.obs.services.model.fs.FSStatusEnum; +import com.obs.services.model.fs.GetAttributeRequest; +import com.obs.services.model.fs.GetBucketFSStatusResult; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.junit.runner.RunWith; +import org.mockito.Mockito; +import org.mockito.runners.MockitoJUnitRunner; +import org.powermock.reflect.Whitebox; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.EOFException; +import java.io.FileNotFoundException; +import java.io.IOException; +import java.io.InputStream; +import java.lang.reflect.Method; +import java.net.SocketException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.HashMap; + +@RunWith(MockitoJUnitRunner.class) +public class ITestOBSRetryMechanism2 { + private OBSFileSystem fs; + private ObsClient obsClient; + private OBSFileSystem mockFs; + private ObsClient mockObsClient; + + private static String testRootPath = + OBSTestUtils.generateUniqueTestPath(); + + private Path testFile = getTestPath("testFile"); + + private Path testDir = getTestPath("testDir"); + + private static byte[] dataSet = ContractTestUtils.dataset(16, 0, 10); + + + ObsException exception400; + ObsException exception403; + ObsException exception404; + ObsException exception409; + ObsException exception416; + + ObsException exception500; + ObsException exception503; + + IOException[] noRetryIOExceptions; + IOException[] retryIOExceptions; + + HashMap noRetryExceptions = new HashMap<>(); + HashMap retryExceptions = new HashMap<>(); + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.setBoolean(OBSConstants.FAST_UPLOAD, false); + conf.setLong(OBSConstants.MULTIPART_SIZE,1024*1024L); + fs = OBSTestUtils.createTestFileSystem(conf); + setRetryParam(); + obsClient = fs.getObsClient(); + initTestEnv(); + initMock(); + + exception400 = new ObsException("mock 400 exception"); + exception400.setResponseCode(400); + exception403 = new ObsException("mock 403 exception"); + exception403.setResponseCode(403); + exception404 = new ObsException("mock 404 exception"); + exception404.setResponseCode(404); + exception409 = new ObsException("mock 409 exception"); + exception409.setResponseCode(409); + + exception416 = new ObsException("mock 416 exception"); + exception416.setResponseCode(416); + + exception500 = new ObsException("mock 500 exception"); + exception500.setResponseCode(500); + exception503 = new ObsException("mock 503 exception"); + exception503.setResponseCode(503); + exception503.setErrorCode("GetQosTokenException"); + + noRetryIOExceptions = new IOException[] { + new OBSIllegalArgumentException("mock OBSIllegalArgumentException"), + new AccessControlException("mock AccessControlException"), + new FileNotFoundException("mock FileNotFoundException"), + new OBSFileConflictException("mock OBSFileConflictException") + }; + retryIOExceptions = new IOException[] { + new OBSQosException("mock OBSQosException", exception503), + new OBSIOException("mock OBSIOException", exception500) + }; + + noRetryExceptions.put(exception400, OBSIllegalArgumentException.class); + noRetryExceptions.put(exception403, AccessControlException.class); + noRetryExceptions.put(exception404, FileNotFoundException.class); + noRetryExceptions.put(exception409, OBSFileConflictException.class); + noRetryExceptions.put(exception416, EOFException.class); + + retryExceptions.put(exception500, OBSIOException.class); + retryExceptions.put(exception503, OBSQosException.class); + } + + private Path getTestPath(String testPath) { + return new Path(testRootPath + "/" + testPath); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + fs.close(); + } + Mockito.reset(mockFs, mockObsClient); + } + + private void initTestEnv() throws Exception { + FSDataOutputStream outputStream = fs.create(testFile, true); + outputStream.write(dataSet); + outputStream.close(); + assertTrue(fs.mkdirs(testDir)); + if (!fs.exists(testDir)) { + throw new IOException("testDir not exist!"); + } + } + + private void initMock() { + mockFs = Mockito.spy(fs); + mockObsClient = Mockito.spy(obsClient); + Whitebox.setInternalState(mockFs, obsClient, mockObsClient); + + // mockFs_retryFail = Mockito.spy(fs_retryFail); + // mockObsClient_retryFail = Mockito.spy(obsClient_retryFail); + // Whitebox.setInternalState(mockFs_retryFail, obsClient_retryFail, mockObsClient_retryFail); + } + + @Test + public void testGetFileStatus() throws IOException { + for (IOException ex : noRetryIOExceptions) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockFs) + .innerGetFileStatus(anyObject()); + try { + mockFs.getFileStatus(testFile); + } catch (IOException e) { + if (e.getMessage().contains("mock OBSFileConflictException")) { + Assert.assertTrue("testGetFileStatus",e.getClass() == FileNotFoundException.class); + } else { + Assert.assertTrue("testGetFileStatus",e.getClass() == ex.getClass()); + } + } + } + + //重试成功 + for (IOException ex : retryIOExceptions) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockFs) + .innerGetFileStatus(anyObject()); + try { + mockFs.getFileStatus(testFile); + } catch (IOException e) { + Assert.assertTrue("testGetFileStatus",false); + } + } + + + //重试失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockFs) + .innerGetFileStatus(anyObject()); + try { + mockFs.getFileStatus(testFile); + } catch (IOException e) { + Assert.assertTrue("testGetFileStatus",e.getClass() == retryExceptions.get(ex)); + } + } + } + + @Test + public void testWriteByUploadPart() { + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .uploadPart(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByUploadPart")); + OBSFSTestUtil.writeData(outputStream, 6* 1024 * 1024); + } catch (IOException e) { + Assert.assertTrue("upload part",e.getMessage().contains("write has error")); + } + } + //重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .uploadPart(anyObject()); + + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .initiateMultipartUpload(anyObject()); + + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .completeMultipartUpload(anyObject()); + + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByUploadPart")); + OBSFSTestUtil.writeData(outputStream, 6* 1024 * 1024); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("upload part",false); + } + } + + } + + @Test + public void testWriteByUploadPartWithInitFail() { + setNewRetryParam(); + OBSWriteOperationHelper writeHelper = fs.getWriteHelper(); + Whitebox.setInternalState(writeHelper, obsClient, mockObsClient); + //重试失败 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .initiateMultipartUpload(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByUploadPart")); + OBSFSTestUtil.writeData(outputStream, 6* 1024 * 1024); + } catch (IOException e) { + Assert.assertTrue("upload part", + e.getMessage().contains(OBSOperateAction.initMultiPartUpload.toString())); + } + } + } + + @Test + public void testWriteByUploadPartWithCompleteFail() { + setNewRetryParam(); + OBSWriteOperationHelper writeHelper = fs.getWriteHelper(); + Whitebox.setInternalState(writeHelper, obsClient, mockObsClient); + + //重试第3次失败 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .completeMultipartUpload(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByUploadPart")); + OBSFSTestUtil.writeData(outputStream, 6* 1024 * 1024); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("upload part",e.getMessage().contains("completeMultipartUpload")); + } + } + } + + @Test + public void testWriteByUploadPartWithUploadFail() { + setNewRetryParam(); + OBSWriteOperationHelper writeHelper = fs.getWriteHelper(); + Whitebox.setInternalState(writeHelper, obsClient, mockObsClient); + + //重试失败 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .uploadPart(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByUploadPart")); + OBSFSTestUtil.writeData(outputStream, 6* 1024 * 1024); + } catch (IOException e) { + Assert.assertTrue("upload part",e.getMessage().contains("write has error")); + } + } + } + + @Test + public void testFsAppend() throws IOException { + if (!fs.isFsBucket()) { + return; + } + FSDataOutputStream out = fs.create(getTestPath("testAppendFile")); + out.write("123456".getBytes()); + out.close(); + + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .writeFile(anyObject()); + FSDataOutputStream outputStream = null; + try { + outputStream = mockFs.append(getTestPath("testAppendFile")); + outputStream.write("123456".getBytes()); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testAppend",e.getClass()== noRetryExceptions.get(ex)); + } + } + + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .writeFile(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.append(getTestPath("testAppendFile")); + outputStream.write("123456".getBytes()); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testAppend",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .writeFile(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.append(getTestPath("testAppendFile")); + outputStream.write("123456".getBytes()); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testAppend", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testFsAppendByHflush() { + if (!fs.isFsBucket()) { + return; + } + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .writeFile(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testAppendByHflush")); + outputStream.write("123456".getBytes()); + outputStream.hflush(); + outputStream.write("123456".getBytes()); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testAppendByHflush",e.getClass()== noRetryExceptions.get(ex)); + } + } + + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .writeFile(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testAppendByHflush")); + outputStream.write("123456".getBytes()); + outputStream.hflush(); + outputStream.write("123456".getBytes()); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testAppendByHflush",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .writeFile(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testAppendByHflush")); + outputStream.write("123456".getBytes()); + outputStream.hflush(); + outputStream.write("123456".getBytes()); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testAppendByHflush", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testWriteByPutObject() { + OBSWriteOperationHelper writeHelper = fs.getWriteHelper(); + Whitebox.setInternalState(writeHelper, fs, mockFs); + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .putObject(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByPutObject")); + OBSFSTestUtil.writeData(outputStream, 4 * 1024 * 1024); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testCreateByPutObject",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .putObject(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByPutObject")); + OBSFSTestUtil.writeData(outputStream, 4 * 1024 * 1024); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testCreateByPutObject",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .putObject(anyObject()); + try { + FSDataOutputStream outputStream = mockFs.create(getTestPath("testCreateByPutObject")); + OBSFSTestUtil.writeData(outputStream, 4 * 1024 * 1024); + outputStream.close(); + } catch (IOException e) { + Assert.assertTrue("testCreateByPutObject", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testIsFolderEmpty() throws Exception { + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + OBSCommonUtils.isFolderEmpty(mockFs, testDir.toString().substring(1)); + } catch (IOException e) { + Assert.assertTrue("testIsFolderEmpty",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + OBSCommonUtils.isFolderEmpty(mockFs, testDir.toString().substring(1)); + } catch (IOException e) { + Assert.assertTrue("testIsFolderEmpty",false); + } + } + + //重试失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + OBSCommonUtils.isFolderEmpty(mockFs, testDir.toString().substring(1)); + } catch (IOException e) { + Assert.assertTrue("testIsFolderEmpty", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testObjectMkdir() { + if (fs.isFsBucket()) { + return; + } + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .putObject(anyObject()); + try { + OBSObjectBucketUtils.createEmptyObject(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testMkdirByCreateEmptyObject"))); + } catch (IOException e) { + Assert.assertTrue("testMkdirByCreateEmptyObject",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .putObject(anyObject()); + try { + OBSObjectBucketUtils.createEmptyObject(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testMkdirByCreateEmptyObject"))); + } catch (IOException e) { + Assert.assertTrue("testMkdirByCreateEmptyObject",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .putObject(anyObject()); + try { + OBSObjectBucketUtils.createEmptyObject(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testMkdirByCreateEmptyObject"))); + } catch (IOException e) { + Assert.assertTrue("testMkdirByCreateEmptyObject", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + // 测试FsCreateFolder重试机制,重试时对于403等特殊异常能正确识别 + public void testFsMkdir() throws IOException { + if (!fs.isFsBucket()) { + return; + } + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .newFolder(anyObject()); + try { + OBSPosixBucketUtils.fsCreateFolder(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testMkdirByFsCreateFolder"))); + } catch (IOException e) { + Assert.assertTrue("testMkdirByFsCreateFolder",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .newFolder(anyObject()); + try { + OBSPosixBucketUtils.fsCreateFolder(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testMkdirByFsCreateFolder"))); + } catch (IOException e) { + Assert.assertTrue("testMkdirByFsCreateFolder",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .newFolder(anyObject()); + try { + OBSPosixBucketUtils.fsCreateFolder(mockFs, + OBSCommonUtils.pathToKey(fs, getTestPath("testMkdirByFsCreateFolder"))); + } catch (IOException e) { + Assert.assertTrue("testMkdirByFsCreateFolder", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testListStatusByListObjects() throws Exception { + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setDelimiter("/"); + request.setMaxKeys(1000); + OBSCommonUtils.listObjects(mockFs, request); + } catch (IOException e) { + Assert.assertTrue("testListStatusByListObjects",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setDelimiter("/"); + request.setMaxKeys(1000); + OBSCommonUtils.listObjects(mockFs, request); + } catch (IOException e) { + Assert.assertTrue("testListStatusByListObjects",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setDelimiter("/"); + request.setMaxKeys(1000); + OBSCommonUtils.listObjects(mockFs, request); + } catch (IOException e) { + Assert.assertTrue("testListStatusByListObjects", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testListStatusByListObjectsRecursively() throws Exception { + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setMaxKeys(1000); + OBSCommonUtils.listObjects(mockFs, request); + } catch (IOException e) { + Assert.assertTrue("testListStatusByListObjects",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setMaxKeys(1000); + OBSCommonUtils.listObjects(mockFs, request); + } catch (IOException e) { + Assert.assertTrue("testListStatusByListObjects",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ListObjectsRequest request = new ListObjectsRequest(); + request.setBucketName(mockFs.getBucket()); + request.setPrefix(testRootPath.substring(1)); + request.setMaxKeys(1000); + OBSCommonUtils.listObjects(mockFs, request); + } catch (IOException e) { + Assert.assertTrue("testListStatusByListObjects", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + // 测试ContinueListObjects重试机制,重试时对于403等特殊异常能正确识别 + public void testListStatusByContinueListObjects() throws Exception { + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ObjectListing objectListing = new ObjectListing(null, null, + mockFs.getBucket(), true, testRootPath.substring(1), null, + 1000, "/", "/", null); + OBSCommonUtils.continueListObjects(mockFs, objectListing); + } catch (IOException e) { + Assert.assertTrue("testListStatusByContinueListObjects",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ObjectListing objectListing = new ObjectListing(null, null, + mockFs.getBucket(), true, testRootPath.substring(1), null, + 1000, "/", "/", null); + OBSCommonUtils.continueListObjects(mockFs, objectListing); + } catch (IOException e) { + Assert.assertTrue("testListStatusByContinueListObjects",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .listObjects((ListObjectsRequest) anyObject()); + try { + ObjectListing objectListing = new ObjectListing(null, null, + mockFs.getBucket(), true, testRootPath.substring(1), null, + 1000, "/", "/", null); + OBSCommonUtils.continueListObjects(mockFs, objectListing); + } catch (IOException e) { + Assert.assertTrue("testListStatusByContinueListObjects", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + // 与testListStatusByListObjectsRecursively的测试点重复,不用再重复测试 + public void testListStatusByContinueListObjectsRecursively() { + } + + @Test + public void testFsTruncate() throws Exception { + if (!fs.isFsBucket()) { + return; + } + + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .truncateObject(anyObject()); + try { + OBSPosixBucketUtils.innerFsTruncateWithRetry(mockFs, testFile, 3); + } catch (IOException e) { + Assert.assertTrue("testFsTruncate",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .truncateObject(anyObject()); + try { + OBSPosixBucketUtils.innerFsTruncateWithRetry(mockFs, testFile, 3); + } catch (IOException e) { + Assert.assertTrue("testFsTruncate",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .truncateObject(anyObject()); + try { + OBSPosixBucketUtils.innerFsTruncateWithRetry(mockFs, testFile, 3); + } catch (IOException e) { + Assert.assertTrue("testFsTruncate", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testFsRenameFile() throws Exception { + if (!fs.isFsBucket()) { + return; + } + + Path testFileSrc = getTestPath("test_file_src"); + FSDataOutputStream outputStream = fs.create(testFileSrc, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + Path testFileDst = getTestPath("test_file_dst"); + String srcKey = OBSCommonUtils.pathToKey(fs, testFileSrc); + String dstKey = OBSCommonUtils.pathToKey(fs, testFileDst); + + + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .renameFile(anyObject()); + try { + boolean success = OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, + testFileSrc, testFileDst, srcKey, dstKey); + //404 FileNotFoundException rename将返回false + assertFalse(success); + } catch (IOException e) { + Assert.assertTrue("testFsRenameFile",e.getClass()== noRetryExceptions.get(ex)); + } + } + + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .renameFile(anyObject()); + try { + boolean success = OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, + testFileSrc, testFileDst, srcKey, dstKey); + //第1次循环重试成功,第2次循环dst已经存在但是src不存在了,根据对FileNotFoundException的处理将返回true + assertTrue(success); + } catch (IOException e) { + Assert.assertTrue("testFsRenameFile",e.getClass() == FileNotFoundException.class ); + } + } + } + + @Test + public void testFsRenameFileWithFail() throws Exception { + if (!fs.isFsBucket()) { + return; + } + setNewRetryParam(); + Path testFileSrc = getTestPath("test_file_src"); + FSDataOutputStream outputStream = fs.create(testFileSrc, false); + byte[] data = {0, 0, 0, 0, 0, 0}; + outputStream.write(data); + outputStream.close(); + Path testFileDst = getTestPath("test_file_dst"); + String srcKey = OBSCommonUtils.pathToKey(fs, testFileSrc); + String dstKey = OBSCommonUtils.pathToKey(fs, testFileDst); + //重试第3次失败 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .renameFile(anyObject()); + try { + OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, + testFileSrc, testFileDst, srcKey, dstKey); + } catch (IOException e) { + Assert.assertTrue("testFsRenameFile", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testFsRenameFolder() throws Exception { + if (!fs.isFsBucket()) { + return; + } + + Path testDirSrc = getTestPath("test_dir_src"); + fs.mkdirs(testDirSrc); + Path testDirDst = getTestPath("test_dir_dst"); + String srcKey = OBSCommonUtils.pathToKey(fs, testDirSrc); + String dstKey = OBSCommonUtils.pathToKey(fs, testDirDst); + + for (Exception ex : noRetryExceptions.keySet()) { + if (((ObsException)ex).getResponseCode() == 403) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .renameFile(anyObject()); + } else { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .renameFile(anyObject()); + } + + try { + boolean success = OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, testDirSrc, testDirDst, + srcKey, dstKey); + //404 FileNotFoundException rename将返回false + assertFalse(success); + } catch (IOException e) { + Assert.assertTrue("testFsRenameFolder",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .renameFile(anyObject()); + try { + boolean success = OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, testDirSrc, testDirDst, + srcKey, dstKey); + //第1次循环重试成功,第2次循环dst已经存在但是src不存在了 + assertTrue(success); + } catch (IOException e) { + Assert.assertTrue("testFsRenameFolder",e.getClass() == FileNotFoundException.class ); + } + } + } + + @Test + public void testFsRenameFolderWithFail() throws Exception { + if (!fs.isFsBucket()) { + return; + } + setNewRetryParam(); + Path testDirSrc = getTestPath("test_dir_src"); + fs.mkdirs(testDirSrc); + Path testDirDst = getTestPath("test_dir_dst"); + String srcKey = OBSCommonUtils.pathToKey(fs, testDirSrc); + String dstKey = OBSCommonUtils.pathToKey(fs, testDirDst); + + //重试第3次失败 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .renameFile(anyObject()); + try { + OBSPosixBucketUtils.innerFsRenameWithRetry(mockFs, testDirSrc, testDirDst, srcKey, dstKey); + } catch (IOException e) { + Assert.assertTrue("testFsRenameFolder", e.getClass() == retryExceptions.get(ex)); + } + } + } + + @Test + public void testGetBucketStatus() { + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getBucketFSStatus(anyObject()); + try { + OBSCommonUtils.getBucketFsStatus(mockObsClient, fs.getBucket()); + } catch (IOException e) { + Assert.assertTrue("testGetBucketStatus",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getBucketFSStatus(anyObject()); + try { + OBSCommonUtils.getBucketFsStatus(mockObsClient, fs.getBucket()); + } catch (IOException e) { + Assert.assertTrue("testGetBucketStatus",false); + } + } + + //重试失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .getBucketFSStatus(anyObject()); + try { + OBSCommonUtils.getBucketFsStatus(mockObsClient, fs.getBucket()); + } catch (IOException e) { + Assert.assertTrue("testGetBucketStatus", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testReadWhenSeekInStreamException() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456789".getBytes()); + outputStream.close(); + byte[] buffer = new byte[6]; + + //读IO异常 + IOException ioException = new IOException("mock IOException"); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int available() throws IOException { + throw new IOException("mock IOException"); + } + + @Override + public int read(byte b[], int off, int len) throws IOException { + System.arraycopy("123456789".getBytes(),0, b, off, len); + return len; + } + @Override + public int read() throws IOException { + return 1; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + + // // 重试成功 + // Mockito.doThrow(ioException) + // .doThrow(ioException) + // .doCallRealMethod() + // .when(mockInputStream) + // .read(anyObject(),anyInt(),anyInt()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(buffer, 0, 3); + Assert.assertTrue("testReadWithBytes", "123".equals(new String(buffer).substring(0,3))); + + inputStream.read(4, buffer, 0, 5); + Assert.assertTrue("testReadByRandom", "12345".equals(new String(buffer).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + + @Test + public void testReadByBytes() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + byte[] buffer = new byte[6]; + + //obsclient异常 + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + + try { + inputStream = mockFs.open(testFile, 4096); + //EOF异常 + int read = inputStream.read(buffer, 0, 5); + Assert.assertTrue("testReadWithBytes",read == -1); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== noRetryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(buffer, 0, 5); + Assert.assertTrue("testReadWithBytes", "12345".equals(new String(buffer).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",false); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(buffer, 0, 5); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== retryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + } + + @Test + public void testReadByBytesWithIO() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + byte[] buffer = new byte[6]; + + //读IO异常 + IOException ioException = new IOException("mock IOException"); + IOException eofException = new EOFException("mock EOFException"); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read(byte b[], int off, int len) throws IOException { + System.arraycopy("123456".getBytes(),0, b, off, len); + return len; + } + @Override + public int read() throws IOException { + return 1; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + // 重试成功 + Mockito.doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(buffer, 0, 5); + Assert.assertTrue("testReadWithBytes", "12345".equals(new String(buffer).substring(0,5))); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + // 重试失败 + setNewRetryParam(); + Mockito.doThrow(ioException) + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(buffer, 0, 5); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + + @Test + public void testReadByOneByte() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + + //obsclient异常 + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + //EOF异常 + int read = inputStream.read(); + Assert.assertTrue("testReadWithOneByte",read == -1); + } catch (IOException e) { + Assert.assertTrue("testReadWithOneByte",e.getClass()== noRetryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + int read = inputStream.read(); + Assert.assertTrue("", Integer.valueOf("123456".charAt(0))==read); + } catch (IOException e) { + Assert.assertTrue("testReadWithOneByte",false); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(); + } catch (IOException e) { + Assert.assertTrue("testReadWithOneByte",e.getClass()== retryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + // //读IO异常 + // IOException ioException = new IOException("mock IOException"); + // IOException eofException = new EOFException("mock EOFException"); + // + // ObsObject obsObject = new ObsObject(); + // obsObject.setBucketName(fs.getBucket()); + // obsObject.setObjectContent(new InputStream() { + // @Override + // public int read() throws IOException { + // return 1; + // } + // }); + // + // InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + // obsObject.setObjectContent(mockInputStream); + // Mockito.doReturn(obsObject) + // .when(mockObsClient) + // .getObject(anyObject()); + // + // // 重试失败 + // Mockito.doThrow(ioException) + // .doThrow(ioException) + // .doThrow(ioException) + // .doCallRealMethod() + // .when(mockInputStream) + // .read(); + // FSDataInputStream inputStream = null; + // try { + // inputStream = mockFs.open(testFile, 4096); + // inputStream.read(); + // } catch (IOException e) { + // Assert.assertTrue("testReadWithOneByte",e.getClass()== ioException.getClass()); + // } finally { + // if (inputStream != null) { + // inputStream.close(); + // } + // } + // + // // 重试成功 + // Mockito.doThrow(ioException) + // .doThrow(ioException) + // .doCallRealMethod() + // .when(mockInputStream) + // .read(); + // inputStream = null; + // try { + // inputStream = mockFs.open(testFile, 4096); + // int read = inputStream.read(); + // Assert.assertTrue("testReadWithOneByte", 1 == read); + // } catch (IOException e) { + // Assert.assertTrue("testReadWithOneByte",e.getClass()== ioException.getClass()); + // } finally { + // if (inputStream != null) { + // inputStream.close(); + // } + // } + } + + @Test + public void testReadByOneByteWithIO() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + + + //读IO异常 + IOException ioException = new IOException("mock IOException"); + IOException eofException = new EOFException("mock EOFException"); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read() throws IOException { + return 1; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + // 重试成功 + Mockito.doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + int read = inputStream.read(); + Assert.assertTrue("testReadWithOneByte", 1 == read); + } catch (IOException e) { + Assert.assertTrue("testReadWithOneByte",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + // 重试失败 + setNewRetryParam(); + Mockito.doThrow(ioException) + .when(mockInputStream) + .read(); + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(); + } catch (IOException e) { + Assert.assertTrue("testReadWithOneByte",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + + @Test + public void testReadByByteBuffer() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + ByteBuffer byteBuffer = ByteBuffer.allocate(5); + //obsclient异常 + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + + try { + inputStream = mockFs.open(testFile, 4096); + //EOF异常 + int read = inputStream.read(byteBuffer); + Assert.assertTrue("testReadWithBytes",read == -1); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== noRetryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + byteBuffer.clear(); + inputStream.read(byteBuffer); + Assert.assertTrue("testReadWithBytes", "12345".equals( + new String(byteBuffer.array()).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",false); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + byteBuffer.clear(); + inputStream.read(byteBuffer); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== retryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + } + + @Test + public void testReadByByteBufferWithIO() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + ByteBuffer byteBuffer = ByteBuffer.allocate(5); + + //读IO异常 + IOException ioException = new IOException("mock IOException"); + IOException eofException = new EOFException("mock EOFException"); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read(byte b[], int off, int len) throws IOException { + System.arraycopy("123456".getBytes(),0, b, off, len); + return len; + } + @Override + public int read() throws IOException { + return 1; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + // 重试成功 + Mockito.doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + byteBuffer.clear(); + inputStream.read(byteBuffer); + Assert.assertTrue("testReadWithBytes", "12345".equals( + new String(byteBuffer.array()).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + // 重试失败 + setNewRetryParam(); + Mockito.doThrow(ioException) + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + try { + inputStream = mockFs.open(testFile, 4096); + byteBuffer.clear(); + inputStream.read(byteBuffer); + } catch (IOException e) { + Assert.assertTrue("testReadWithBytes",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + + @Test + public void testReadByRandom() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + byte[] buffer = new byte[6]; + + //obsclient异常 + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + + try { + inputStream = mockFs.open(testFile, 4096); + //EOF异常 + int read = inputStream.read(1, buffer, 0, 4); + Assert.assertTrue("testReadByRandom",read == -1); + } catch (IOException e) { + Assert.assertTrue("testReadByRandom",e.getClass()== noRetryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + Assert.assertTrue("testReadByRandom", "23456".equals(new String(buffer).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadByRandom",false); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + } catch (IOException e) { + Assert.assertTrue("testReadByRandom",e.getClass()== retryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + } + + @Test + public void testReadByRandomWithIO() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + byte[] buffer = new byte[6]; + + //读IO异常 + IOException ioException = new IOException("mock IOException"); + IOException eofException = new EOFException("mock EOFException"); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read(byte b[], int off, int len) throws IOException { + System.arraycopy("123456".getBytes(),0, b, off, len); + return len; + } + @Override + public int read() throws IOException { + return 1; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + // 重试成功 + Mockito.doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + Assert.assertTrue("testReadByRandom", "12345".equals(new String(buffer).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadByRandom",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + // 重试失败 + setNewRetryParam(); + Mockito.doThrow(ioException) + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + } catch (IOException e) { + Assert.assertTrue("testReadByRandom",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + + //randomReadWithNewInputStream 重试机制 + @Test + public void testReadByRandomWithOptimize() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + byte[] buffer = new byte[6]; + + Mockito.doReturn(false) + .when(mockFs).isReadTransformEnabled(); + + //obsclient异常 + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + + try { + inputStream = mockFs.open(testFile, 4096); + //EOF异常 + int read = inputStream.read(1, buffer, 0, 4); + Assert.assertTrue("testReadByRandomWithOptimize",read == -1); + } catch (IOException e) { + Assert.assertTrue("testReadByRandomWithOptimize",e.getClass()== noRetryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + Assert.assertTrue("testReadByRandomWithOptimize", "23456".equals(new String(buffer).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadByRandomWithOptimize",false); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .getObject(anyObject()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + } catch (IOException e) { + Assert.assertTrue("testReadByRandomWithOptimize",e.getClass()== retryExceptions.get(ex)); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + } + + //randomReadWithNewInputStream 重试机制 + @Test + public void testReadByRandomWithOptimizeIO() throws Exception { + Path testFile = getTestPath("test_file"); + FSDataOutputStream outputStream = fs.create(testFile); + outputStream.write("123456".getBytes()); + outputStream.close(); + byte[] buffer = new byte[6]; + + Mockito.doReturn(false) + .when(mockFs).isReadTransformEnabled(); + + //读IO异常 + IOException ioException = new IOException("mock IOException"); + IOException eofException = new EOFException("mock EOFException"); + + ObsObject obsObject = new ObsObject(); + obsObject.setBucketName(fs.getBucket()); + obsObject.setObjectContent(new InputStream() { + @Override + public int read(byte b[], int off, int len) throws IOException { + System.arraycopy("123456".getBytes(),0, b, off, len); + return len; + } + @Override + public int read() throws IOException { + return 1; + } + }); + + InputStream mockInputStream = Mockito.spy(obsObject.getObjectContent()); + obsObject.setObjectContent(mockInputStream); + Mockito.doReturn(obsObject) + .when(mockObsClient) + .getObject(anyObject()); + + // 重试成功 + Mockito.doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + FSDataInputStream inputStream = null; + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + Assert.assertTrue("testReadByRandomWithOptimize", "12345".equals(new String(buffer).substring(0,5))); + } catch (IOException e) { + Assert.assertTrue("testReadByRandomWithOptimize",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + + + // 重试失败 + setNewRetryParam(); + Mockito.doThrow(ioException) + .doThrow(ioException) + .doThrow(ioException) + .doCallRealMethod() + .when(mockInputStream) + .read(anyObject(),anyInt(),anyInt()); + + try { + inputStream = mockFs.open(testFile, 4096); + inputStream.read(1, buffer, 0, 5); + } catch (IOException e) { + Assert.assertTrue("testReadByRandomWithOptimize",e.getClass()== ioException.getClass()); + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + @Test + public void testDelete() throws Exception { + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .deleteObject(anyObject()); + try { + OBSCommonUtils.deleteObject(mockFs, + testFile.toString().substring(1)); + } catch (IOException e) { + Assert.assertTrue("testDelete",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .deleteObject(anyObject()); + try { + OBSCommonUtils.deleteObject(mockFs, + testFile.toString().substring(1)); + } catch (IOException e) { + Assert.assertTrue("testDelete",false); + } + } + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .deleteObject(anyObject()); + try { + OBSCommonUtils.deleteObject(mockFs, + testFile.toString().substring(1)); + } catch (IOException e) { + Assert.assertTrue("testDelete", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testObjectRenameByCopyObject() throws Exception { + if (fs.isFsBucket()) { + return; + } + Path srcFile = getTestPath("srcFile"); + Path destFile = getTestPath("destFile"); + FSDataOutputStream os = fs.create(srcFile); + OBSFSTestUtil.writeData(os, 1024 * 1024); + os.close(); + + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .copyObject(anyObject()); + try { + OBSObjectBucketUtils.copyFile(mockFs, srcFile.toString().substring(1), + destFile.toString().substring(1), 10); + } catch (IOException e) { + Assert.assertTrue("testObjectRenameByCopyObject",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .copyObject(anyObject()); + try { + OBSObjectBucketUtils.copyFile(mockFs, srcFile.toString().substring(1), + destFile.toString().substring(1), 10); + } catch (IOException e) { + Assert.assertTrue("testObjectRenameByCopyObject",false); + } + } + } + + @Test + public void testObjectRenameByCopyObjectFail() throws Exception { + if (fs.isFsBucket()) { + return; + } + Path srcFile = getTestPath("srcFile"); + Path destFile = getTestPath("destFile"); + FSDataOutputStream os = fs.create(srcFile); + OBSFSTestUtil.writeData(os, 1024 * 1024); + os.close(); + + //重试失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .copyObject(anyObject()); + try { + OBSObjectBucketUtils.copyFile(mockFs, srcFile.toString().substring(1), + destFile.toString().substring(1), 10); + } catch (IOException e) { + Assert.assertTrue("testObjectRenameByCopyObject", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + public void testObjectRenameByCopyPart() throws Exception { + if (fs.isFsBucket()) { + return; + } + Path srcFile = getTestPath("srcFile"); + Path destFile = getTestPath("destFile"); + FSDataOutputStream os = fs.create(srcFile); + OBSFSTestUtil.writeData(os, 10 * 1024 * 1024); + os.close(); + + //多段copy阈值设置为5MB + Mockito.doReturn(5 * 1024 * 1024L) + .when(mockFs).getCopyPartSize(); + for (Exception ex : noRetryExceptions.keySet()) { + Mockito.doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .copyPart(anyObject()); + try { + OBSObjectBucketUtils.copyFile(mockFs, srcFile.toString().substring(1), + destFile.toString().substring(1), 10 * 1024 * 1024); + } catch (IOException e) { + Assert.assertTrue("testObjectRenameByCopyPart",e.getClass()== noRetryExceptions.get(ex)); + } + } + // 重试第3次成功 + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .doThrow(ex) + .doThrow(ex) + .doThrow(ex) + .doCallRealMethod() + .when(mockObsClient) + .copyPart(anyObject()); + try { + OBSObjectBucketUtils.copyFile(mockFs, srcFile.toString().substring(1), + destFile.toString().substring(1), 10 * 1024 * 1024); + } catch (IOException e) { + Assert.assertTrue("testObjectRenameByCopyPart",false); + } + } + } + + @Test + public void testObjectRenameByCopyPartFail() throws Exception { + if (fs.isFsBucket()) { + return; + } + Path srcFile = getTestPath("srcFile"); + Path destFile = getTestPath("destFile"); + FSDataOutputStream os = fs.create(srcFile); + OBSFSTestUtil.writeData(os, 10 * 1024 * 1024); + os.close(); + + //多段copy阈值设置为5MB + Mockito.doReturn(5 * 1024 * 1024L) + .when(mockFs).getCopyPartSize(); + + //重试第3次失败 + setNewRetryParam(); + for (Exception ex : retryExceptions.keySet()) { + Mockito.doThrow(ex) + .when(mockObsClient) + .copyPart(anyObject()); + try { + OBSObjectBucketUtils.copyFile(mockFs, srcFile.toString().substring(1), + destFile.toString().substring(1), 10 * 1024 * 1024); + } catch (IOException e) { + Assert.assertTrue("testObjectRenameByCopyPart", e.getClass()== retryExceptions.get(ex)); + } + } + } + + @Test + // 测试文件桶递归删除时,服务端返回409,在OBSA侧执行重试 + public void testFsDeleteWith409Exception() throws Exception { + if (!fs.isFsBucket()) { + return; + } + // 构造多级目录结构 + Path dir01 = getTestPath("dir01/"); + Path subDir01 = getTestPath("dir01/subDir01/"); + Path subFile01 = getTestPath("dir01/subFile01"); + fs.mkdirs(dir01); + fs.mkdirs(subDir01); + FSDataOutputStream os = fs.create(subFile01); + OBSFSTestUtil.writeData(os, 1024 * 1024); + os.close(); + + boolean hasException = false; + Mockito.doThrow(exception409) + .doThrow(exception409) + .doCallRealMethod() + .when(mockObsClient) + .deleteObject(anyString(), anyString()); + Mockito.doThrow(exception409) + .doThrow(exception409) + .doCallRealMethod() + .when(mockObsClient) + .deleteObjects(anyObject()); + + // 409重试后成功 + try { + mockFs.delete(dir01, true); + } catch (Exception e) { + hasException = true; + } + assertFalse("delete folder should not has exception", hasException); + } + + @Test + // 测试文件桶递归删除时,服务端返回409,在OBSA侧执行重试 + public void testFsDeleteWith409ExceptionFail() throws Exception { + if (!fs.isFsBucket()) { + return; + } + // 构造多级目录结构 + Path dir01 = getTestPath("dir01/"); + Path subDir01 = getTestPath("dir01/subDir01/"); + Path subFile01 = getTestPath("dir01/subFile01"); + fs.mkdirs(dir01); + fs.mkdirs(subDir01); + FSDataOutputStream os = fs.create(subFile01); + OBSFSTestUtil.writeData(os, 1024 * 1024); + os.close(); + + setNewRetryParam(); + boolean hasException = false; + // 409重试后不成功 + Mockito.doThrow(exception409) + .when(mockObsClient) + .deleteObject(anyString(), anyString()); + Mockito.doThrow(exception409) + .when(mockObsClient) + .deleteObjects(anyObject()); + try { + mockFs.delete(dir01, true); + } catch (Exception e) { + hasException = true; + } + assertTrue("delete folder has exception", hasException); + } + + + private void setRetryParam() { + Configuration conf = new Configuration(); + conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 5); + conf.setLong(OBSConstants.RETRY_SLEEP_MAXTIME, 10); + conf.setLong(OBSConstants.RETRY_QOS_SLEEP_BASETIME, 5); + conf.setLong(OBSConstants.RETRY_QOS_SLEEP_MAXTIME, 10); + OBSCommonUtils.init(fs, conf); + } + + private void setNewRetryParam() { + Configuration conf = new Configuration(); + conf.setLong(OBSConstants.RETRY_MAXTIME, 10); + conf.setLong(OBSConstants.RETRY_SLEEP_BASETIME, 5); + conf.setLong(OBSConstants.RETRY_SLEEP_MAXTIME, 10); + conf.setLong(OBSConstants.RETRY_QOS_MAXTIME, 10); + conf.setLong(OBSConstants.RETRY_QOS_SLEEP_BASETIME, 5); + conf.setLong(OBSConstants.RETRY_QOS_SLEEP_MAXTIME, 10); + OBSCommonUtils.init(fs, conf); + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSTestUtils.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSTestUtils.java new file mode 100644 index 0000000..26bf239 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSTestUtils.java @@ -0,0 +1,117 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import static org.apache.hadoop.fs.obs.OBSTestUtils.*; + +/** + * Test the test utils. Why an integration test? it's needed to verify property + * pushdown. + */ +@Deprecated +public class ITestOBSTestUtils extends Assert { + private static final Logger LOG = + LoggerFactory.getLogger(ITestOBSTestUtils.class); + + public static final String KEY = "undefined.property"; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Before + public void clear() { + System.clearProperty(KEY); + } + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Test + // 测试getTestProperty功能正确性 + public void testGetTestProperty() throws Throwable { + Configuration conf = new Configuration(false); + conf.addResource(OBSContract.CONTRACT_XML); + assertEquals("a", getTestProperty(conf, KEY, "a")); + conf.set(KEY, "\t b \n"); + assertEquals("b", getTestProperty(conf, KEY, "a")); + System.setProperty(KEY, "c"); + assertEquals("c", getTestProperty(conf, KEY, "a")); + unsetSysprop(); + assertEquals("b", getTestProperty(conf, KEY, "a")); + } + + @Test + // 测试getTestPropertyLong功能正确性 + public void testGetTestPropertyLong() throws Throwable { + Configuration conf = new Configuration(false); + assertEquals(1, getTestPropertyLong(conf, KEY, 1)); + conf.setInt(KEY, 2); + assertEquals(2, getTestPropertyLong(conf, KEY, 1)); + System.setProperty(KEY, "3"); + assertEquals(3, getTestPropertyLong(conf, KEY, 1)); + } + + @Test + // 测试getTestPropertyInt功能正确性 + public void testGetTestPropertyInt() throws Throwable { + Configuration conf = new Configuration(false); + assertEquals(1, getTestPropertyInt(conf, KEY, 1)); + conf.setInt(KEY, 2); + assertEquals(2, getTestPropertyInt(conf, KEY, 1)); + System.setProperty(KEY, "3"); + assertEquals(3, getTestPropertyInt(conf, KEY, 1)); + conf.unset(KEY); + assertEquals(3, getTestPropertyInt(conf, KEY, 1)); + unsetSysprop(); + assertEquals(5, getTestPropertyInt(conf, KEY, 5)); + } + + @Test + // 测试getTestPropertyBool功能正确性 + public void testGetTestPropertyBool() throws Throwable { + Configuration conf = new Configuration(false); + assertTrue(getTestPropertyBool(conf, KEY, true)); + conf.set(KEY, "\tfalse \n"); + assertFalse(getTestPropertyBool(conf, KEY, true)); + System.setProperty(KEY, "true"); + assertTrue(getTestPropertyBool(conf, KEY, true)); + unsetSysprop(); + assertEquals("false", getTestProperty(conf, KEY, "true")); + conf.unset(KEY); + assertTrue(getTestPropertyBool(conf, KEY, true)); + } + + protected void unsetSysprop() { + System.setProperty(KEY, UNSET_PROPERTY); + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSTruncate.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSTruncate.java new file mode 100644 index 0000000..7cc6104 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSTruncate.java @@ -0,0 +1,1003 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.security.AccessControlException; +import org.junit.After; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.FileNotFoundException; +import java.io.IOException; + +public class ITestOBSTruncate { + private OBSFileSystem fs; + + private static String testRootPath = OBSTestUtils.generateUniqueTestPath(); + + private final int partSize = 5 * 1024 * 1024; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.set(OBSConstants.MULTIPART_SIZE, String.valueOf(partSize)); + fs = OBSTestUtils.createTestFileSystem(conf); + if (fs.exists(new Path(testRootPath))) { + fs.delete(new Path(testRootPath), true); + } + fs.mkdirs(new Path(testRootPath)); + } + + @After + public void tearDown() throws Exception { + if (fs != null) { + fs.delete(new Path(testRootPath), true); + } + } + + @Test + // create一个文件并写入数据5MB-1Byte(长度小于段大小), + // 之后再truncate到10Byte,预期截断成功 + public void truncate01() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize - 1, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB(长度等于段大小), + // 之后再truncate到10Byte,预期截断成功 + public void truncate02() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 之后再truncate到段大小5MB,预期截断成功 + public void truncate03() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = partSize; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 之后再truncate到超过段大小5MB+10Byte,预期截断成功 + public void truncate04() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据500Byte(长度仍然小于段大小), + // 之后再truncate到10Byte,预期截断成功 + public void truncate05() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(500, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据500Byte(长度仍然小于段大小), + // 之后再truncate到200Byte,预期截断成功 + public void truncate06() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(500, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = 200; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据5MB(长度超过段大小), + // 之后再truncate到200Byte,预期截断成功 + public void truncate07() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = 200; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据5MB(长度超过段大小), + // 之后再truncate到5MB,预期截断成功 + public void truncate08() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据5MB(长度超过段大小), + // 之后再truncate到5MB+10Byte,预期截断成功 + public void truncate09() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 然后append一些数据500Byte, + // 之后再truncate到段大小5MB,预期截断成功 + public void truncate10() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(500, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 然后append一些数据500Byte, + // 之后再truncate到超过段大小5MB+10Byte,预期截断成功 + public void truncate11() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(500, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 然后append一些数据500Byte, + // 之后再truncate到超过段大小5MB+200Byte,预期截断成功 + public void truncate12() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(500, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 200; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 然后append一些数据5MB, + // 之后再truncate到超过段大小5MB+200Byte,预期截断成功 + public void truncate13() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 200; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 然后append一些数据5MB, + // 之后再truncate到两个段大小10MB,预期截断成功 + public void truncate14() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize * 2; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度大于段大小), + // 然后append一些数据20MB, + // 之后再truncate到两个段大小10MB+10Byte,预期截断成功 + public void truncate15() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize * 4, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize * 2 + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 然后append一些数据20MB, + // 之后再truncate到两个段大小15MB+10Byte,预期截断成功 + public void truncate16() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize * 4, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize * 3 + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 然后append一些数据20MB, + // 之后再truncate到两个段大小15MB+10Byte,预期截断成功 + public void truncate17() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize * 4, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize * 3 + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + newLength = partSize * 2 + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + newLength = partSize + 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + newLength = partSize; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + newLength = partSize - 10; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // 在目标路径的父目录是一个文件的情况下, + // 执行truncate,预期抛AccessControlException + public void truncate18() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String parentDir = "parentDir"; + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + parentDir + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath.getParent()); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + boolean hasException = false; + try { + fs.truncate(srcFilePath, 10); + } catch (AccessControlException e) { + hasException = true; + } + assertTrue(hasException); + + fs.delete(srcFilePath.getParent()); + } + + @Test + // 在目标路径是一个目录的情况下, + // 执行truncate,预期抛FileNotFoundException + public void truncate19() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String parentDir = "parentDir"; + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + parentDir + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + fs.mkdirs(srcFilePath); + + boolean hasException = false; + try { + fs.truncate(srcFilePath, 1); + } catch (FileNotFoundException e) { + hasException = true; + } + assertTrue(hasException); + + fs.delete(srcFilePath.getParent()); + } + + @Test + // create一个文件并写入数据5MB-1Byte(长度小于段大小), + // 之后再truncate到5MB-1Byte(长度不变),预期截断成功 + public void truncate20() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize - 1, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = partSize - 1; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 之后再truncate到5MB+100Byte(长度不变),预期截断成功 + public void truncate21() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 100; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据5MB(长度超过段大小), + // 之后再truncate到5MB+100Byte(长度不变),预期截断成功 + public void truncate22() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 100; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB-1Byte(长度小于段大小), + // 之后再truncate到0,预期截断成功 + public void truncate23() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize - 1, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = 0; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据5MB+100Byte(长度大于段大小), + // 之后再truncate到0,预期截断成功 + public void truncate24() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(partSize + 100, 'a', + 26); + stream.write(data); + stream.close(); + + long newLength = 0; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据5MB(长度超过段大小), + // 之后再truncate到0,预期截断成功 + public void truncate25() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = 0; + assertTrue(fs.truncate(srcFilePath, newLength)); + assertEquals(newLength, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据5MB(长度超过段大小), + // 之后再truncate到5MB+200Byte(长度变大), + // 预期抛HadoopIllegalArgumentException + public void truncate26() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + stream.close(); + + long newLength = partSize + 200; + boolean hasException = false; + try { + fs.truncate(srcFilePath, newLength); + } catch (IOException e) { + if(e.getCause() instanceof HadoopIllegalArgumentException){ + hasException = true; + } + } + assertTrue(hasException); + assertEquals(partSize + 100, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小),但未close文件, + // 之后再truncate到10Byte,预期抛AlreadyBeingCreatedException + public void truncate27() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.hflush(); + + long newLength = 10; + boolean hasException = false; + try { + fs.truncate(srcFilePath, newLength); + } catch (OBSAlreadyBeingCreatedException e) { + hasException = true; + } + assertTrue(hasException); + stream.close(); + assertEquals(100, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } + + @Test + // create一个文件并写入数据100Byte(长度小于段大小), + // 然后append一些数据5MB(长度超过段大小),但未close文件, + // 之后再truncate到5MB,预期抛AlreadyBeingCreatedException + public void truncate28() throws IOException { + if(!fs.isFsBucket()) { + return; + } + + String fileName = "file"; + Path srcFilePath = new Path( + testRootPath + "/" + fileName); + if (fs.exists(srcFilePath)) { + fs.delete(srcFilePath); + } + + FSDataOutputStream stream = fs.create(srcFilePath); + byte[] data = ContractTestUtils.dataset(100, 'a', + 26); + stream.write(data); + stream.close(); + + stream = fs.append(srcFilePath); + data = ContractTestUtils.dataset(partSize, 'a', 26); + stream.write(data); + + long newLength = partSize; + boolean hasException = false; + try { + fs.truncate(srcFilePath, newLength); + } catch (OBSAlreadyBeingCreatedException e) { + hasException = true; + } + assertTrue(hasException); + stream.close(); + assertEquals(partSize + 100, fs.getFileStatus(srcFilePath).getLen()); + + fs.delete(srcFilePath); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSWriteOperationHelper.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSWriteOperationHelper.java new file mode 100644 index 0000000..b4d877a --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/ITestOBSWriteOperationHelper.java @@ -0,0 +1,70 @@ +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertNull; + +import com.obs.services.model.PutObjectRequest; +import com.obs.services.model.UploadPartRequest; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.BeforeClass; +import org.junit.Test; + +import java.io.ByteArrayInputStream; +import java.io.IOException; + +public class ITestOBSWriteOperationHelper { + + private static OBSWriteOperationHelper writeHelper; + + @BeforeClass + public static void beforeClass() throws IOException { + Configuration conf = OBSContract.getConfiguration(null); + OBSFileSystem fileSystem = OBSTestUtils.createTestFileSystem(conf); + writeHelper = new OBSWriteOperationHelper(fileSystem); + } + + @Test + public void testNewPutRequestForChecksumType() { + final String digest = "123"; + byte[] data = new byte[10]; + ByteArrayInputStream inputStream = new ByteArrayInputStream(data); + PutObjectRequest request = + writeHelper.newPutRequest("object01", inputStream, data.length, OBSDataBlocks.ChecksumType.NONE, digest); + assertNull(request.getMetadata().getContentMd5()); + assertFalse(request.getUserHeaders().containsKey(OBSWriteOperationHelper.CONTENT_SHA256)); + + request = writeHelper.newPutRequest("object01", inputStream, data.length, OBSDataBlocks.ChecksumType.MD5, digest); + assertEquals(digest, request.getMetadata().getContentMd5()); + assertFalse(request.getUserHeaders().containsKey(OBSWriteOperationHelper.CONTENT_SHA256)); + + request = + writeHelper.newPutRequest("object01", inputStream, data.length, OBSDataBlocks.ChecksumType.SHA256, digest); + assertNull(request.getMetadata().getContentMd5()); + assertEquals(digest, request.getUserHeaders().get(OBSWriteOperationHelper.CONTENT_SHA256)); + } + + @Test + public void testNewUploadPartRequestForChecksumType() { + final String digest = "123"; + byte[] data = new byte[10]; + ByteArrayInputStream inputStream = new ByteArrayInputStream(data); + UploadPartRequest request = + writeHelper.newUploadPartRequest("object01", "1", 1, 10, inputStream, OBSDataBlocks.ChecksumType.NONE, digest); + assertNull(request.getContentMd5()); + assertFalse(request.getUserHeaders().containsKey(OBSWriteOperationHelper.CONTENT_SHA256)); + + request = + writeHelper.newUploadPartRequest("object01", "1", 1, 10, inputStream, OBSDataBlocks.ChecksumType.MD5, digest); + assertEquals(digest, request.getContentMd5()); + assertFalse(request.getUserHeaders().containsKey(OBSWriteOperationHelper.CONTENT_SHA256)); + + request = + writeHelper.newUploadPartRequest("object01", "1", 1, 10, inputStream, OBSDataBlocks.ChecksumType.SHA256, digest); + assertNull(request.getContentMd5()); + assertEquals(digest, request.getUserHeaders().get(OBSWriteOperationHelper.CONTENT_SHA256)); + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MainTest.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MainTest.java new file mode 100644 index 0000000..4a98945 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MainTest.java @@ -0,0 +1,166 @@ +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.fs.obs.contract.TestOBSContractAppend; +import org.apache.hadoop.fs.obs.contract.TestOBSContractCreate; +import org.apache.hadoop.fs.obs.contract.TestOBSContractDelete; +import org.apache.hadoop.fs.obs.contract.TestOBSContractGetFileStatus; +import org.apache.hadoop.fs.obs.contract.TestOBSContractMkdir; +import org.apache.hadoop.fs.obs.contract.TestOBSContractOpen; +import org.apache.hadoop.fs.obs.contract.TestOBSContractRename; +import org.apache.hadoop.fs.obs.contract.TestOBSContractRootDir; +import org.apache.hadoop.fs.obs.contract.TestOBSContractSeek; +import org.apache.hadoop.fs.obs.security.AuthorizeProviderTest; +import org.junit.internal.TextListener; +import org.junit.runner.JUnitCore; + +/** + * Test main class + */ +public class MainTest { + public static void main(String[] args) { + if (args.length != 4) { + throw new IllegalArgumentException("ERROR: parameters invalid, expected 4 args, but get " + args.length); + } + // initialise test environment info + OBSTestUtils.userAk = args[0]; + OBSTestUtils.userSk = args[1]; + OBSTestUtils.endPoint = args[2]; + OBSTestUtils.bucketName = args[3]; + System.out.println("Begin to run tests!"); + JUnitCore engine = new JUnitCore(); + engine.addListener(new TextListener(System.out)); // required to print reports + System.out.println("Begin to run ITestOBSAppend tests:"); + engine.run(ITestOBSAppend.class); + System.out.println("Begin to run ITestOBSAppendOutputStream tests:"); + engine.run(ITestOBSAppendOutputStream.class); + System.out.println("Begin to run ITestOBSBlockingThreadPoolExecutorService tests:"); + engine.run(ITestOBSBlockingThreadPoolExecutorService.class); + System.out.println("Begin to run ITestOBSBlockOutputArray tests:"); + engine.run(ITestOBSArrayBufferOutputStream.class); + System.out.println("Begin to run ITestOBSBlockOutputStream tests:"); + engine.run(ITestOBSDiskBufferOutputStream.class); + System.out.println("Begin to run ITestOBSBlockSize tests:"); + engine.run(ITestOBSBlockSize.class); + System.out.println("Begin to run ITestOBSBucketAcl tests:"); + engine.run(ITestOBSBucketAcl.class); + System.out.println("Begin to run ITestOBSCloseAndFinalize tests:"); + engine.run(ITestOBSCloseFunction.class); + System.out.println("Begin to run ITestOBSConfiguration tests:"); + engine.run(ITestOBSConfiguration.class); + System.out.println("Begin to run ITestOBSCopyFromLocalFile tests:"); + engine.run(ITestOBSCopyFromLocalFile.class); + System.out.println("Begin to run ITestOBSCreate tests:"); + engine.run(ITestOBSCreate.class); + System.out.println("Begin to run ITestOBSCredentialsInURL tests:"); + engine.run(ITestOBSCredentialsInURL.class); + System.out.println("Begin to run ITestOBSDataBlocks tests:"); + engine.run(ITestOBSDataBlocks.class); + System.out.println("Begin to run ITestOBSDefaultInformation tests:"); + engine.run(ITestOBSDefaultInformation.class); + System.out.println("Begin to run ITestOBSDeleteAndRenameManyFiles tests:"); + engine.run(ITestOBSDeleteAndRenameManyFiles.class); + System.out.println("Begin to run ITestOBSFileSystem tests:"); + engine.run(ITestOBSFileSystem.class); + System.out.println("Begin to run ITestOBSFileSystemContract tests:"); + engine.run(ITestOBSFileSystemContract.class); + System.out.println("Begin to run ITestOBSFSDataOutputStream tests:"); + engine.run(ITestOBSFSDataOutputStream.class); + System.out.println("Begin to run ITestOBSGetAndSetWorkingDirectory tests:"); + engine.run(ITestOBSGetAndSetWorkingDirectory.class); + System.out.println("Begin to run ITestOBSGetContentSummary tests:"); + engine.run(ITestOBSGetContentSummary.class); + System.out.println("Begin to run ITestOBSGetFileStatusAndExist tests:"); + engine.run(ITestOBSGetFileStatusAndExist.class); + System.out.println("Begin to run ITestOBSHFlush tests:"); + engine.run(ITestOBSHFlush.class); + System.out.println("Begin to run ITestOBSInputStream tests:"); + engine.run(ITestOBSInputStream.class); + System.out.println("Begin to run ITestOBSMemArtsCCInputStream tests:"); + engine.run(ITestOBSMemArtsCCInputStream.class); + System.out.println("Begin to run ITestOBSMemArtsCCInputStreamBufIO tests:"); + engine.run(ITestOBSMemArtsCCInputStreamBufIO.class); + System.out.println("Begin to run ITestOBSTruncate tests:"); + engine.run(ITestOBSTruncate.class); + System.out.println("Begin to run ITestOBSListFiles tests:"); + engine.run(ITestOBSListFiles.class); + System.out.println("Begin to run ITestOBSListLocatedStatus tests:"); + engine.run(ITestOBSListLocatedStatus.class); + System.out.println("Begin to run ITestOBSListStatus tests:"); + engine.run(ITestOBSListStatus.class); + System.out.println("Begin to run ITestOBSMetricInfo tests:"); + engine.run(ITestOBSMetricInfo.class); + System.out.println("Begin to run ITestOBSMkdirs tests:"); + engine.run(ITestOBSMkdirs.class); + System.out.println("Begin to run ITestOBSMultiDeleteObjects tests:"); + engine.run(ITestOBSMultiDeleteObjects.class); + System.out.println("Begin to run ITestOBSOpen tests:"); + engine.run(ITestOBSOpen.class); + System.out.println("Begin to run ITestOBSWriteOperationHelper tests:"); + engine.run(ITestOBSWriteOperationHelper.class); + System.out.println("Begin to run ITestOBSOutputStream tests:"); + engine.run(ITestOBSOutputStream.class); + System.out.println("Begin to run ITestOBSRename tests:"); + engine.run(ITestOBSRename.class); + System.out.println("Begin to run ITestOBSRetryMechanism tests:"); + engine.run(ITestOBSRetryMechanism.class); + System.out.println("Begin to run ITestOBSRetryMechanism2 tests:"); + engine.run(ITestOBSRetryMechanism2.class); + System.out.println("Begin to run ITestOBSInvoker tests:"); + engine.run(ITestOBSInvoker.class); + System.out.println("Begin to run ITestOBSBucketPolicy tests:"); + engine.run(ITestOBSBucketPolicy.class); + System.out.println("Begin to run ITestOBSCloseCheck tests:"); + engine.run(ITestOBSCloseProtect.class); + System.out.println("Begin to run ITestOBSTrash tests:"); + engine.run(ITestOBSFastDelete.class); + System.out.println("Begin to run ITestOBSTestUtils tests:"); + engine.run(ITestOBSTestUtils.class); + System.out.println("Begin to run TestOBSContractAppend tests:"); + engine.run(TestOBSContractAppend.class); + System.out.println("Begin to run TestOBSContractCreate tests:"); + engine.run(TestOBSContractCreate.class); + System.out.println("Begin to run TestOBSContractDelete tests:"); + engine.run(TestOBSContractDelete.class); + System.out.println("Begin to run TestOBSContractGetFileStatus tests:"); + engine.run(TestOBSContractGetFileStatus.class); + System.out.println("Begin to run TestOBSContractMkdir tests:"); + engine.run(TestOBSContractMkdir.class); + System.out.println("Begin to run TestOBSContractOpen tests:"); + engine.run(TestOBSContractOpen.class); + System.out.println("Begin to run TestOBSContractRename tests:"); + engine.run(TestOBSContractRename.class); + System.out.println("Begin to run TestOBSContractRootDir tests:"); + engine.run(TestOBSContractRootDir.class); + System.out.println("Begin to run TestOBSContractSeek tests:"); + engine.run(TestOBSContractSeek.class); + System.out.println("Begin to run TestOBSFileContextCreateMkdir tests:"); + engine.run(TestOBSFileContextCreateMkdir.class); + System.out.println("Begin to run TestOBSFileContextMainOperations tests:"); + engine.run(TestOBSFileContextMainOperations.class); + System.out.println("Begin to run TestOBSFileContextURI tests:"); + engine.run(TestOBSFileContextURI.class); + System.out.println("Begin to run TestOBSFileContextUtil tests:"); + engine.run(TestOBSFileContextUtil.class); + System.out.println("Begin to run TestOBSFileSystemContract tests:"); + engine.run(TestOBSFileSystemContract.class); + System.out.println("Begin to run TestOBSFSMainOperations tests:"); + engine.run(TestOBSFSMainOperations.class); + System.out.println("Begin to run ITestOBSHDFSWrapperFileSystem tests:"); + engine.run(ITestOBSHDFSWrapperFileSystem.class); + System.out.println("Begin to run ITestOBSHDFSWrapper tests:"); + engine.run(ITestOBSHDFSWrapper.class); + System.out.println("Begin to run ITestOBSLoginHelper tests:"); + engine.run(ITestOBSLoginHelper.class); + System.out.println("Begin to run AuthorizeProviderTest tests:"); + engine.run(AuthorizeProviderTest.class); + System.out.println("Begin to run ITestOBSMemArtsCCInputStreamTrafficReport tests:"); + engine.run(ITestOBSMemArtsCCInputStreamTrafficReport.class); + System.out.println("Begin to run ITestOBSMemArtsCCInputStreamStatistics tests:"); + engine.run(ITestOBSMemArtsCCInputStreamStatistics.class); + System.out.println("Begin to run ITestOBSFastDeleteV2 tests:"); + engine.run(ITestOBSFastDeleteV2.class); + System.out.println("Begin to run ITestOBSDisguisePermissionSupport tests:"); + engine.run(ITestOBSDisguisePermissionSupport.class); + System.out.println("Running finished."); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MockMetricsConsumer.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MockMetricsConsumer.java new file mode 100644 index 0000000..d15c998 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MockMetricsConsumer.java @@ -0,0 +1,100 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; + +/** + * Default class for consuming metrics. + */ +class MockMetricsConsumer implements BasicMetricsConsumer { + + /** + * Class logger. + */ + private static final Logger LOG = LoggerFactory.getLogger( + MockMetricsConsumer.class); + + /** + * URI of the FileSystem instance. + */ + private URI uri; + + /** + * Configuration of the FileSystem instance. + */ + private Configuration conf; + + private MetricRecord mr; + + /** + * Default metrics consumer that prints debug logs. + * + * @param uriName URI of the owner FileSystem + */ + MockMetricsConsumer(final URI uriName, + final Configuration configuration) { + this.uri = uriName; + this.conf = configuration; + LOG.info("MockMetricsConsumer with URI [{}] and " + + "Configuration[{}]", this.uri, this.conf); + } + + /** + * Put metrics to the consumer. + * + * @param metricRecord metric record to be consumed + * @return true for success, or false for failure + */ + @Override + public boolean putMetrics(MetricRecord metricRecord) { + if (metricRecord.getKind().equals(MetricKind.normal)) { + LOG.debug("[Metrics]: url[{}], action [{}], kind[{}], costTime[{}] ", this.uri, + metricRecord.getObsOperateAction(), metricRecord.getKind(), metricRecord.getCostTime()); + setMetricNormalInfo(metricRecord); + }else { + LOG.debug("[Metrics]: url[{}], action [{}], kind[{}], exception[{}] ", this.uri, + metricRecord.getObsOperateAction(), metricRecord.getKind(), metricRecord.getExceptionIns()); + setMetricAbnormalInfo(metricRecord); + } + return true; + } + + private synchronized void setMetricAbnormalInfo(MetricRecord metricRecord) { + mr = metricRecord; + } + + private synchronized void setMetricNormalInfo(MetricRecord metricRecord) { + mr = metricRecord; + } + + public MetricRecord getMr() { + return mr; + } + + @Override + public void close() { + + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MockOBSClientFactory.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MockOBSClientFactory.java new file mode 100644 index 0000000..23eb331 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/MockOBSClientFactory.java @@ -0,0 +1,39 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one or more + * contributor license agreements. See the NOTICE file distributed with this + * work for additional information regarding copyright ownership. The ASF + * licenses this file to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance with the License. + * You may obtain a copy of the License at + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.mockito.Mockito.*; + +import java.net.URI; + +import com.obs.services.ObsClient; + +/** + * An {@link OBSClientFactory} that returns Mockito mocks of the + * interface suitable for unit testing. + */ +public class MockOBSClientFactory implements OBSClientFactory { + + @Override + public ObsClient createObsClient(URI name) { + String bucket = name.getHost(); + ObsClient obsClient = mock(ObsClient.class); + when(obsClient.headBucket(bucket)).thenReturn(true); + return obsClient; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSAppendTestUtil.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSAppendTestUtil.java new file mode 100644 index 0000000..3972bf8 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSAppendTestUtil.java @@ -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 + *

+ * http://www.apache.org/licenses/LICENSE-2.0 + *

+ * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, WITHOUT + * WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the + * License for the specific language governing permissions and limitations under + * the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.fs.CommonConfigurationKeys; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.DFSInputStream; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.io.OutputStream; +import java.util.Arrays; +import java.util.Random; + +/** + * Utilities for append-related tests + */ +public class OBSAppendTestUtil { + /** + * For specifying the random number generator seed, + * change the following value: + */ + static final Long RANDOM_NUMBER_GENERATOR_SEED = null; + + static final Logger LOG = LoggerFactory.getLogger(OBSAppendTestUtil.class); + + private static final Random SEED = new Random(); + + static { + final long seed = RANDOM_NUMBER_GENERATOR_SEED == null ? + SEED.nextLong() : RANDOM_NUMBER_GENERATOR_SEED; + LOG.info("seed=" + seed); + SEED.setSeed(seed); + } + + private static final ThreadLocal RANDOM + = new ThreadLocal() { + @Override + protected Random initialValue() { + final Random r = new Random(); + synchronized (SEED) { + final long seed = SEED.nextLong(); + r.setSeed(seed); + LOG.info(Thread.currentThread().getName() + ": seed=" + seed); + } + return r; + } + }; + + static final int BLOCK_SIZE = 204800; + + static final int NUM_BLOCKS = 10; + + static final int FILE_SIZE = NUM_BLOCKS * BLOCK_SIZE + 1; + + static long seed = -1; + + static int nextInt() { + return RANDOM.get().nextInt(); + } + + static int nextInt(int n) { + return RANDOM.get().nextInt(n); + } + + static int nextLong() { + return RANDOM.get().nextInt(); + } + + public static byte[] randomBytes(long seed, int size) { + LOG.info("seed=" + seed + ", size=" + size); + final byte[] b = new byte[size]; + final Random rand = new Random(seed); + rand.nextBytes(b); + return b; + } + + /** + * @return a random file partition of length n. + */ + public static int[] randomFilePartition(int n, int parts) { + int[] p = new int[parts]; + for (int i = 0; i < p.length; i++) { + p[i] = nextInt(n - i - 1) + 1; + } + Arrays.sort(p); + for (int i = 1; i < p.length; i++) { + if (p[i] <= p[i - 1]) { + p[i] = p[i - 1] + 1; + } + } + + LOG.info("partition=" + Arrays.toString(p)); + assertTrue("i=0", p[0] > 0 && p[0] < n); + for (int i = 1; i < p.length; i++) { + assertTrue("i=" + i, p[i] > p[i - 1] && p[i] < n); + } + return p; + } + + static void sleep(long ms) { + try { + Thread.sleep(ms); + } catch (InterruptedException e) { + LOG.info("ms=" + ms, e); + } + } + + public static void write(OutputStream out, int offset, int length) + throws IOException { + final byte[] bytes = new byte[length]; + for (int i = 0; i < length; i++) { + bytes[i] = (byte) (offset + i); + } + out.write(bytes); + } + + public static void check(FileSystem fs, Path p, long length) + throws IOException { + int i = -1; + try { + final FileStatus status = fs.getFileStatus(p); + FSDataInputStream in = fs.open(p); + if (in.getWrappedStream() instanceof DFSInputStream) { + long len + = ((DFSInputStream) in.getWrappedStream()).getFileLength(); + assertEquals(length, len); + } else { + assertEquals(length, status.getLen()); + } + + for (i++; i < length; i++) { + assertEquals((byte) i, (byte) in.read()); + } + i = -(int) length; + assertEquals(-1, in.read()); //EOF + in.close(); + } catch (IOException ioe) { + throw new IOException("p=" + p + ", length=" + length + ", i=" + i, + ioe); + } + } + + public static void check(DistributedFileSystem fs, Path p, int position, + int length) throws IOException { + byte[] buf = new byte[length]; + int i = 0; + try { + FSDataInputStream in = fs.open(p); + in.read(position, buf, 0, buf.length); + for (i = position; i < length + position; i++) { + assertEquals((byte) i, buf[i - position]); + } + in.close(); + } catch (IOException ioe) { + throw new IOException("p=" + p + ", length=" + length + ", i=" + i, + ioe); + } + } + + /** + * create a buffer that contains the entire test file data. + */ + public static byte[] initBuffer(int size) { + if (seed == -1) { + seed = nextLong(); + } + return randomBytes(seed, size); + } + + /** + * Creates a file but does not close it + * Make sure to call close() on the returned stream + * + * @throws IOException an exception might be thrown + */ + public static FSDataOutputStream createFile(FileSystem fileSys, Path name, + int repl) + throws IOException { + return fileSys.create(name, true, + fileSys.getConf() + .getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), + (short) repl, BLOCK_SIZE); + } + + public static void checkFullFile(FileSystem fs, Path file, int len, + final byte[] compareContent) throws IOException { + checkFullFile(fs, file, len, compareContent, file.toString()); + } + + /** + * Compare the content of a file created from FileSystem and Path with + * the specified byte[] buffer's content + * + * @throws IOException an exception might be thrown + */ + public static void checkFullFile(FileSystem fs, Path name, int len, + final byte[] compareContent, String message) throws IOException { + checkFullFile(fs, name, len, compareContent, message, true); + } + + public static void checkFullFile(FileSystem fs, Path name, int len, + final byte[] compareContent, String message, + boolean checkFileStatus) throws IOException { + if (checkFileStatus) { + final FileStatus status = fs.getFileStatus(name); + assertEquals( + "len=" + len + " but status.getLen()=" + status.getLen(), + len, status.getLen()); + } + + FSDataInputStream stm = fs.open(name); + byte[] actual = new byte[len]; + stm.readFully(0, actual); + checkData(actual, 0, compareContent, message); + stm.close(); + } + + private static void checkData(final byte[] actual, int from, + final byte[] expected, String message) { + for (int idx = 0; idx < actual.length; idx++) { + assertEquals( + message + " byte " + (from + idx) + " differs. expected " + + expected[from + idx] + " actual " + actual[idx], + expected[from + idx], actual[idx]); + actual[idx] = 0; + } + } + + public static void testAppend(FileSystem fs, Path p) throws IOException { + final int size = 1000; + final byte[] bytes = randomBytes(seed, size); + + { //create file + final FSDataOutputStream out = fs.create(p, (short) 1); + out.write(bytes); + out.close(); + assertEquals(bytes.length, fs.getFileStatus(p).getLen()); + } + + final int appends = 50; + for (int i = 2; i < appends; i++) { + //append + final FSDataOutputStream out = fs.append(p); + out.write(bytes); + out.close(); + assertEquals(i * bytes.length, fs.getFileStatus(p).getLen()); + } + + // Check the appended content + final FSDataInputStream in = fs.open(p); + for (int i = 0; i < appends - 1; i++) { + byte[] read = new byte[size]; + in.read(i * bytes.length, read, 0, size); + assertArrayEquals(bytes, read); + } + in.close(); + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSFSTestUtil.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSFSTestUtil.java new file mode 100644 index 0000000..a78c855 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSFSTestUtil.java @@ -0,0 +1,385 @@ +package org.apache.hadoop.fs.obs; + +import com.obs.services.ObsClient; +import com.obs.services.model.AccessControlList; +import com.obs.services.model.GrantAndPermission; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.*; +import org.apache.hadoop.fs.contract.ContractTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.BufferedWriter; +import java.io.File; +import java.io.FileWriter; +import java.io.IOException; +import java.util.Arrays; +import java.util.EnumSet; +import java.util.Set; + +import static org.junit.Assert.*; + +public class OBSFSTestUtil { + + static final int TEST_BUFF_SIZE = 128; + + static final int MODULUS = 128; + + static byte[] testBuffer = ContractTestUtils.dataset(TEST_BUFF_SIZE, 0, + MODULUS); + + private static final Logger LOGGER = LoggerFactory.getLogger( + OBSFSTestUtil.class); + + public static void setBucketAcl(ObsClient obsClient, String bucketName, + AccessControlList acl) { + assertNotNull("ObsClient should not be null.", obsClient); + assertNotNull("bucketName should not be null.", bucketName); + assertNotNull("acl should not be null.", acl); + + obsClient.setBucketAcl(bucketName, acl); + } + + public static void setBucketAcl(ObsClient obsClient, String bucketName, + AccessControlList acl, + String specificUser, String specificUserPermission, String allUsers, + String allUsersPermission) { + assertNotNull("ObsClient should not be null.", obsClient); + assertNotNull("bucketName should not be null.", bucketName); + assertNotNull("acl should not be null.", acl); + + obsClient.setBucketAcl(bucketName, acl); + AccessControlList accessControlList = obsClient.getBucketAcl( + bucketName); + Set grants = accessControlList.getGrants(); + boolean aclOk = true; + for (GrantAndPermission grant : grants) { + String accountId = grant.getGrantee().getIdentifier(); + String permission = grant.getPermission().getPermissionString(); + if (accountId.equals(specificUser) && !permission.equals( + specificUserPermission)) { + aclOk = false; + break; + } else if (accountId.equals(allUsers) && !permission.equals( + allUsersPermission)) { + aclOk = false; + break; + } + } + assertTrue(aclOk); + } + + public static FSDataOutputStream createStream(FileSystem fs, + Path objectPath) throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + return createStream(fs, objectPath, true); + } + + public static FSDataOutputStream createStream(FileSystem fs, + Path objectPath, boolean overwrite) + throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + FSDataOutputStream stream = fs.create(objectPath, overwrite); + + return stream; + } + + public static FSDataOutputStream createStreamWithFlag(FileSystem fs, + Path objectPath, + boolean overwrite, CreateFlag flag) + throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(flag); + FSDataOutputStream stream = fs.create(objectPath, overwrite); + + return stream; + } + + public static FSDataOutputStream createAppendStream(FileSystem fs, + Path objectPath) throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + return createAppendStream(fs, objectPath, + new FsPermission((short) 00644), flags); + } + + public static FSDataOutputStream createAppendStream(FileSystem fs, + Path objectPath, FsPermission permission, + EnumSet flags) throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + assertEquals(true, flags.contains(CreateFlag.APPEND)); + + FSDataOutputStream stream = fs.create(objectPath, permission, flags, + fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(objectPath), + fs.getDefaultBlockSize(objectPath), null); + + return stream; + } + + public static FSDataOutputStream createNonRecursiveStream(FileSystem fs, + Path objectPath) throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + FsPermission permission = new FsPermission((short) 00644); + + FSDataOutputStream stream = fs.createNonRecursive(objectPath, + permission, flags, + fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(objectPath), + fs.getDefaultBlockSize(objectPath), null); + + return stream; + } + + public static FSDataOutputStream createNonRecursiveAppendStream( + FileSystem fs, Path objectPath) throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + EnumSet flags = EnumSet.noneOf(CreateFlag.class); + flags.add(CreateFlag.APPEND); + FsPermission permission = new FsPermission((short) 00644); + + FSDataOutputStream stream = fs.createNonRecursive(objectPath, + permission, flags, + fs.getConf().getInt("io.file.buffer.size", 4096), + fs.getDefaultReplication(objectPath), + fs.getDefaultBlockSize(objectPath), null); + + return stream; + } + + public static void writeData(FSDataOutputStream outputStream, long len) + throws IOException { + assertNotNull("FSDataOutputStream should not be null.", outputStream); + assertTrue(len >= 0); + + long writtenLen = 0; + long remainingLen; + Throwable var10 = null; + try { + while (writtenLen < len) { + remainingLen = len - writtenLen; + if (remainingLen < testBuffer.length) { + outputStream.write(testBuffer, 0, (int) remainingLen); + writtenLen += remainingLen; + } else { + outputStream.write(testBuffer); + writtenLen += testBuffer.length; + } + } + } catch (Throwable var21) { + var10 = var21; + throw var21; + } finally { + if (var10 != null) { + try { + outputStream.close(); + } catch (Throwable var20) { + var10.addSuppressed(var20); + } + } + } + + assertEquals(len, writtenLen); + } + + public static void deletePathRecursive(FileSystem fs, Path objectPath) + throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + if (fs.exists(objectPath)) { + fs.delete(objectPath, true); + } + } + + public static void assertFileHasLength(FileSystem fs, Path objectPath, + int expected) throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + + ContractTestUtils.assertFileHasLength(fs, objectPath, expected); + } + + public static void assertPathExistence(FileSystem fs, Path objectPath, + boolean shouldExist) throws IOException { + if (shouldExist) { + ContractTestUtils.assertPathExists(fs, "Path not exist.", + objectPath); + } else { + ContractTestUtils.assertPathDoesNotExist(fs, "Path still exist.", + objectPath); + } + } + + public static void readFile(FileSystem fs, Path objectPath, long start, + long needReadLen) throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + assertTrue("start and readLen should not be negative.", + start >= 0 && needReadLen >= 0); + + // FileStatus fileStatus = fs.getFileStatus(objectPath); + // if (!fileStatus.isFile() || fileStatus.getLen() < start || fileStatus.getLen() < needReadLen) { + // throw new IOException("invalid read"); + // } + + FSDataInputStream inputStream = null; + try { + inputStream = fs.open(objectPath); + inputStream.seek(start); + byte[] readBuffer = new byte[TEST_BUFF_SIZE]; + long totalBytesRead = 0L; + long remainingLen; + while (true) { + remainingLen = needReadLen - totalBytesRead; + if (remainingLen <= 0) { + break; + } + + int bytesRead = inputStream.read(readBuffer, 0, + remainingLen > readBuffer.length + ? readBuffer.length + : (int) remainingLen); + if (bytesRead < 0) { + if (totalBytesRead != needReadLen) { + throw new IOException("Expected to read " + needReadLen + + " bytes but only received " + + totalBytesRead); + } + break; + } + + totalBytesRead += bytesRead; + } + + assertEquals(needReadLen, totalBytesRead); + assertEquals(0, remainingLen); + } catch (IOException e) { + throw e; + } finally { + if (inputStream != null) { + inputStream.close(); + } + } + } + + public static void verifyReceivedData(FileSystem fs, Path objectPath, + long expectedSize) + throws IOException { + assertNotNull("FileSystem should not be null.", fs); + assertNotNull("Path should not be null.", objectPath); + assertTrue("expectedSize should be positive.", expectedSize >= 0); + + ContractTestUtils.verifyReceivedData(fs, objectPath, expectedSize, + testBuffer.length, MODULUS); + } + + public static boolean createLocalTestDir(String relativePath) { + File dir = new File(relativePath); + if (dir.exists()) { + return true; + } + + return dir.mkdirs(); + } + + public static File createLocalTestFile(String relativeFileName) + throws IOException { + File localFile = new File(relativeFileName); + if (localFile.exists()) { + deleteLocalFile(relativeFileName); + } + + if (null != localFile.getParentFile() && !localFile.getParentFile() + .exists()) { + LOGGER.error(String.format("parentFile %s not exist!", + localFile.getParent())); + throw new IOException(String.format("parentFile %s not exist!", + localFile.getParent())); + } + + if (localFile.createNewFile()) { + return localFile; + } else { + LOGGER.error(String.format("create file %s failed!", + localFile.getParent())); + throw new IOException(String.format("create file %s failed!", + localFile.getParent())); + } + } + + public static void writeLocalFile(File localFile, long size) + throws IOException { + BufferedWriter writer = null; + try { + writer = new BufferedWriter(new FileWriter(localFile)); + long writtenLen = 0; + long remainingLen; + while (writtenLen < size) { + remainingLen = size - writtenLen; + if (remainingLen >= testBuffer.length) { + writer.write(new String(testBuffer)); + writtenLen += testBuffer.length; + } else { + writer.write(new String( + Arrays.copyOfRange(testBuffer, 0, (int) remainingLen))); + writtenLen += remainingLen; + } + } + writer.close(); + + assertEquals(size, writtenLen); + } finally { + if (writer != null) { + writer.close(); + } + } + } + + public static void deleteLocalFile(String relativePath) { + File file = new File(relativePath); + if (!file.exists()) { + return; + } + + if (file.isFile() || file.listFiles().length == 0) { + file.delete(); + } else { + for (File subFile : file.listFiles()) { + deleteLocalFile(subFile.getPath()); + } + file.delete(); + } + + return; + } + + public static void clearBucket() throws IOException { + Configuration conf = OBSContract.getConfiguration(null); + OBSFileSystem fs = OBSTestUtils.createTestFileSystem(conf); + Path path = new Path("/"); + FileStatus[] fs_list = fs.listStatus(path); + for(FileStatus f : fs_list) { + fs.delete(f.getPath(),true); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java new file mode 100644 index 0000000..a8af77f --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestConstants.java @@ -0,0 +1,61 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +/** + * Constants for OBS Testing. + */ +public class OBSTestConstants { + + /** + * Prefix for any cross-filesystem scale test options. + */ + static final String SCALE_TEST = "scale.test."; + + /** + * Name of the test filesystem. + */ + public static final String TEST_FS_OBS_NAME = "fs.contract.test.fs.obs"; + + /** + * Run the encryption tests? + */ + static final String KEY_ENCRYPTION_TESTS ="fs.contract.test.fs.obs" + + ".encryption.enabled"; + + /** + * The number of operations to perform: {@value}. + */ + static final String KEY_OPERATION_COUNT = SCALE_TEST + "operation.count"; + + /** + * The default number of operations to perform: {@value}. + */ + static final long DEFAULT_OPERATION_COUNT = 1000; + + /** + * Fork ID passed down from maven if the test is running in parallel. + */ + static final String TEST_UNIQUE_FORK_ID = "test.unique.fork.id"; + + /** + * Timeout in Milliseconds for standard tests: {@value}. + */ + public static final int OBS_TEST_TIMEOUT = 10 * 60 * 1000; +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestRule.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestRule.java new file mode 100644 index 0000000..4897600 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestRule.java @@ -0,0 +1,35 @@ +package org.apache.hadoop.fs.obs; + +import org.junit.rules.TestRule; +import org.junit.runner.Description; +import org.junit.runners.model.Statement; + +public class OBSTestRule implements TestRule { + @Override + public Statement apply(final Statement statement, + final Description description) { + return new OBSStatement(statement, description); + } + + class OBSStatement extends Statement { + private final Statement base; + private final Description description; + + public OBSStatement(Statement base, Description description) { + this.base = base; + this.description = description; + } + + @Override + public void evaluate() throws Throwable { + String methodName = description.getMethodName(); + String beforeMsg = String.format("Begin to run testcase %s", + methodName); + String afterMsg = String.format("Finish run testcase %s success", + methodName); + System.out.println(beforeMsg); + base.evaluate(); + System.out.println(afterMsg); + } + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java new file mode 100644 index 0000000..5001d12 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/OBSTestUtils.java @@ -0,0 +1,402 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.Path; +import org.junit.Assume; +import org.junit.internal.AssumptionViolatedException; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.io.IOException; +import java.net.URI; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; +import static org.apache.hadoop.fs.obs.OBSTestConstants.*; +import static org.apache.hadoop.fs.obs.OBSConstants.*; +import static org.junit.Assert.*; + +/** + * Utilities for the OBS tests. + */ +public final class OBSTestUtils { + private static final Logger LOG = LoggerFactory.getLogger( + OBSTestUtils.class); + + /** + * Value to set a system property to (in maven) to declare that a property + * has been unset. + */ + public static final String UNSET_PROPERTY = "unset"; + + /** + * AK. + */ + public static String userAk = ""; + + /** + * SK. + */ + public static String userSk = ""; + + /** + * End point. + */ + public static String endPoint = ""; + + /** + * Bucket name. + */ + public static String bucketName = ""; + + /** + * Create the test filesystem. + *

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

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

+ * If the test.fs.obs.name property is not set, this will trigger a JUnit + * failure. + * + * @param conf configuration + * @param purge flag to enable Multipart purging + * @return the FS + * @throws IOException IO Problems + * @throws AssumptionViolatedException if the FS is not named + */ + public static OBSFileSystem createTestFileSystem(Configuration conf, + boolean purge) + throws IOException { + + String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, ""); + + boolean liveTest = !OBSCommonUtils.isStringEmpty(fsname); + URI testURI = null; + if (liveTest) { + testURI = URI.create(fsname); + liveTest = testURI.getScheme().equals(OBSConstants.OBS_SCHEME); + } + if (!liveTest) { + // This doesn't work with our JUnit 3 style test cases, so instead we'll + // make this whole class not run by default + throw new AssumptionViolatedException( + "No test filesystem in " + TEST_FS_OBS_NAME); + } + OBSFileSystem fs1 = new OBSFileSystem(); + //enable purging in tests + if (purge) { + conf.setBoolean(PURGE_EXISTING_MULTIPART, true); + // but a long delay so that parallel multipart tests don't + // suddenly start timing out + conf.setInt(PURGE_EXISTING_MULTIPART_AGE, 30 * 60); + } + fs1.initialize(testURI, conf); + return fs1; + } + + /** + * Create a file context for tests. + *

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

+ * Multipart purging is enabled. + * + * @param conf configuration + * @return the FS + * @throws IOException IO Problems + * @throws AssumptionViolatedException if the FS is not named + */ + public static FileContext createTestFileContext(Configuration conf) + throws IOException { + String fsname = conf.getTrimmed(TEST_FS_OBS_NAME, ""); + + boolean liveTest = !OBSCommonUtils.isStringEmpty(fsname); + URI testURI = null; + if (liveTest) { + testURI = URI.create(fsname); + liveTest = testURI.getScheme().equals(OBSConstants.OBS_SCHEME); + } + if (!liveTest) { + // This doesn't work with our JUnit 3 style test cases, so instead we'll + // make this whole class not run by default + throw new AssumptionViolatedException("No test filesystem in " + + TEST_FS_OBS_NAME); + } + FileContext fc = FileContext.getFileContext(testURI, conf); + return fc; + } + + /** + * Get a long test property. + *

    + *
  1. Look up configuration value (which can pick up core-default.xml), + * using {@code defVal} as the default value (if conf != null). + *
  2. + *
  3. Fetch the system property.
  4. + *
  5. If the system property is not empty or "(unset)": + * it overrides the conf value. + *
  6. + *
+ * This puts the build properties in charge of everything. It's not a + * perfect design; having maven set properties based on a file, as ant let + * you do, is better for customization. + *

+ * As to why there's a special (unset) value, see + * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven} + * + * @param conf config: may be null + * @param key key to look up + * @param defVal default value + * @return the evaluated test property. + */ + public static long getTestPropertyLong(Configuration conf, + String key, long defVal) { + return Long.valueOf( + getTestProperty(conf, key, Long.toString(defVal))); + } + + /** + * Get a test property value in bytes, using k, m, g, t, p, e suffixes. + * {@link org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix#string2long(String)} + *

    + *
  1. Look up configuration value (which can pick up core-default.xml), + * using {@code defVal} as the default value (if conf != null). + *
  2. + *
  3. Fetch the system property.
  4. + *
  5. If the system property is not empty or "(unset)": + * it overrides the conf value. + *
  6. + *
+ * This puts the build properties in charge of everything. It's not a + * perfect design; having maven set properties based on a file, as ant let + * you do, is better for customization. + *

+ * As to why there's a special (unset) value, see + * {@link http://stackoverflow.com/questions/7773134/null-versus-empty-arguments-in-maven} + * + * @param conf config: may be null + * @param key key to look up + * @param defVal default value + * @return the evaluated test property. + */ + public static long getTestPropertyBytes(Configuration conf, + String key, String defVal) { + return org.apache.hadoop.util.StringUtils.TraditionalBinaryPrefix + .string2long(getTestProperty(conf, key, defVal)); + } + + /** + * Get an integer test property; algorithm described in {@link + * #getTestPropertyLong(Configuration, String, long)}. + * + * @param key key to look up + * @param defVal default value + * @return the evaluated test property. + */ + public static int getTestPropertyInt(Configuration conf, + String key, int defVal) { + return (int) getTestPropertyLong(conf, key, defVal); + } + + /** + * Get a boolean test property; algorithm described in {@link + * #getTestPropertyLong(Configuration, String, long)}. + * + * @param key key to look up + * @param defVal default value + * @return the evaluated test property. + */ + public static boolean getTestPropertyBool(Configuration conf, + String key, + boolean defVal) { + return Boolean.valueOf( + getTestProperty(conf, key, Boolean.toString(defVal))); + } + + /** + * Get a string test property. + *

    + *
  1. Look up configuration value (which can pick up core-default.xml), + * using {@code defVal} as the default value (if conf != null). + *
  2. + *
  3. Fetch the system property.
  4. + *
  5. If the system property is not empty or "(unset)": + * it overrides the conf value. + *
  6. + *
+ * This puts the build properties in charge of everything. It's not a + * perfect design; having maven set properties based on a file, as ant let + * you do, is better for customization. + *

+ * As to why there's a special (unset) value, see + * + * @param conf config: may be null + * @param key key to look up + * @param defVal default value + * @return the evaluated test property. + * @see + * Stack Overflow + */ + + public static String getTestProperty(Configuration conf, + String key, + String defVal) { + String confVal = conf != null ? conf.getTrimmed(key, defVal) : defVal; + String propval = System.getProperty(key); + return + OBSCommonUtils.isStringNotEmpty(propval) && !UNSET_PROPERTY.equals(propval) + ? propval : confVal; + } + + /** + * Verify the class of an exception. If it is not as expected, rethrow it. + * Comparison is on the exact class, not subclass-of inference as offered by + * {@code instanceof}. + * + * @param clazz the expected exception class + * @param ex the exception caught + * @return the exception, if it is of the expected class + * @throws Exception the exception passed in. + */ + public static Exception verifyExceptionClass(Class clazz, + Exception ex) + throws Exception { + if (!(ex.getClass().equals(clazz))) { + throw ex; + } + return ex; + } + + /** + * Turn off FS Caching: use if a filesystem with different options from the + * default is required. + * + * @param conf configuration to patch + */ + public static void disableFilesystemCaching(Configuration conf) { + conf.setBoolean("fs.obs.impl.disable.cache", true); + } + + /** + * Skip a test if encryption tests are disabled. + * + * @param configuration configuration to probe + */ + public static void skipIfEncryptionTestsDisabled( + Configuration configuration) { + if (!configuration.getBoolean(KEY_ENCRYPTION_TESTS, true)) { + skip("Skipping encryption tests"); + } + } + + /** + * Create a test path, using the value of {@link OBSTestConstants#TEST_UNIQUE_FORK_ID} + * if it is set. + * + * @param defVal default value + * @return a path + */ + public static Path createTestPath(Path defVal) { + String testUniqueForkId = System.getProperty( + OBSTestConstants.TEST_UNIQUE_FORK_ID); + return testUniqueForkId == null ? defVal : + new Path("/" + testUniqueForkId, "test"); + } + + /** + * This class should not be instantiated. + */ + private OBSTestUtils() { + } + + /** + * Set a bucket specific property to a particular value. If the generic key + * passed in has an {@code fs.obs. prefix}, that's stripped off, so that + * when the the bucket properties are propagated down to the generic values, + * that value gets copied down. + * + * @param conf configuration to set + * @param bucket bucket name + * @param genericKey key; can start with "fs.obs." + * @param value value to set + */ + public static void setBucketOption(Configuration conf, String bucket, + String genericKey, String value) { + final String baseKey = genericKey.startsWith(FS_OBS_PREFIX) ? + genericKey.substring(FS_OBS_PREFIX.length()) + : genericKey; + conf.set(FS_OBS_BUCKET_PREFIX + bucket + '.' + baseKey, value); + } + + /** + * Assert that a configuration option matches the expected value. + * + * @param conf configuration + * @param key option key + * @param expected expected value + */ + public static void assertOptionEquals(Configuration conf, + String key, + String expected) { + assertEquals("Value of " + key, expected, conf.get(key)); + } + + /** + * Assume that a condition is met. If not: log at WARN and then throw an + * {@link AssumptionViolatedException}. + * + * @param message + * @param condition + */ + public static void assume(String message, boolean condition) { + if (!condition) { + LOG.warn(message); + } + Assume.assumeTrue(message, condition); + } + + /** + * Generate unique test path for multiple user tests. + * + * @return root test path + */ + public static String generateUniqueTestPath() { + String testUniqueForkId = System.getProperty("test.unique.fork.id"); + return testUniqueForkId == null ? "/test" : + "/" + testUniqueForkId + "/test"; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java new file mode 100644 index 0000000..90a4a6a --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFSMainOperations.java @@ -0,0 +1,94 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.TestFSMainOperationsLocalFileSystem; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.Before; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +/** + *

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

+ *

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

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

+ */ +public class TestOBSFSMainOperations extends + TestFSMainOperationsLocalFileSystem { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + @Before + public void setUp() throws Exception { + skipTestCheck(); + Configuration conf = OBSContract.getConfiguration(null); + fSys = OBSTestUtils.createTestFileSystem(conf); + } + + @Override + public void testListStatusThrowsExceptionForUnreadableDir() { + skip("Unsupport."); + } + + @Override + @After + public void tearDown() throws Exception { + if (fSys != null) { + super.tearDown(); + } + } + + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Test + @Override + public void testRenameNonExistentPath() throws Exception { + skip("Unsupport."); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java new file mode 100644 index 0000000..d16c5a0 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextCreateMkdir.java @@ -0,0 +1,83 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileContextCreateMkdirBaseTest; +import org.apache.hadoop.fs.FileContextTestHelper; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; + +import java.io.IOException; +import java.net.URI; +import java.util.UUID; + +/** + * File context create mkdir test cases on obs file system. + */ +public class TestOBSFileContextCreateMkdir extends + FileContextCreateMkdirBaseTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Override + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.addResource(OBSContract.CONTRACT_XML); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + FileSystem fs = OBSTestUtils.createTestFileSystem(conf); + fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + super.setUp(); + } + + @Override + protected FileContextTestHelper createFileContextHelper() { + // On Windows, root directory path is created from local running + // directory. + // obs does not support ':' as part of the path which results in + // failure. + return new FileContextTestHelper(UUID.randomUUID().toString()); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java new file mode 100644 index 0000000..b4d6389 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextMainOperations.java @@ -0,0 +1,116 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileContextMainOperationsBaseTest; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; +import java.net.URI; + +/** + * Rename test cases on obs file system. + */ +public class TestOBSFileContextMainOperations extends + FileContextMainOperationsBaseTest { + + private OBSFileSystem fs; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Override + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + fs = OBSTestUtils.createTestFileSystem(conf); + fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + super.setUp(); + } + + @Override + protected boolean listCorruptedBlocksSupported() { + return false; + } + + @Override + @Test + public void testSetVerifyChecksum() { + skip("Unsupport."); + } + + @Test + @Override + public void testCreateFlagCreateAppendExistingFile() throws IOException { + if(!fs.isFsBucket()) { + return; + }else { + super.testCreateFlagCreateAppendExistingFile(); + } + } + + @Test + @Override + public void testWriteInNonExistentDirectory() throws IOException { + if(!fs.isFsBucket()) { + return; + }else { + super.testWriteInNonExistentDirectory(); + } + } + + @Test + @Override + public void testCreateFlagAppendExistingFile() throws IOException { + if(!fs.isFsBucket()) { + return; + }else { + super.testCreateFlagAppendExistingFile(); + } + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java new file mode 100644 index 0000000..2495542 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileContextURI.java @@ -0,0 +1,191 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.DelegateToFileSystem; +import org.apache.hadoop.fs.FileContext; +import org.apache.hadoop.fs.FileContextTestHelper; +import org.apache.hadoop.fs.FileContextURIBase; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.apache.hadoop.fs.permission.FsPermission; +import org.junit.AfterClass; +import org.junit.Assert; +import org.junit.Assume; +import org.junit.Before; +import org.junit.BeforeClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; +import java.net.URI; + +/** + *

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

+ * + *

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

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

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

+ */ +public class TestOBSFileContextURI extends FileContextURIBase { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Override + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.addResource(OBSContract.CONTRACT_XML); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + FileSystem fs = OBSTestUtils.createTestFileSystem(conf); + fc1 = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + + fc2 = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + super.setUp(); + } + + @Override + public void testFileStatus() { + skip("Unsupport."); + } + + + @Test + @Override + public void testCreateFile() throws IOException { + String[] fileNames = new String[]{"testFile", "test File", "test*File", "test#File", "test1234", "1234Test", "test)File", "test_File", "()&^%$#@!~_+}{> + * A collection of Util tests for the {@link FileContext#util()}. This test + * should be used for testing an instance of {@link FileContext#util()} that has + * been initialized to a specific default FileSystem such a LocalFileSystem, + * HDFS,OBS, etc. + *

+ *

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

+ */ +public class TestOBSFileContextUtil extends FileContextUtilBase { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @BeforeClass + public static void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @Override + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + conf.addResource(OBSContract.CONTRACT_XML); + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + throw new Exception("Default file system not configured."); + } + + URI uri = new URI(fileSystem); + FileSystem fs = OBSTestUtils.createTestFileSystem(conf); + fc = FileContext.getFileContext(new DelegateToFileSystem(uri, fs, + conf, fs.getScheme(), false) { + }, conf); + super.setUp(); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java new file mode 100644 index 0000000..fb84a87 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/TestOBSFileSystemContract.java @@ -0,0 +1,71 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystemContractBaseTest; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.AfterClass; +import org.junit.Assume; +import org.junit.Before; + +import java.io.IOException; + +/** + * Tests a live OBS system. If your keys and bucket aren't specified, all tests + * are marked as passed. + *

+ * This uses BlockJUnit4ClassRunner because FileSystemContractBaseTest from + * TestCase which uses the old Junit3 runner that doesn't ignore assumptions + * properly making it impossible to skip the tests if we don't have a valid + * bucket. + **/ +@Deprecated +public class TestOBSFileSystemContract extends FileSystemContractBaseTest { + + @Before + public void setUp() throws Exception { + skipTestCheck(); + Configuration conf = OBSContract.getConfiguration(null); + conf.addResource(OBSContract.CONTRACT_XML); + fs = OBSTestUtils.createTestFileSystem(conf); + System.out.println("Begin to run testcase TestOBSFileSystemContract"); + } + + @After + public void tearDown() throws Exception { + super.tearDown(); + System.out.println("Finish run testcase TestOBSFileSystemContract success"); + } + + @Override + public void testMkdirsWithUmask() { + Assume.assumeTrue("unsupport.", true); + } + + public void skipTestCheck() { + Assume.assumeTrue(OBSContract.isContractTestEnabled()); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/OBSContract.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/OBSContract.java new file mode 100644 index 0000000..6b68078 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/OBSContract.java @@ -0,0 +1,84 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.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.obs.OBSConstants; +import org.apache.hadoop.fs.obs.OBSTestConstants; +import org.apache.hadoop.fs.obs.OBSTestUtils; + +/** + * The contract of OBS: only enabled if the test bucket is provided. + */ +public class OBSContract extends AbstractBondedFSContract { + + public static final String CONTRACT_XML = "contract/obs.xml"; + + public OBSContract(Configuration conf) { + super(conf); + //insert the base features + addConfResource(CONTRACT_XML); + this.setConf(getConfiguration(conf)); + } + + @Override + public String getScheme() { + return "obs"; + } + + @Override + public Path getTestPath() { + return OBSTestUtils.createTestPath(super.getTestPath()); + } + + public synchronized static boolean isContractTestEnabled() { + Configuration conf = null; + boolean isContractTestEnabled = true; + + if (conf == null) { + conf = getConfiguration(null); + } + String fileSystem = conf.get(OBSTestConstants.TEST_FS_OBS_NAME); + if (fileSystem == null || fileSystem.trim().length() == 0) { + isContractTestEnabled = false; + } + return isContractTestEnabled; + } + + public synchronized static Configuration getConfiguration( + final Configuration conf) { + Configuration newConf = conf == null ? new Configuration() + : new Configuration(conf); + newConf.addResource(CONTRACT_XML); + if (!OBSTestUtils.userAk.equals("") + && !OBSTestUtils.userSk.equals("") + && !OBSTestUtils.endPoint.equals("") + && !OBSTestUtils.endPoint.equals("")) { + newConf.set(OBSConstants.ACCESS_KEY, OBSTestUtils.userAk); + newConf.set(OBSConstants.SECRET_KEY, OBSTestUtils.userSk); + newConf.set(OBSConstants.ENDPOINT, OBSTestUtils.endPoint); + newConf.set(OBSTestConstants.TEST_FS_OBS_NAME, + OBSTestUtils.bucketName); + } + + return newConf; + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractAppend.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractAppend.java new file mode 100644 index 0000000..fa6fad4 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractAppend.java @@ -0,0 +1,88 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import static org.apache.hadoop.fs.contract.ContractTestUtils.skip; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractAppendTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSFileSystem; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.apache.hadoop.fs.obs.OBSTestUtils; +import org.junit.AfterClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +/** + * Append test cases on obs file system. + */ +public class TestOBSContractAppend extends AbstractContractAppendTest { + + private OBSFileSystem fs; + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract(final Configuration config) { + return new OBSContract(config); + } + + @Override + public void setup() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + super.setup(); + } + + @Test + @Override + public void testRenameFileBeingAppended() throws Throwable { + skip("Unsupport."); + } + + @Test + @Override + public void testAppendToEmptyFile() throws Throwable { + if(!fs.isFsBucket()) { + return; + }else { + super.testAppendToEmptyFile(); + } + } + + @Test + @Override + public void testAppendToExistingFile() throws Throwable { + if(!fs.isFsBucket()) { + return; + }else { + super.testAppendToExistingFile(); + } + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractCreate.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractCreate.java new file mode 100644 index 0000000..32494a2 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractCreate.java @@ -0,0 +1,62 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractCreateTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.junit.AfterClass; +import org.junit.Rule; + +import java.io.IOException; + +/** + * Create test cases on obs file system. + */ +public class TestOBSContractCreate extends AbstractContractCreateTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @Override + public void testCreatedFileIsImmediatelyVisible() throws Throwable { + this.getContract().getConf().setBoolean("fs.contract.create-visibility-delayed", + true); + super.testCreatedFileIsImmediatelyVisible(); + } + + public void testCreatedFileIsVisibleOnFlush() throws Throwable { + this.getContract().getConf().setBoolean("fs.contract.is-blobstore", + true); + super.testCreatedFileIsVisibleOnFlush(); + } + + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractDelete.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractDelete.java new file mode 100644 index 0000000..12cd36e --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractDelete.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractDeleteTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.junit.AfterClass; +import org.junit.Rule; + +import java.io.IOException; + +/** + * Delete test cases on obs file system. + */ +public class TestOBSContractDelete extends AbstractContractDeleteTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractGetFileStatus.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractGetFileStatus.java new file mode 100644 index 0000000..0f671a3 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractGetFileStatus.java @@ -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 + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractGetFileStatusTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.junit.Rule; + +/** + * Get file status test cases on obs file system. + */ +public class TestOBSContractGetFileStatus extends + AbstractContractGetFileStatusTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract( + final Configuration conf) { + return new OBSContract(conf); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractMkdir.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractMkdir.java new file mode 100644 index 0000000..ab1df59 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractMkdir.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractMkdirTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.junit.AfterClass; +import org.junit.Rule; + +import java.io.IOException; + +/** + * Mkdir test cases on obs file system. + */ +public class TestOBSContractMkdir extends AbstractContractMkdirTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractOpen.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractOpen.java new file mode 100644 index 0000000..4924528 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractOpen.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractOpenTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.junit.AfterClass; +import org.junit.Rule; + +import java.io.IOException; + +/** + * Open test cases on obs file system. + */ +public class TestOBSContractOpen extends AbstractContractOpenTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractRename.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractRename.java new file mode 100644 index 0000000..f33e60f --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractRename.java @@ -0,0 +1,48 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractRenameTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.junit.AfterClass; +import org.junit.Rule; + +import java.io.IOException; + +/** + * Rename test cases on obs file system. + */ +public class TestOBSContractRename extends AbstractContractRenameTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractRootDir.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractRootDir.java new file mode 100644 index 0000000..be12709 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractRootDir.java @@ -0,0 +1,70 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.contract.AbstractContractRootDirectoryTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSFileSystem; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.apache.hadoop.fs.obs.OBSTestUtils; +import org.junit.AfterClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.IOException; + +/** + * Root directory test cases on obs file system. + */ +public class TestOBSContractRootDir extends AbstractContractRootDirectoryTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + private OBSFileSystem fs; + + @Override + public void setup() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + super.setup(); + } + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + @Test + @Override + public void testRmNonEmptyRootDirNonRecursive() throws Throwable { + if(!fs.isFsBucket()) { + return; + }else { + super.testRmNonEmptyRootDirNonRecursive(); + } + } + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractSeek.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractSeek.java new file mode 100644 index 0000000..600cd18 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/contract/TestOBSContractSeek.java @@ -0,0 +1,52 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.fs.obs.contract; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.contract.AbstractContractSeekTest; +import org.apache.hadoop.fs.contract.AbstractFSContract; +import org.apache.hadoop.fs.obs.OBSFSTestUtil; +import org.apache.hadoop.fs.obs.OBSTestRule; +import org.junit.AfterClass; +import org.junit.Rule; +import org.junit.Test; + +import java.io.EOFException; +import java.io.IOException; + +/** + * Seek test cases on obs file system. + */ +public class TestOBSContractSeek extends AbstractContractSeekTest { + + @Rule + public OBSTestRule testRule = new OBSTestRule(); + + @Override + protected AbstractFSContract createContract(final Configuration conf) { + return new OBSContract(conf); + } + + + @AfterClass + public static void clearBucket() throws IOException { + OBSFSTestUtil.clearBucket(); + } +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/mock/MockMemArtsCCClient.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/mock/MockMemArtsCCClient.java new file mode 100644 index 0000000..efbaaba --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/mock/MockMemArtsCCClient.java @@ -0,0 +1,178 @@ +package org.apache.hadoop.fs.obs.mock; + +import com.obs.services.ObsClient; +import com.obs.services.model.GetObjectRequest; +import com.obs.services.model.ObsObject; + +import org.apache.hadoop.fs.obs.OBSFileSystem; +import org.apache.hadoop.fs.obs.TrafficStatistics; +import org.apache.hadoop.fs.obs.memartscc.CcGetShardParam; +import org.apache.hadoop.fs.obs.memartscc.MemArtsCCClient; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InputStream; +import java.nio.ByteBuffer; + +public class MockMemArtsCCClient extends MemArtsCCClient { + private OBSFileSystem fs; + + // private Configuration conf; + + private boolean initFail; + + // private OBSInputStream underlyingStream; + + private long bytesReaded = 0; + + String ak; + + String sk; + + String token; + + String endpoint; + + String bucket; + + boolean enablePosix; + + boolean nextIOException; + + boolean nextEOFException; + + boolean nextCacheMiss; + + private class StatisticsTotal { + long originalTraffic; + + long applicationTraffic; + + long missTraffic; + + long hitTraffic; + } + + StatisticsTotal statisticsTotal = new StatisticsTotal(); + + public MockMemArtsCCClient(final OBSFileSystem obsFileSystem, boolean initFail, String bucket) { + super("", true); // nouse + this.fs = obsFileSystem; + this.initFail = initFail; + this.bucket = bucket; + MockMemArtsCCClient.bufferPool.initialize(128, 1024*1024); + } + + @Override + public int init(String config, String otherInfo) { + if (initFail) { + return -1; + } + return 0; + } + + @Override + public int read(boolean isPrefetch, long prefetchStart, long prefetchEnd, ByteBuffer buf, long offset, long len, + String objectKey, long modifyTime, String etag, boolean isConsistencyCheck) throws EOFException, IOException { + if (nextReadCacheMiss()) { + return CCREAD_RETCODE_CACHEMISS; + } + if (nextReadThrowEOFException()) { + throw new EOFException("mock ccread EOFException"); + } + if (nextReadThrowIOException()) { + throw new IOException("mock ccread IOException"); + } + if (len == 0) { + return 0; + } + GetObjectRequest req = new GetObjectRequest(); + req.setBucketName(this.bucket); + req.setObjectKey(objectKey); + req.setRangeStart(offset); + req.setRangeEnd(offset + len - 1); + ObsClient client = this.fs.getObsClient(); + ObsObject obj = client.getObject(req); + InputStream is = obj.getObjectContent(); + int bread = 0; + int off = 0; + byte[] tmpBuf = new byte[(int)len]; + int ret = is.read(tmpBuf, off, (int) len); + this.bytesReaded += ret; + bread += ret; + off += ret; + while (ret > 0) { + ret = is.read(tmpBuf, off, (int) len - off); + off += ret; + bread += ret; + } + is.close(); + buf.position(0); + buf.limit(bread); + buf.put(tmpBuf, 0, bread); + return bread; + } + + @Override + public int getObjectShardInfo(CcGetShardParam ccGetShardParam) { + throw new UnsupportedOperationException("un-implemented mock ccGetObjectShardInfo"); + } + + @Override + public void close() { + + } + + @Override + public void reportReadStatistics(TrafficStatistics trafficStatistics) { + long Q = trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q); + long QDot = trafficStatistics.getStatistics(TrafficStatistics.TrafficType.QDot); + long Q2 = trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q2); + long Q1 = trafficStatistics.getStatistics(TrafficStatistics.TrafficType.Q1); + + statisticsTotal.originalTraffic += Q; + statisticsTotal.applicationTraffic += QDot; + statisticsTotal.hitTraffic += Q2; + statisticsTotal.missTraffic += Q1; + } + + public void printTotalStatistics() { + System.out.printf("Total: Q:%d Q`:%d Q2:%d Q1:%d\n", + statisticsTotal.originalTraffic, + statisticsTotal.applicationTraffic, + statisticsTotal.hitTraffic, + statisticsTotal.missTraffic); + } + + public boolean nextReadThrowIOException() { + boolean ret = this.nextIOException; + this.nextEOFException = false; + return ret; + } + + public boolean nextReadThrowEOFException() { + boolean ret = this.nextEOFException; + this.nextEOFException = false; + return ret; + } + + public boolean nextReadCacheMiss() { + boolean ret = this.nextCacheMiss; + this.nextCacheMiss = false; + return ret; + } + + public void setNextCCReadThrowIOException() { + this.nextIOException = true; + } + + public void setNextCCReadThrowEOFException() { + this.nextEOFException = true; + } + + public void setNextCCReadReturnCacheMiss() { + this.nextCacheMiss = true; + } + + public static final int CCREAD_RETCODE_CACHEMISS = -100; +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/mock/MockObsClient.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/mock/MockObsClient.java new file mode 100644 index 0000000..92aa69c --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/mock/MockObsClient.java @@ -0,0 +1,198 @@ +package org.apache.hadoop.fs.obs.mock; + +import com.obs.services.IObsCredentialsProvider; +import com.obs.services.ObsClient; +import com.obs.services.ObsConfiguration; +import com.obs.services.exception.ObsException; +import com.obs.services.model.ListObjectsRequest; +import com.obs.services.model.ObjectListing; +import com.obs.services.model.ObsObject; +import com.obs.services.model.fs.ContentSummaryFsRequest; +import com.obs.services.model.fs.ContentSummaryFsResult; +import com.obs.services.model.fs.DirContentSummary; +import com.obs.services.model.fs.DirSummary; +import com.obs.services.model.fs.ListContentSummaryFsRequest; +import com.obs.services.model.fs.ListContentSummaryFsResult; + +import org.apache.hadoop.fs.obs.OBSCommonUtils; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +import java.util.ArrayList; +import java.util.List; +import java.util.function.Function; + +public class MockObsClient extends ObsClient { + + private static final Logger LOG = LoggerFactory.getLogger(MockObsClient.class); + + private Function listCSResultCallback; + + private boolean getCSUnsupported; + + private int getCSNum; + + private int responseCode; + + private String errorMsg; + + public MockObsClient(IObsCredentialsProvider credentialsProvider, ObsConfiguration conf) { + super(credentialsProvider, conf); + } + + public void setListCSResultCallback(Function callback) { + this.listCSResultCallback = callback; + } + + /** + * use list to simulate listContentSummaryFs. + * ignore the file in dir. + * for each dir: + * use {fakeSubDirNumPerDir} to fill #dir, this may not equal to actual dir + * use {fakeFileNumPerDir} to fill #file + * use {fakeFileSize} to fill #filesize + * @param request + * @return + * @throws ObsException + */ + @Override + public ListContentSummaryFsResult listContentSummaryFs(ListContentSummaryFsRequest request) throws ObsException { + ListContentSummaryFsResult res = new ListContentSummaryFsResult(); + List dirContentSummaryList = new ArrayList<>(); + for (ListContentSummaryFsRequest.DirLayer dir :request.getDirLayers()) { + DirContentSummary dirContentSummary = listSingleContentSummaryFs(request.getBucketName(), request.getMaxKeys(), dir); + dirContentSummaryList.add(dirContentSummary); + } + res.setDirContentSummaries(dirContentSummaryList); + if (this.listCSResultCallback != null) { + return this.listCSResultCallback.apply(res); + } + return res; + } + + private DirContentSummary listSingleContentSummaryFs(String bucket, int maxKeys, ListContentSummaryFsRequest.DirLayer dir) { + ListObjectsRequest listObjectsRequest = new ListObjectsRequest(); + listObjectsRequest.setBucketName(bucket); + listObjectsRequest.setPrefix(maybeAddTrailingSlash(dir.getKey())); + listObjectsRequest.setMarker(dir.getMarker()); + listObjectsRequest.setMaxKeys(maxKeys); + listObjectsRequest.setDelimiter("/"); + ObjectListing objectListing = listObjects(listObjectsRequest); + DirContentSummary ret = new DirContentSummary(); + List dirSummaries = new ArrayList<>(); + ret.setKey(dir.getKey()); + if (objectListing.isTruncated()) { + ret.setTruncated(true); + ret.setNextMarker(objectListing.getNextMarker()); + } + for (String dirname :objectListing.getCommonPrefixes()) { + ContentSummaryFsRequest req = new ContentSummaryFsRequest(); + req.setBucketName(bucket); + req.setDirName(dirname); + ContentSummaryFsResult dirResult = getContentSummaryFs(req); + dirSummaries.add(dirResult.getContentSummary()); + } + ret.setSubDir(dirSummaries); + return ret; + } + + public ContentSummaryFsResult getContentSummaryFsUnsuppored(ContentSummaryFsRequest request) throws ObsException { + ObsException e = new ObsException("mock exception: unsupported"); + e.setResponseCode(responseCode); + e.setErrorMessage(errorMsg); + getCSNum++; + LOG.warn("mock exception: {}", errorMsg); + throw e; + } + + /** + * use getFileStatus to simulate getContentSummaryFs. + * ignore the file in dir. + * use {fakeSubDirNumPerDir} to fill #dir, this may not equal to actual dir + * use {fakeFileNumPerDir} to fill #file + * use {fakeFileSize} to fill #filesize + * @param request + * @return + * @throws ObsException + */ + @Override + public ContentSummaryFsResult getContentSummaryFs(ContentSummaryFsRequest request) throws ObsException { + if (isGetCSUnsupported()) { + return getContentSummaryFsUnsuppored(request); + } + long dNum = 0; + long fNum = 0; + long fSize = 0; + String marker = null; + boolean truncated; + do { + ListObjectsRequest listObjectsRequest = new ListObjectsRequest(); + listObjectsRequest.setBucketName(request.getBucketName()); + listObjectsRequest.setPrefix(maybeAddTrailingSlash(request.getDirName())); + listObjectsRequest.setMaxKeys(1000); + listObjectsRequest.setMarker(marker); + listObjectsRequest.setDelimiter("/"); + ObjectListing objectListing = listObjects(listObjectsRequest); + truncated = objectListing.isTruncated(); + marker = objectListing.getNextMarker(); + dNum += objectListing.getCommonPrefixes().size(); + for (ObsObject obj : objectListing.getObjects()) { + if (obj.getObjectKey().equals(maybeAddTrailingSlash(request.getDirName()))) { + continue; + } + fSize += obj.getMetadata().getContentLength(); + fNum ++; + } + } while(truncated); + + ContentSummaryFsResult res = new ContentSummaryFsResult(); + DirSummary dirSummary = new DirSummary(); + dirSummary.setDirCount(dNum); + dirSummary.setFileCount(fNum); + dirSummary.setFileSize(fSize); + dirSummary.setName(request.getDirName()); + res.setContentSummary(dirSummary); + return res; + } + + private String maybeAddTrailingSlash(final String key) { + if (OBSCommonUtils.isStringNotEmpty(key) && !key.endsWith("/")) { + return key + '/'; + } else { + return key; + } + } + + public int getGetCSNum() { + return getCSNum; + } + + public void setGetCSNum(int getCSNum) { + this.getCSNum = getCSNum; + } + + public boolean isGetCSUnsupported() { + return getCSUnsupported; + } + + public void setGetCSUnsupported(boolean getCSUnsupported) { + this.getCSUnsupported = getCSUnsupported; + } + + public int getResponseCode() { + return responseCode; + } + + public void setResponseCode(int responseCode) { + this.responseCode = responseCode; + } + + public String getErrorMsg() { + return errorMsg; + } + + public void setErrorMsg(String errorMsg) { + this.errorMsg = errorMsg; + } + +} diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/security/AuthorizeProviderTest.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/security/AuthorizeProviderTest.java new file mode 100644 index 0000000..6fd6d05 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/security/AuthorizeProviderTest.java @@ -0,0 +1,245 @@ +/* + * Copyright (c) Huawei Technologies Co., Ltd. 2021-2021. All rights reserved. + */ + +package org.apache.hadoop.fs.obs.security; + +import static org.apache.hadoop.fs.obs.security.MockAuthorizeProvider.*; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.obs.OBSFileSystem; +import org.apache.hadoop.fs.obs.OBSTestUtils; +import org.apache.hadoop.fs.obs.contract.OBSContract; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +/** + * description + * + * @since 2021-10-13 + */ +public class AuthorizeProviderTest { + private static OBSFileSystem fs; + private static FileSystem fs_with_auth; + + private static Path TEST_BASE_FOLDER_PATH; + + private static Path TEST_READ_ONLY_FILE_PATH_0; + private static Path TEST_READ_ONLY_FOLDER_PATH; + + private static Path TEST_WRITE_ONLY_FILE_PATH_0; + private static Path TEST_WRITE_ONLY_FILE_PATH_1; + private static Path TEST_WRITE_ONLY_FOLDER_PATH; + + private static Path TEST_READ_WRITE_FILE_PATH; + private static Path TEST_READ_WRITE_FOLDER_PATH; + + @Before + public void setUp() throws Exception { + Configuration conf = OBSContract.getConfiguration(null); + fs = OBSTestUtils.createTestFileSystem(conf); + + conf.set("fs.obs.authorize.provider","org.apache.hadoop.fs.obs.security.MockAuthorizeProvider"); + fs_with_auth = OBSTestUtils.createTestFileSystem(conf); + + TEST_BASE_FOLDER_PATH = new Path(fs_with_auth.getUri() + "/" + TEST_BASE_FOLDER); + TEST_READ_ONLY_FILE_PATH_0 = new Path(fs_with_auth.getUri() + "/" + TEST_READ_ONLY_FILE_0); + TEST_READ_ONLY_FOLDER_PATH = new Path(fs_with_auth.getUri() + "/" + TEST_READ_ONLY_FOLDER); + + TEST_WRITE_ONLY_FILE_PATH_0 = new Path(fs_with_auth.getUri() + "/" + TEST_WRITE_ONLY_FILE_0); + TEST_WRITE_ONLY_FILE_PATH_1 = new Path(fs_with_auth.getUri() + "/" + TEST_WRITE_ONLY_FILE_1); + TEST_WRITE_ONLY_FOLDER_PATH = new Path(fs_with_auth.getUri() + "/" + TEST_WRITE_ONLY_FOLDER); + + TEST_READ_WRITE_FILE_PATH = new Path(fs_with_auth.getUri() + "/" + TEST_READ_WRITE_FILE); + TEST_READ_WRITE_FOLDER_PATH = new Path(fs_with_auth.getUri() + "/" + TEST_READ_WRITE_FOLDER); + } + + @After + public void tearDown() throws Exception { + fs.delete(TEST_BASE_FOLDER_PATH,true); + fs.close(); + fs_with_auth.close(); + } + + @Test + public void testCreateFileAuthorized() throws Exception { + fs_with_auth.create(TEST_WRITE_ONLY_FILE_PATH_0); + } + + @Test + public void testCreateFileUnauthorized() { + try { + fs_with_auth.create(TEST_READ_ONLY_FILE_PATH_0); + } catch (IOException e) { + Assert.assertTrue("testCreateFileUnauthorized",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testAppendFileAuthorized() throws Exception { + if (!fs.isFsBucket()) { + return; + } + fs.create(TEST_WRITE_ONLY_FILE_PATH_0).close(); + fs_with_auth.append(TEST_WRITE_ONLY_FILE_PATH_0).close(); + } + + @Test + public void testAppendFileUnauthorized() { + if (!fs.isFsBucket()) { + return; + } + try { + fs.create(TEST_READ_ONLY_FILE_PATH_0).close(); + fs_with_auth.append(TEST_READ_ONLY_FILE_PATH_0).close(); + } catch (IOException e) { + Assert.assertTrue("testAppendFileUnauthorized",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testTruncateFileAuthorized() throws Exception { + if (!fs.isFsBucket()) { + return; + } + fs.create(TEST_WRITE_ONLY_FILE_PATH_0).close(); + fs_with_auth.truncate(TEST_WRITE_ONLY_FILE_PATH_0,0); + } + + @Test + public void testTruncateFileUnauthorized() { + if (!fs.isFsBucket()) { + return; + } + try { + fs.create(TEST_READ_ONLY_FILE_PATH_0).close(); + fs_with_auth.truncate(TEST_READ_ONLY_FILE_PATH_0,0); + } catch (IOException e) { + Assert.assertTrue("testTruncateFileUnauthorized",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testMkdirAuthorized() throws Exception { + fs_with_auth.mkdirs(TEST_WRITE_ONLY_FOLDER_PATH); + } + + @Test + public void testMkdirUnauthorized() { + try { + fs_with_auth.mkdirs(TEST_READ_ONLY_FOLDER_PATH); + } catch (IOException e) { + Assert.assertTrue("testMkdirUnauthorized",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testOpenAuthorized() throws Exception { + fs.create(TEST_READ_ONLY_FILE_PATH_0).close(); + fs_with_auth.open(TEST_READ_ONLY_FILE_PATH_0).close(); + } + + @Test + public void testOpenUnauthorized() { + try { + fs.create(TEST_WRITE_ONLY_FILE_PATH_0).close(); + fs_with_auth.open(TEST_WRITE_ONLY_FILE_PATH_0).close(); + } catch (IOException e) { + Assert.assertTrue("testMkdirUnauthorized",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testDeleteAuthorized() throws Exception { + fs.create(TEST_WRITE_ONLY_FILE_PATH_0).close(); + fs_with_auth.delete(TEST_WRITE_ONLY_FILE_PATH_0,false); + + fs.mkdirs(TEST_WRITE_ONLY_FOLDER_PATH); + fs_with_auth.delete(TEST_WRITE_ONLY_FOLDER_PATH,true); + } + + @Test + public void testDeleteUnauthorized() { + try { + fs.create(TEST_READ_ONLY_FILE_PATH_0).close(); + fs_with_auth.delete(TEST_READ_ONLY_FILE_PATH_0,false); + } catch (IOException e) { + Assert.assertTrue("testDeleteUnauthorized",e instanceof OBSAuthorizationException); + } + + try { + fs.mkdirs(TEST_READ_ONLY_FOLDER_PATH); + fs_with_auth.delete(TEST_READ_ONLY_FOLDER_PATH,true); + } catch (IOException e) { + Assert.assertTrue("testDeleteUnauthorized",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testListRoot() throws Exception { + try { + Path root = new Path(fs_with_auth.getUri() + "/"); + fs_with_auth.listStatus(root); + } catch (IOException e) { + Assert.assertTrue("testListRoot",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testListAuthorized() throws Exception { + fs.create(TEST_READ_ONLY_FILE_PATH_0).close(); + fs_with_auth.listStatus(TEST_READ_ONLY_FILE_PATH_0); + + fs.mkdirs(TEST_READ_ONLY_FOLDER_PATH); + fs_with_auth.listStatus(TEST_READ_ONLY_FOLDER_PATH); + } + + @Test + public void testListUnauthorized() { + try { + fs.create(TEST_WRITE_ONLY_FILE_PATH_0).close(); + fs_with_auth.listStatus(TEST_WRITE_ONLY_FILE_PATH_0); + } catch (IOException e) { + Assert.assertTrue("testListUnauthorized",e instanceof OBSAuthorizationException); + } + + try { + fs.mkdirs(TEST_WRITE_ONLY_FOLDER_PATH); + fs_with_auth.listStatus(TEST_WRITE_ONLY_FOLDER_PATH); + } catch (IOException e) { + Assert.assertTrue("testListUnauthorized",e instanceof OBSAuthorizationException); + } + } + + @Test + public void testRenameAuthorized() throws Exception { + fs.create(TEST_WRITE_ONLY_FILE_PATH_0).close(); + fs_with_auth.rename(TEST_WRITE_ONLY_FILE_PATH_0,TEST_WRITE_ONLY_FILE_PATH_1); + + fs.mkdirs(TEST_READ_ONLY_FOLDER_PATH); + fs_with_auth.rename(TEST_WRITE_ONLY_FOLDER_PATH,TEST_READ_WRITE_FOLDER_PATH); + } + + @Test + public void testRenameUnauthorized() { + try { + fs.create(TEST_READ_ONLY_FILE_PATH_0).close(); + fs_with_auth.rename(TEST_READ_ONLY_FILE_PATH_0,TEST_WRITE_ONLY_FILE_PATH_1); + } catch (IOException e) { + Assert.assertTrue("testListUnauthorized",e instanceof OBSAuthorizationException); + } + + try { + fs.mkdirs(TEST_READ_ONLY_FOLDER_PATH); + fs_with_auth.rename(TEST_READ_ONLY_FOLDER_PATH,TEST_WRITE_ONLY_FOLDER_PATH); + } catch (IOException e) { + Assert.assertTrue("testRenameUnauthorized",e instanceof OBSAuthorizationException); + } + } +} \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/security/MockAuthorizeProvider.java b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/security/MockAuthorizeProvider.java new file mode 100644 index 0000000..d902b80 --- /dev/null +++ b/hadoop-huaweicloud/src/test/java/org/apache/hadoop/fs/obs/security/MockAuthorizeProvider.java @@ -0,0 +1,76 @@ +/* + * Copyright (c) Huawei Technologies Co., Ltd. 2021-2021. All rights reserved. + */ + +package org.apache.hadoop.fs.obs.security; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.security.token.Token; + +import java.io.IOException; +import java.util.HashSet; +import java.util.Set; +import java.util.stream.Collectors; +import java.util.stream.Stream; + +/** + * description + * + * @since 2021-10-13 + */ +public class MockAuthorizeProvider implements AuthorizeProvider,DelegationTokenCapability { + public static final String TEST_BASE_FOLDER = "rangertest/"; + public static final String TEST_READ_ONLY_FILE_0 = TEST_BASE_FOLDER + "readOnlyFile0"; + public static final String TEST_READ_ONLY_FILE_1 = TEST_BASE_FOLDER + "readOnlyFile1"; + public static final String TEST_READ_ONLY_FOLDER = TEST_BASE_FOLDER + "readOnlyFolder"; + + public static final String TEST_WRITE_ONLY_FILE_0 = TEST_BASE_FOLDER + "writeOnlyFile0"; + public static final String TEST_WRITE_ONLY_FILE_1 = TEST_BASE_FOLDER + "writeOnlyFile1"; + public static final String TEST_WRITE_ONLY_FOLDER = TEST_BASE_FOLDER + "writeOnlyFolder"; + + public static final String TEST_READ_WRITE_FILE = TEST_BASE_FOLDER + "readWriteFile"; + public static final String TEST_READ_WRITE_FOLDER = TEST_BASE_FOLDER + "readWriteFolder"; + + private Set readOnlyPaths = new HashSet(); + private Set writeOnlyPaths = new HashSet(); + private Set readWritePaths = new HashSet(); + + @Override + public void init(Configuration conf) { + readOnlyPaths.add(TEST_READ_ONLY_FILE_0); + readOnlyPaths.add(TEST_READ_ONLY_FILE_1); + readOnlyPaths.add(TEST_READ_ONLY_FOLDER); + + writeOnlyPaths.add(TEST_WRITE_ONLY_FILE_0); + writeOnlyPaths.add(TEST_WRITE_ONLY_FILE_1); + writeOnlyPaths.add(TEST_WRITE_ONLY_FOLDER); + + readWritePaths.add(TEST_READ_WRITE_FILE); + readWritePaths.add(TEST_READ_WRITE_FOLDER); + } + + @Override + public boolean isAuthorized(String bucket, String key, AccessType action) { + Set reads = Stream.concat(readOnlyPaths.stream(), readWritePaths.stream()) + .collect(Collectors.toSet()); + Set writes = Stream.concat(writeOnlyPaths.stream(), readWritePaths.stream()).collect(Collectors.toSet()); + if (action.equals(AccessType.READ) && reads.contains(key)) { + return true; + } else if (action.equals(AccessType.WRITE) && writes.contains(key)) { + return true; + } else { + return false; + } + } + + @Override + public String getCanonicalServiceName() { + return null; + } + + @Override + public Token getDelegationToken(String renewer) throws IOException { + return null; + } +} diff --git a/hadoop-huaweicloud/src/test/resources/assembly.xml b/hadoop-huaweicloud/src/test/resources/assembly.xml new file mode 100644 index 0000000..b2679d9 --- /dev/null +++ b/hadoop-huaweicloud/src/test/resources/assembly.xml @@ -0,0 +1,38 @@ + + assembly + + jar + + false + + + / + true + true + test + + + + + + ${project.build.directory}/test-classes + / + + **/*.* + + true + + + + ${project.build.directory}/classes + / + + **/*.class + + true + + + \ No newline at end of file diff --git a/hadoop-huaweicloud/src/test/resources/contract/obs.xml b/hadoop-huaweicloud/src/test/resources/contract/obs.xml new file mode 100644 index 0000000..6a3c5c0 --- /dev/null +++ b/hadoop-huaweicloud/src/test/resources/contract/obs.xml @@ -0,0 +1,139 @@ + + + + + fs.contract.test.root-tests-enabled + true + + + + fs.contract.test.supports-concat + true + + + + fs.contract.rename-returns-false-if-source-missing + true + + + + fs.contract.test.random-seek-count + 10 + + + + fs.contract.is-case-sensitive + true + + + + fs.contract.rename-returns-true-if-dest-exists + false + + + + fs.contract.rename-returns-true-if-source-missing + false + + + + fs.contract.rename-creates-dest-dirs + false + + + + fs.contract.rename-remove-dest-if-empty-dir + false + + + + fs.contract.supports-settimes + true + + + + fs.contract.supports-append + true + + + + fs.contract.supports-atomic-directory-delete + true + + + + fs.contract.supports-atomic-rename + true + + + + fs.contract.supports-block-locality + true + + + + fs.contract.supports-concat + true + + + + fs.contract.supports-seek + true + + + + fs.contract.supports-seek-on-closed-file + true + + + + fs.contract.rejects-seek-past-eof + true + + + + fs.contract.supports-available-on-closed-file + true + + + + fs.contract.supports-strict-exceptions + false + + + + fs.contract.supports-unix-permissions + true + + + + fs.contract.rename-overwrites-dest + false + + + + fs.contract.supports-append + true + + + + fs.contract.supports-getfilestatus + true + + diff --git a/hadoop-huaweicloud/src/test/resources/core-site.xml b/hadoop-huaweicloud/src/test/resources/core-site.xml new file mode 100644 index 0000000..3dd4e02 --- /dev/null +++ b/hadoop-huaweicloud/src/test/resources/core-site.xml @@ -0,0 +1,154 @@ + + + + + + + + + + hadoop.tmp.dir + target/build/test + A base for other temporary directories. + true + + + + hadoop.security.authentication + simple + + + + fs.obs.impl + org.apache.hadoop.fs.obs.OBSFileSystem + The implementation class of the obs Filesystem + + + fs.obs.connection.establish.timeout + 60000 + + + fs.obs.connection.timeout + 60000 + + + fs.obs.idle.connection.time + 30000 + + + fs.obs.max.idle.connections + 10 + + + fs.obs.connection.maximum + 1000 + + + fs.obs.attempts.maximum + 5 + + + fs.obs.upload.stream.retry.buffer.size + 524288 + + + fs.obs.read.buffer.size + 8192 + + + fs.obs.write.buffer.size + 8192 + + + fs.obs.socket.recv.buffer + -1 + + + fs.obs.socket.send.buffer + -1 + + + fs.obs.keep.alive + true + + + fs.obs.validate.certificate + false + + + fs.obs.verify.response.content.type + true + + + fs.obs.strict.hostname.verification + false + + + fs.obs.cname + false + + + + fs.obs.test.local.path + /uplod_file + + + + fs.obs.fast.upload + true + + + fs.obs.multipart.size + 10485760 + + + fs.obs.experimental.input.fadvise + random + + + + fs.hdfs.impl + org.apache.hadoop.fs.obs.OBSHDFSFileSystem + + + + fs.obs.memartscc.inputstream.statistics.report.interval + 30 + + + + fs.obs.memartscc.inputstream.statistics.report.enable + false + + + + + + + + + + diff --git a/hadoop-huaweicloud/src/test/resources/log4j.properties b/hadoop-huaweicloud/src/test/resources/log4j.properties new file mode 100644 index 0000000..703113d --- /dev/null +++ b/hadoop-huaweicloud/src/test/resources/log4j.properties @@ -0,0 +1,23 @@ +# Licensed under the Apache License, Version 2.0 (the "License"); +# you may not use this file except in compliance with the License. +# You may obtain a copy of the License at +# +# http://www.apache.org/licenses/LICENSE-2.0 +# +# Unless required by applicable law or agreed to in writing, software +# distributed under the License is distributed on an "AS IS" BASIS, +# WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. +# See the License for the specific language governing permissions and +# limitations under the License. +# log4j configuration used during build and unit tests + +log4j.rootLogger=error,stdout +log4j.threshold=ALL +log4j.appender.stdout=org.apache.log4j.ConsoleAppender +log4j.appender.stdout.layout=org.apache.log4j.PatternLayout +log4j.appender.stdout.layout.ConversionPattern=%d{ISO8601} [%t] %-5p %c{2} (%F:%M(%L)) - %m%n + +log4j.logger.org.apache.hadoop.util.NativeCodeLoader=ERROR + +# for debugging low level obs operations, uncomment this line +log4j.logger.org.apache.hadoop.fs.obs=INFO diff --git a/release/hadoop-huaweicloud-2.6.0-hw-53.8.jar b/release/hadoop-huaweicloud-2.6.0-hw-53.8.jar new file mode 100644 index 0000000..759a0c5 Binary files /dev/null and b/release/hadoop-huaweicloud-2.6.0-hw-53.8.jar differ diff --git a/release/hadoop-huaweicloud-2.7.3-hw-53.8.jar b/release/hadoop-huaweicloud-2.7.3-hw-53.8.jar new file mode 100644 index 0000000..117e986 Binary files /dev/null and b/release/hadoop-huaweicloud-2.7.3-hw-53.8.jar differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-35.jar b/release/hadoop-huaweicloud-2.8.3-hw-35.jar deleted file mode 100644 index 50dd1d4..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-35.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-36.jar b/release/hadoop-huaweicloud-2.8.3-hw-36.jar deleted file mode 100644 index f51e606..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-36.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-37.jar b/release/hadoop-huaweicloud-2.8.3-hw-37.jar deleted file mode 100644 index c890519..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-37.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-39.jar b/release/hadoop-huaweicloud-2.8.3-hw-39.jar deleted file mode 100644 index 2540e93..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-39.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-40.1.jar b/release/hadoop-huaweicloud-2.8.3-hw-40.1.jar deleted file mode 100644 index 6fcc3b7..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-40.1.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-40.jar b/release/hadoop-huaweicloud-2.8.3-hw-40.jar deleted file mode 100644 index 8d67118..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-40.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-41.jar b/release/hadoop-huaweicloud-2.8.3-hw-41.jar deleted file mode 100644 index c5af0ee..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-41.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-42.jar b/release/hadoop-huaweicloud-2.8.3-hw-42.jar deleted file mode 100644 index 1d443e1..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-42.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-43.jar b/release/hadoop-huaweicloud-2.8.3-hw-43.jar deleted file mode 100644 index 1564f55..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-43.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-2.8.3-hw-45.jar b/release/hadoop-huaweicloud-2.8.3-hw-45.jar deleted file mode 100644 index e21c211..0000000 Binary files a/release/hadoop-huaweicloud-2.8.3-hw-45.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-35.jar b/release/hadoop-huaweicloud-3.1.1-hw-35.jar deleted file mode 100644 index 07166ae..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-35.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-36.jar b/release/hadoop-huaweicloud-3.1.1-hw-36.jar deleted file mode 100644 index e956bad..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-36.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-37.jar b/release/hadoop-huaweicloud-3.1.1-hw-37.jar deleted file mode 100644 index 33569fb..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-37.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-39.jar b/release/hadoop-huaweicloud-3.1.1-hw-39.jar deleted file mode 100644 index 78141ee..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-39.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-40.1.jar b/release/hadoop-huaweicloud-3.1.1-hw-40.1.jar deleted file mode 100644 index 0faca6c..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-40.1.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-40.jar b/release/hadoop-huaweicloud-3.1.1-hw-40.jar deleted file mode 100644 index 18c0d08..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-40.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-41.jar b/release/hadoop-huaweicloud-3.1.1-hw-41.jar deleted file mode 100644 index 0c0cdb1..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-41.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-42.jar b/release/hadoop-huaweicloud-3.1.1-hw-42.jar deleted file mode 100644 index 90c78e0..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-42.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-43.jar b/release/hadoop-huaweicloud-3.1.1-hw-43.jar deleted file mode 100644 index 293e08c..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-43.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-45.jar b/release/hadoop-huaweicloud-3.1.1-hw-45.jar deleted file mode 100644 index 38a781b..0000000 Binary files a/release/hadoop-huaweicloud-3.1.1-hw-45.jar and /dev/null differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-53.7.jar b/release/hadoop-huaweicloud-3.1.1-hw-53.7.jar new file mode 100644 index 0000000..a7e47cf Binary files /dev/null and b/release/hadoop-huaweicloud-3.1.1-hw-53.7.jar differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-53.8.jar b/release/hadoop-huaweicloud-3.1.1-hw-53.8.jar new file mode 100644 index 0000000..44fab56 Binary files /dev/null and b/release/hadoop-huaweicloud-3.1.1-hw-53.8.jar differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-54.0.jar b/release/hadoop-huaweicloud-3.1.1-hw-54.0.jar new file mode 100644 index 0000000..9b1b077 Binary files /dev/null and b/release/hadoop-huaweicloud-3.1.1-hw-54.0.jar differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-54.1.jar b/release/hadoop-huaweicloud-3.1.1-hw-54.1.jar new file mode 100644 index 0000000..6c63ce0 Binary files /dev/null and b/release/hadoop-huaweicloud-3.1.1-hw-54.1.jar differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-54.3.jar b/release/hadoop-huaweicloud-3.1.1-hw-54.3.jar new file mode 100644 index 0000000..a543572 Binary files /dev/null and b/release/hadoop-huaweicloud-3.1.1-hw-54.3.jar differ diff --git a/release/hadoop-huaweicloud-3.1.1-hw-54.5.jar b/release/hadoop-huaweicloud-3.1.1-hw-54.5.jar new file mode 100644 index 0000000..3985628 Binary files /dev/null and b/release/hadoop-huaweicloud-3.1.1-hw-54.5.jar differ