[ 
https://issues.apache.org/jira/browse/HDDS-460?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=16620951#comment-16620951
 ] 

Nanda kumar edited comment on HDDS-460 at 9/19/18 5:52 PM:
-----------------------------------------------------------

Thanks [~elek] for working on the patch. Overall it looks pretty good.

In ReplicateContainerCommandHandler, we should not mark the command as success 
as soon as we queue ReplicationTask. We have to update it only when the task 
status is marked as DONE or FAILED.
ReplicationSupervisor pool size can be made configurable, this can be done in a 
follow-up jira.


was (Author: nandakumar131):
Thanks [~elek] for working on the patch. Overall it looks pretty good.

In ReplicateContainerCommandHandler, we should not mark the command as success 
[updateCommandStatus(context, command, true, LOG)] as soon as we queue 
ReplicationTask. We have to update it only when the task status is marked as 
DONE or FAILED.
ReplicationSupervisor pool size can be made configurable, this can be done in a 
follow-up jira.

> Replication manager failed to import container data
> ---------------------------------------------------
>
>                 Key: HDDS-460
>                 URL: https://issues.apache.org/jira/browse/HDDS-460
>             Project: Hadoop Distributed Data Store
>          Issue Type: Bug
>          Components: SCM
>    Affects Versions: 0.2.1
>            Reporter: Nilotpal Nandi
>            Assignee: Elek, Marton
>            Priority: Major
>         Attachments: HDDS-460-ozone-0.2.001.patch, 
> all-node-ozone-logs-1536920345.tar.gz
>
>
> replication manager is not able to import downloaded container data on 
> datanode failure. 
>  
> snippet of ozone.log
> -----------------------------
> {noformat}
> 2018-09-14 09:34:05,249 [grpc-default-executor-139] INFO 
> (GrpcReplicationClient.java:161) - Container is downloaded to 
> /tmp/container-copy/container-14.tar.gz
>  2018-09-14 09:34:05,389 [grpc-default-executor-131] ERROR 
> (ReplicateContainerCommandHandler.java:164) - Can't import the downloaded 
> container data id=8
>  java.io.EOFException
>  at 
> org.apache.commons.compress.compressors.gzip.GzipCompressorInputStream.read(GzipCompressorInputStream.java:241)
>  at 
> org.apache.commons.compress.archivers.tar.TarBuffer.readBlock(TarBuffer.java:224)
>  at 
> org.apache.commons.compress.archivers.tar.TarBuffer.readRecord(TarBuffer.java:195)
>  at 
> org.apache.commons.compress.archivers.tar.TarArchiveInputStream.read(TarArchiveInputStream.java:486)
>  at 
> org.apache.commons.compress.archivers.tar.TarArchiveInputStream.skip(TarArchiveInputStream.java:182)
>  at 
> org.apache.commons.compress.archivers.tar.TarArchiveInputStream.getNextTarEntry(TarArchiveInputStream.java:220)
>  at 
> org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker.unpackContainerDescriptor(TarContainerPacker.java:200)
>  at 
> org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ReplicateContainerCommandHandler.importContainer(ReplicateContainerCommandHandler.java:144)
>  at 
> org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ReplicateContainerCommandHandler.lambda$handle$0(ReplicateContainerCommandHandler.java:121)
>  at 
> java.util.concurrent.CompletableFuture.uniAccept(CompletableFuture.java:656)
>  at 
> java.util.concurrent.CompletableFuture$UniAccept.tryFire(CompletableFuture.java:632)
>  at 
> java.util.concurrent.CompletableFuture.postComplete(CompletableFuture.java:474)
>  at 
> java.util.concurrent.CompletableFuture.complete(CompletableFuture.java:1962)
>  at 
> org.apache.hadoop.ozone.container.replication.GrpcReplicationClient$StreamDownloader.onCompleted(GrpcReplicationClient.java:160)
>  at 
> org.apache.ratis.shaded.io.grpc.stub.ClientCalls$StreamObserverToCallListenerAdapter.onClose(ClientCalls.java:418)
>  at 
> org.apache.ratis.shaded.io.grpc.PartialForwardingClientCallListener.onClose(PartialForwardingClientCallListener.java:39)
>  at 
> org.apache.ratis.shaded.io.grpc.ForwardingClientCallListener.onClose(ForwardingClientCallListener.java:23)
>  at 
> org.apache.ratis.shaded.io.grpc.ForwardingClientCallListener$SimpleForwardingClientCallListener.onClose(ForwardingClientCallListener.java:40)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.CensusStatsModule$StatsClientInterceptor$1$1.onClose(CensusStatsModule.java:684)
>  at 
> org.apache.ratis.shaded.io.grpc.PartialForwardingClientCallListener.onClose(PartialForwardingClientCallListener.java:39)
>  at 
> org.apache.ratis.shaded.io.grpc.ForwardingClientCallListener.onClose(ForwardingClientCallListener.java:23)
>  at 
> org.apache.ratis.shaded.io.grpc.ForwardingClientCallListener$SimpleForwardingClientCallListener.onClose(ForwardingClientCallListener.java:40)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.CensusTracingModule$TracingClientInterceptor$1$1.onClose(CensusTracingModule.java:403)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.ClientCallImpl.closeObserver(ClientCallImpl.java:459)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.ClientCallImpl.access$300(ClientCallImpl.java:63)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl.close(ClientCallImpl.java:546)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl.access$600(ClientCallImpl.java:467)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.ClientCallImpl$ClientStreamListenerImpl$1StreamClosed.runInContext(ClientCallImpl.java:584)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.ContextRunnable.run(ContextRunnable.java:37)
>  at 
> org.apache.ratis.shaded.io.grpc.internal.SerializingExecutor.run(SerializingExecutor.java:123)
>  at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1142)
>  at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:617)
>  at java.lang.Thread.run(Thread.java:745){noformat}



--
This message was sent by Atlassian JIRA
(v7.6.3#76005)

---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to