You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-issues@hadoop.apache.org by "wujinhu (JIRA)" <ji...@apache.org> on 2018/10/23 02:18:00 UTC
[jira] [Updated] (HADOOP-15607) AliyunOSS: fix duplicated
partNumber issue in AliyunOSSBlockOutputStream
[ https://issues.apache.org/jira/browse/HADOOP-15607?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel ]
wujinhu updated HADOOP-15607:
-----------------------------
Resolution: Fixed
Status: Resolved (was: Patch Available)
> AliyunOSS: fix duplicated partNumber issue in AliyunOSSBlockOutputStream
> -------------------------------------------------------------------------
>
> Key: HADOOP-15607
> URL: https://issues.apache.org/jira/browse/HADOOP-15607
> Project: Hadoop Common
> Issue Type: Bug
> Affects Versions: 3.1.0, 2.10.0, 2.9.1, 3.2.0, 3.0.3
> Reporter: wujinhu
> Assignee: wujinhu
> Priority: Critical
> Fix For: 2.10.0, 3.2.0, 2.9.2, 3.0.4, 3.1.2
>
> Attachments: HADOOP-15607-branch-2.001.patch, HADOOP-15607.001.patch, HADOOP-15607.002.patch, HADOOP-15607.003.patch, HADOOP-15607.004.patch
>
>
> When I generated data with hive-tpcds tool, I got exception below:
> 2018-07-16 14:50:43,680 INFO mapreduce.Job: Task Id : attempt_1531723399698_0001_m_000052_0, Status : FAILED
> Error: com.aliyun.oss.OSSException: The list of parts was not in ascending order. Parts list must specified in order by part number.
> [ErrorCode]: InvalidPartOrder
> [RequestId]: 5B4C40425FCC208D79D1EAF5
> [HostId]: 100.103.0.137
> [ResponseError]:
> <?xml version="1.0" encoding="UTF-8"?>
> <Error>
> <Code>InvalidPartOrder</Code>
> <Message>The list of parts was not in ascending order. Parts list must specified in order by part number.</Message>
> <RequestId>5B4C40425FCC208D79D1EAF5</RequestId>
> <HostId>xx.xx.xx.xx</HostId>
> <ErrorDetail>current PartNumber 3, you given part number 3is not in ascending order</ErrorDetail>
> </Error>
> at com.aliyun.oss.common.utils.ExceptionFactory.createOSSException(ExceptionFactory.java:99)
> at com.aliyun.oss.internal.OSSErrorResponseHandler.handle(OSSErrorResponseHandler.java:69)
> at com.aliyun.oss.common.comm.ServiceClient.handleResponse(ServiceClient.java:248)
> at com.aliyun.oss.common.comm.ServiceClient.sendRequestImpl(ServiceClient.java:130)
> at com.aliyun.oss.common.comm.ServiceClient.sendRequest(ServiceClient.java:68)
> at com.aliyun.oss.internal.OSSOperation.send(OSSOperation.java:94)
> at com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:149)
> at com.aliyun.oss.internal.OSSOperation.doOperation(OSSOperation.java:113)
> at com.aliyun.oss.internal.OSSMultipartOperation.completeMultipartUpload(OSSMultipartOperation.java:185)
> at com.aliyun.oss.OSSClient.completeMultipartUpload(OSSClient.java:790)
> at org.apache.hadoop.fs.aliyun.oss.AliyunOSSFileSystemStore.completeMultipartUpload(AliyunOSSFileSystemStore.java:643)
> at org.apache.hadoop.fs.aliyun.oss.AliyunOSSBlockOutputStream.close(AliyunOSSBlockOutputStream.java:120)
> at org.apache.hadoop.fs.FSDataOutputStream$PositionCache.close(FSDataOutputStream.java:72)
> at org.apache.hadoop.fs.FSDataOutputStream.close(FSDataOutputStream.java:101)
> at org.apache.hadoop.mapreduce.lib.output.TextOutputFormat$LineRecordWriter.close(TextOutputFormat.java:106)
> at org.apache.hadoop.mapreduce.lib.output.MultipleOutputs.close(MultipleOutputs.java:574)
> at org.notmysock.tpcds.GenTable$DSDGen.cleanup(GenTable.java:169)
> at org.apache.hadoop.mapreduce.Mapper.run(Mapper.java:149)
> at org.apache.hadoop.mapred.MapTask.runNewMapper(MapTask.java:799)
> at org.apache.hadoop.mapred.MapTask.run(MapTask.java:347)
> at org.apache.hadoop.mapred.YarnChild$2.run(YarnChild.java:174)
> at java.security.AccessController.doPrivileged(Native Method)
> at javax.security.auth.Subject.doAs(Subject.java:422)
> at org.apache.hadoop.security.UserGroupInformation.doAs(UserGroupInformation.java:1686)
>
> I reviewed code below,
> {code:java}
> blockId {code}
> has thread synchronization problem
> {code:java}
> // code placeholder
> private void uploadCurrentPart() throws IOException {
> blockFiles.add(blockFile);
> blockStream.flush();
> blockStream.close();
> if (blockId == 0) {
> uploadId = store.getUploadId(key);
> }
> ListenableFuture<PartETag> partETagFuture =
> executorService.submit(() -> {
> PartETag partETag = store.uploadPart(blockFile, key, uploadId,
> blockId + 1);
> return partETag;
> });
> partETagsFutures.add(partETagFuture);
> blockFile = newBlockFile();
> blockId++;
> blockStream = new BufferedOutputStream(new FileOutputStream(blockFile));
> }
> {code}
--
This message was sent by Atlassian JIRA
(v7.6.3#76005)
---------------------------------------------------------------------
To unsubscribe, e-mail: common-issues-unsubscribe@hadoop.apache.org
For additional commands, e-mail: common-issues-help@hadoop.apache.org