[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320111=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320111
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:33
Start Date: 29/Sep/19 03:33
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329336173
 
 

 ##
 File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/OneReplicaPipelineSafeModeRule.java
 ##
 @@ -75,69 +66,59 @@ public OneReplicaPipelineSafeModeRule(String ruleName, 
EventQueue eventQueue,
 HDDS_SCM_SAFEMODE_ONE_NODE_REPORTED_PIPELINE_PCT  +
 " value should be >= 0.0 and <= 1.0");
 
+// Exclude CLOSED pipeline
 int totalPipelineCount =
 pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
-HddsProtos.ReplicationFactor.THREE).size();
+HddsProtos.ReplicationFactor.THREE, Pipeline.PipelineState.OPEN)
+.size() +
+pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
 
 Review comment:
   I guess the original idea of this OneReplicaPipelineSafeModeRule is to check 
THREE factor Pipeline only since ONE factor pipeline is not favored in 
production environment. 
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320111)
Time Spent: 9h 50m  (was: 9h 40m)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 9h 50m
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320110=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320110
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:27
Start Date: 29/Sep/19 03:27
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329336020
 
 

 ##
 File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/pipeline/SCMPipelineManager.java
 ##
 @@ -148,17 +166,20 @@ private void initializePipelineState() throws 
IOException {
   }
 
   @Override
-  public synchronized Pipeline createPipeline(
-  ReplicationType type, ReplicationFactor factor) throws IOException {
+  public synchronized Pipeline createPipeline(ReplicationType type,
+  ReplicationFactor factor) throws IOException {
 lock.writeLock().lock();
 try {
   Pipeline pipeline = pipelineFactory.create(type, factor);
   pipelineStore.put(pipeline.getId().getProtobuf().toByteArray(),
   pipeline.getProtobufMessage().toByteArray());
   stateManager.addPipeline(pipeline);
   nodeManager.addPipeline(pipeline);
-  metrics.incNumPipelineCreated();
-  metrics.createPerPipelineMetrics(pipeline);
+  metrics.incNumPipelineAllocated();
+  if (pipeline.isOpen()) {
+metrics.incNumPipelineCreated();
+metrics.createPerPipelineMetrics(pipeline);
+  }
 
 Review comment:
   This statement is for ONE factor RATIS pipeline which is in OPEN after 
created.  It doesn't go through openPipeline. 
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320110)
Time Spent: 9h 40m  (was: 9.5h)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 9h 40m
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320109=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320109
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:24
Start Date: 29/Sep/19 03:24
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329335964
 
 

 ##
 File path: 
hadoop-hdds/common/src/main/java/org/apache/hadoop/hdds/scm/pipeline/Pipeline.java
 ##
 @@ -65,6 +66,7 @@ private Pipeline(PipelineID id, ReplicationType type,
 this.factor = factor;
 this.state = state;
 this.nodeStatus = nodeStatus;
+this.creationTime = System.currentTimeMillis();
   }
 
 Review comment:
   OK
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320109)
Time Spent: 9.5h  (was: 9h 20m)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 9.5h
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320108=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320108
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:22
Start Date: 29/Sep/19 03:22
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329335901
 
 

 ##
 File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/SafeModeHandler.java
 ##
 @@ -129,7 +129,8 @@ private void cleanupPipelines() {
 List pipelineList = scmPipelineManager.getPipelines();
 pipelineList.forEach((pipeline) -> {
   try {
-if (pipeline.getPipelineState() == Pipeline.PipelineState.ALLOCATED) {
+if (pipeline.getPipelineState() == Pipeline.PipelineState.ALLOCATED &&
+pipeline.isAllocationTimeout()) {
   scmPipelineManager.finalizeAndDestroyPipeline(pipeline, false);
 
 Review comment:
   Right, there is a TODO.  Will complete the logic in HDDS-2177, "Add a 
srubber thread to detect creation failure pipelines in ALLOCATED state".  
   This patch is too big. I'd like to keep mandatory change in this patch and 
have others in new tasks. 
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320108)
Time Spent: 9h 20m  (was: 9h 10m)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 9h 20m
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320106=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320106
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:14
Start Date: 29/Sep/19 03:14
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329335686
 
 

 ##
 File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/safemode/HealthyPipelineSafeModeRule.java
 ##
 @@ -49,43 +39,52 @@
  * through in a cluster.
  */
 public class HealthyPipelineSafeModeRule
-extends SafeModeExitRule{
+extends SafeModeExitRule{
 
   public static final Logger LOG =
   LoggerFactory.getLogger(HealthyPipelineSafeModeRule.class);
-  private final PipelineManager pipelineManager;
   private final int healthyPipelineThresholdCount;
   private int currentHealthyPipelineCount = 0;
-  private final Set processedDatanodeDetails =
-  new HashSet<>();
 
   HealthyPipelineSafeModeRule(String ruleName, EventQueue eventQueue,
   PipelineManager pipelineManager,
   SCMSafeModeManager manager, Configuration configuration) {
 super(manager, ruleName, eventQueue);
-this.pipelineManager = pipelineManager;
 double healthyPipelinesPercent =
 configuration.getDouble(HddsConfigKeys.
 HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT,
 HddsConfigKeys.
 HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT_DEFAULT);
 
+int minHealthyPipelines = 0;
+
+boolean createPipelineInSafemode = configuration.getBoolean(
+HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION,
+HddsConfigKeys.HDDS_SCM_SAFEMODE_PIPELINE_CREATION_DEFAULT);
+
+if (createPipelineInSafemode) {
+  minHealthyPipelines =
+  configuration.getInt(HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_PIPELINE,
+  HddsConfigKeys.HDDS_SCM_SAFEMODE_MIN_PIPELINE_DEFAULT);
+}
+
 Preconditions.checkArgument(
 (healthyPipelinesPercent >= 0.0 && healthyPipelinesPercent <= 1.0),
 HddsConfigKeys.
 HDDS_SCM_SAFEMODE_HEALTHY_PIPELINE_THRESHOLD_PCT
 + " value should be >= 0.0 and <= 1.0");
 
-// As we want to wait for 3 node pipelines
-int pipelineCount =
+// As we want to wait for RATIS write pipelines, no matter ONE or THREE
+int pipelineCount = pipelineManager.getPipelines(
+HddsProtos.ReplicationType.RATIS, Pipeline.PipelineState.OPEN).size() +
 pipelineManager.getPipelines(HddsProtos.ReplicationType.RATIS,
-HddsProtos.ReplicationFactor.THREE).size();
+Pipeline.PipelineState.ALLOCATED).size();
 
 
 Review comment:
   Will check it.  Last time I remember an integration test failure with  
CLOSED pipelines involved. 
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320106)
Time Spent: 9h 10m  (was: 9h)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 9h 10m
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320104=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320104
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:11
Start Date: 29/Sep/19 03:11
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329335622
 
 

 ##
 File path: 
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/ClosePipelineCommandHandler.java
 ##
 @@ -0,0 +1,166 @@
+/**
+ * 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.ozone.container.common.statemachine.commandhandler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.
+StorageContainerDatanodeProtocolProtos.ClosePipelineCommandProto;
+import org.apache.hadoop.hdds.protocol.proto.
+StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.hdds.ratis.RatisHelper;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client
+.CertificateClient;
+import org.apache.hadoop.ozone.container.common.statemachine
+.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.ClosePipelineCommand;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.grpc.GrpcTlsConfig;
+import org.apache.ratis.protocol.RaftGroupId;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.retry.RetryPolicy;
+import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.concurrent.atomic.AtomicLong;
+
+/**
+ * Handler for close pipeline command received from SCM.
+ */
+public class ClosePipelineCommandHandler implements CommandHandler {
+
+  private static final Logger LOG =
+  LoggerFactory.getLogger(ClosePipelineCommandHandler.class);
+
+  private AtomicLong invocationCount = new AtomicLong(0);
+  private long totalTime;
+
+  /**
+   * Constructs a closePipelineCommand handler.
+   */
+  public ClosePipelineCommandHandler() {
+  }
+
+  /**
+   * Handles a given SCM command.
+   *
+   * @param command   - SCM Command
+   * @param ozoneContainer- Ozone Container.
+   * @param context   - Current Context.
+   * @param connectionManager - The SCMs that we are talking to.
+   */
+  @Override
+  public void handle(SCMCommand command, OzoneContainer ozoneContainer,
+  StateContext context, SCMConnectionManager connectionManager) {
+invocationCount.incrementAndGet();
+final long startTime = Time.monotonicNow();
+final DatanodeDetails datanode = context.getParent()
+.getDatanodeDetails();
+final ClosePipelineCommandProto closeCommand =
+((ClosePipelineCommand)command).getProto();
+final PipelineID pipelineID = PipelineID.getFromProtobuf(
+closeCommand.getPipelineID());
+
+try {
+  destroyPipeline(datanode, pipelineID, context);
+  LOG.info("Close Pipeline #{} command on datanode #{}.", pipelineID,
+  datanode.getUuidString());
+} catch (IOException e) {
+  LOG.error("Can't close pipeline #{}", pipelineID, e);
+} finally {
+  long endTime = Time.monotonicNow();
+  totalTime += endTime - startTime;
+}
+
+  }
+
+  /**
+   * Returns the command type that this command handler handles.
+   *
+   * 

[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320105=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320105
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:11
Start Date: 29/Sep/19 03:11
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329335624
 
 

 ##
 File path: 
hadoop-hdds/container-service/src/main/java/org/apache/hadoop/ozone/container/common/statemachine/commandhandler/CreatePipelineCommandHandler.java
 ##
 @@ -0,0 +1,228 @@
+/**
+ * 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.ozone.container.common.statemachine.commandhandler;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdds.protocol.DatanodeDetails;
+import org.apache.hadoop.hdds.protocol.proto.
+StorageContainerDatanodeProtocolProtos.CreatePipelineCommandProto;
+import org.apache.hadoop.hdds.protocol.proto.
+StorageContainerDatanodeProtocolProtos.SCMCommandProto;
+import org.apache.hadoop.hdds.protocol.proto.
+StorageContainerDatanodeProtocolProtos.CreatePipelineACKProto;
+import org.apache.hadoop.hdds.ratis.RatisHelper;
+import org.apache.hadoop.hdds.scm.ScmConfigKeys;
+import org.apache.hadoop.hdds.scm.client.HddsClientUtils;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.security.x509.SecurityConfig;
+import org.apache.hadoop.hdds.security.x509.certificate.client
+.CertificateClient;
+import org.apache.hadoop.io.MultipleIOException;
+import org.apache.hadoop.ozone.container.common.statemachine
+.SCMConnectionManager;
+import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
+import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
+import org.apache.hadoop.ozone.protocol.commands.CommandStatus;
+import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommand;
+import org.apache.hadoop.ozone.protocol.commands.CreatePipelineCommandStatus;
+import org.apache.hadoop.ozone.protocol.commands.SCMCommand;
+import org.apache.hadoop.util.Time;
+import org.apache.ratis.client.RaftClient;
+import org.apache.ratis.grpc.GrpcTlsConfig;
+import org.apache.ratis.protocol.NotLeaderException;
+import org.apache.ratis.protocol.RaftClientReply;
+import org.apache.ratis.protocol.RaftGroup;
+import org.apache.ratis.protocol.RaftGroupId;
+import org.apache.ratis.protocol.RaftPeer;
+import org.apache.ratis.retry.RetryPolicy;
+import org.apache.ratis.rpc.SupportedRpcType;
+import org.apache.ratis.util.TimeDuration;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Collection;
+import java.util.Collections;
+import java.util.List;
+import java.util.concurrent.RejectedExecutionException;
+import java.util.concurrent.atomic.AtomicLong;
+import java.util.function.Consumer;
+import java.util.stream.Collectors;
+
+/**
+ * Handler for create pipeline command received from SCM.
+ */
+public class CreatePipelineCommandHandler implements CommandHandler {
+
+  private static final Logger LOG =
+  LoggerFactory.getLogger(CreatePipelineCommandHandler.class);
+
+  private AtomicLong invocationCount = new AtomicLong(0);
+  private long totalTime;
+
+  /**
+   * Constructs a createPipelineCommand handler.
+   */
+  public CreatePipelineCommandHandler() {
+  }
+
+  /**
+   * Handles a given SCM command.
+   *
+   * @param command   - SCM Command
+   * @param ozoneContainer- Ozone Container.
+   * @param context   - Current Context.
+   * @param connectionManager - The SCMs that we are talking to.
+   */
+  @Override
+  public void handle(SCMCommand command, OzoneContainer ozoneContainer,
+  StateContext context, SCMConnectionManager connectionManager) {
+invocationCount.incrementAndGet();
+final long startTime = Time.monotonicNow();
+final DatanodeDetails dn = context.getParent()
+

[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320103=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320103
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:10
Start Date: 29/Sep/19 03:10
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329335596
 
 

 ##
 File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
 ##
 @@ -188,6 +208,15 @@ public AllocatedBlock allocateBlock(final long size, 
ReplicationType type,
   // TODO: #CLUTIL Remove creation logic when all replication types and
   // factors are handled by pipeline creator
   pipeline = pipelineManager.createPipeline(type, factor);
+  // wait until pipeline is ready
+  long current = System.currentTimeMillis();
+  while (!pipeline.isOpen() && System.currentTimeMillis() <
+  (current + pipelineCreateWaitTimeout)) {
+try {
+  Thread.sleep(1000);
+} catch (InterruptedException e) {
+}
+  }
 
 Review comment:
   This create pipeline in block allocation path is kind of debating.  A 
current comment sys "TODO: #CLUTIL Remove creation logic when all replication 
types and factors are handled by pipeline creator". 
   To the detail,  "ALLOCATED" state will be handled in task HDDS-2177, "Add a 
srubber thread to detect creation failure pipelines in ALLOCATED state".  
Currently the pipelineCreateWaitTimeout is calculated based on 
"hdds.command.status.report.interval" and "hdds.heartbeat.interval", under the 
condition that the connection between Datanode and SCM is in good state.  What 
if pipeline is created successfully, while the connection to SCM broken and 
restored after a while. Would we wait a little longer to decide whether 
pipeline creation success or failure.  So in HDDS-2177,  I plan to have a 
configurable property for the pipeline creation timeout.  Every ALLOCATED 
pipeline, which exceeds the creation timeout will be claimed failure and 
garbage collected. 
   Whether using CompleteFuture or while loop,  we all need a timeout.  This is 
on block allocation path,  how many latency can a synchronous API tolerate?  
Maybe the best way is not create pipeline in such case if we can make sure 
there are enough pipelines to use after  exited safe mode.
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320103)
Time Spent: 8h 40m  (was: 8.5h)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 8h 40m
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320102=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320102
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 03:10
Start Date: 29/Sep/19 03:10
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on pull request #1469: HDDS-2034. 
Async RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#discussion_r329335596
 
 

 ##
 File path: 
hadoop-hdds/server-scm/src/main/java/org/apache/hadoop/hdds/scm/block/BlockManagerImpl.java
 ##
 @@ -188,6 +208,15 @@ public AllocatedBlock allocateBlock(final long size, 
ReplicationType type,
   // TODO: #CLUTIL Remove creation logic when all replication types and
   // factors are handled by pipeline creator
   pipeline = pipelineManager.createPipeline(type, factor);
+  // wait until pipeline is ready
+  long current = System.currentTimeMillis();
+  while (!pipeline.isOpen() && System.currentTimeMillis() <
+  (current + pipelineCreateWaitTimeout)) {
+try {
+  Thread.sleep(1000);
+} catch (InterruptedException e) {
+}
+  }
 
 Review comment:
   This create pipeline in block allocation path is kind of debating.  A 
current comment sys "TODO: #CLUTIL Remove creation logic when all replication 
types and factors are handled by pipeline creator". 
   To the detail,  "ALLOCATED" state will be handled in task HDDS-2177, "Add a 
srubber thread to detect creation failure pipelines in ALLOCATED state".  
Currently the pipelineCreateWaitTimeout is calculated based on 
"hdds.command.status.report.interval" and "hdds.heartbeat.interval", under the 
condition that the connection between Datanode and SCM is in good state.  What 
if pipeline is created successfully, while the connection to SCM broken and 
restored after a while. Would we wait a little longer to decide whether 
pipeline creation success or failure.  So in HDDS-2177,  I plan to have a 
configurable property for the pipeline creation timeout.  Every ALLOCATED 
pipeline, which exceeds the creation timeout will be claimed failure and 
garbage collected. 
   Whether using CompleteFuture or while loop,  we all need a timeout.  This is 
on block allocation path,  how many latency can a synchronous API tolerate?  
Maybe the best way is not create pipeline in such case if we can make sure 
there are enough pipelines to use after  exiting safe mode.
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320102)
Time Spent: 8.5h  (was: 8h 20m)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 8.5h
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-1569) Add ability to SCM for creating multiple pipelines with same datanode

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-1569?focusedWorklogId=320101=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320101
 ]

ASF GitHub Bot logged work on HDDS-1569:


Author: ASF GitHub Bot
Created on: 29/Sep/19 02:58
Start Date: 29/Sep/19 02:58
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on issue #1431: HDDS-1569 
Support creating multiple pipelines with same datanode
URL: https://github.com/apache/hadoop/pull/1431#issuecomment-536243358
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |::|--:|:|:|
   | 0 | reexec | 79 | Docker mode activated. |
   ||| _ Prechecks _ |
   | +1 | dupname | 1 | No case conflicting files found. |
   | +1 | @author | 0 | The patch does not contain any @author tags. |
   | +1 | test4tests | 0 | The patch appears to include 13 new or modified test 
files. |
   ||| _ HDDS-1564 Compile Tests _ |
   | 0 | mvndep | 64 | Maven dependency ordering for branch |
   | -1 | mvninstall | 31 | hadoop-hdds in HDDS-1564 failed. |
   | -1 | mvninstall | 37 | hadoop-ozone in HDDS-1564 failed. |
   | -1 | compile | 18 | hadoop-hdds in HDDS-1564 failed. |
   | -1 | compile | 13 | hadoop-ozone in HDDS-1564 failed. |
   | +1 | checkstyle | 56 | HDDS-1564 passed |
   | +1 | mvnsite | 0 | HDDS-1564 passed |
   | +1 | shadedclient | 933 | branch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 18 | hadoop-hdds in HDDS-1564 failed. |
   | -1 | javadoc | 18 | hadoop-ozone in HDDS-1564 failed. |
   | 0 | spotbugs | 1017 | Used deprecated FindBugs config; considering 
switching to SpotBugs. |
   | -1 | findbugs | 27 | hadoop-hdds in HDDS-1564 failed. |
   | -1 | findbugs | 17 | hadoop-ozone in HDDS-1564 failed. |
   ||| _ Patch Compile Tests _ |
   | 0 | mvndep | 28 | Maven dependency ordering for patch |
   | -1 | mvninstall | 34 | hadoop-hdds in the patch failed. |
   | -1 | mvninstall | 34 | hadoop-ozone in the patch failed. |
   | -1 | compile | 21 | hadoop-hdds in the patch failed. |
   | -1 | compile | 15 | hadoop-ozone in the patch failed. |
   | -1 | javac | 21 | hadoop-hdds in the patch failed. |
   | -1 | javac | 15 | hadoop-ozone in the patch failed. |
   | +1 | checkstyle | 25 | hadoop-hdds: The patch generated 0 new + 0 
unchanged - 3 fixed = 0 total (was 3) |
   | +1 | checkstyle | 29 | The patch passed checkstyle in hadoop-ozone |
   | +1 | mvnsite | 0 | the patch passed |
   | +1 | whitespace | 0 | The patch has no whitespace issues. |
   | +1 | xml | 2 | The patch has no ill-formed XML file. |
   | +1 | shadedclient | 815 | patch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 18 | hadoop-hdds in the patch failed. |
   | -1 | javadoc | 16 | hadoop-ozone in the patch failed. |
   | -1 | findbugs | 28 | hadoop-hdds in the patch failed. |
   | -1 | findbugs | 16 | hadoop-ozone in the patch failed. |
   ||| _ Other Tests _ |
   | -1 | unit | 25 | hadoop-hdds in the patch failed. |
   | -1 | unit | 23 | hadoop-ozone in the patch failed. |
   | +1 | asflicense | 30 | The patch does not generate ASF License warnings. |
   | | | 2595 | |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | Client=19.03.0 Server=19.03.0 base: 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1431 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient findbugs checkstyle xml |
   | uname | Linux 7d17d9b577c2 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 
11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | HDDS-1564 / 7b5a5fe |
   | Default Java | 1.8.0_222 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/branch-mvninstall-hadoop-hdds.txt
 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/branch-mvninstall-hadoop-ozone.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/branch-compile-hadoop-hdds.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/branch-compile-hadoop-ozone.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/branch-javadoc-hadoop-hdds.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/branch-javadoc-hadoop-ozone.txt
 |
   | findbugs | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1431/15/artifact/out/branch-findbugs-hadoop-hdds.txt
 |
   | findbugs | 

[jira] [Work logged] (HDDS-2034) Async RATIS pipeline creation and destroy through heartbeat commands

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2034?focusedWorklogId=320100=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320100
 ]

ASF GitHub Bot logged work on HDDS-2034:


Author: ASF GitHub Bot
Created on: 29/Sep/19 02:36
Start Date: 29/Sep/19 02:36
Worklog Time Spent: 10m 
  Work Description: ChenSammi commented on issue #1469: HDDS-2034. Async 
RATIS pipeline creation and destroy through heartbea…
URL: https://github.com/apache/hadoop/pull/1469#issuecomment-536242054
 
 
   > @ChenSammi Thanks a lot for working on this! Please find my comments below.
   > 
   > 1. For the SafeModeRules, if we allow pipeline creation during safe 
mode we need to modify the rules so that newly created pipelines are not 
counted in the rule.
   > 
   > 2. Can we just trigger PipelineReport from the datanodes after 
creation of pipeline instead of CreatePipelineACK? That would greatly simplify 
the OPEN pipeline code.
   
   Thanks @lokeshj1703  for view the patch. 
   1. For SafeModeRules, I do modified the HealthyPipelineSafeModeRule a bit.  
Add two properties, one is "hdds.scm.safemode.pipeline.creation" to control 
whether create pipeline in safemode. 
   Another is "hdds.scm.safemode.min.pipeline" control the minimum pipeline 
number to exit safe mode when create pipeline in safemode is enabled.  
   2.  It's a good point. I will check the code to see if I leverage the 
trigger pipelineReport to replace the  CreatePipelineACK.
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320100)
Time Spent: 8h 20m  (was: 8h 10m)

> Async RATIS pipeline creation and destroy through heartbeat commands
> 
>
> Key: HDDS-2034
> URL: https://issues.apache.org/jira/browse/HDDS-2034
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>Reporter: Sammi Chen
>Assignee: Sammi Chen
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 8h 20m
>  Remaining Estimate: 0h
>
> Currently, pipeline creation and destroy are synchronous operations. SCM 
> directly connect to each datanode of the pipeline through gRPC channel to 
> create the pipeline to destroy the pipeline.  
> This task is to remove the gRPC channel, send pipeline creation and destroy 
> action through heartbeat command to each datanode.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-6524) Choosing datanode retries times considering with block replica number

2019-09-28 Thread Jira


[ 
https://issues.apache.org/jira/browse/HDFS-6524?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940187#comment-16940187
 ] 

Íñigo Goiri commented on HDFS-6524:
---

Can you walk me over the tests here?
We used to have a replication factor of 1 and now is 3.

> Choosing datanode  retries times considering with block replica number
> --
>
> Key: HDFS-6524
> URL: https://issues.apache.org/jira/browse/HDFS-6524
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: hdfs-client
>Affects Versions: 3.0.0-alpha1
>Reporter: Liang Xie
>Assignee: Lisheng Sun
>Priority: Minor
>  Labels: BB2015-05-TBR
> Attachments: HDFS-6524.001.patch, HDFS-6524.002.patch, 
> HDFS-6524.003.patch, HDFS-6524.004.patch, HDFS-6524.005(2).patch, 
> HDFS-6524.005.patch, HDFS-6524.006.patch, HDFS-6524.txt
>
>
> Currently the chooseDataNode() does retry with the setting: 
> dfsClientConf.maxBlockAcquireFailures, which by default is 3 
> (DFS_CLIENT_MAX_BLOCK_ACQUIRE_FAILURES_DEFAULT = 3), it would be better 
> having another option, block replication factor. One cluster with only  two 
> block replica setting, or using Reed-solomon encoding solution with one 
> replica factor. It helps to reduce the long tail latency.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14850) Optimize FileSystemAccessService#getFileSystemConfiguration

2019-09-28 Thread Hudson (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940186#comment-16940186
 ] 

Hudson commented on HDFS-14850:
---

SUCCESS: Integrated in Jenkins build Hadoop-trunk-Commit #17412 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/17412/])
HDFS-14850. Optimize FileSystemAccessService#getFileSystemConfiguration. 
(inigoiri: rev d8313b227495d748abe8884eee34db507476cee1)
* (edit) 
hadoop-hdfs-project/hadoop-hdfs-httpfs/src/main/java/org/apache/hadoop/lib/service/hadoop/FileSystemAccessService.java


> Optimize FileSystemAccessService#getFileSystemConfiguration
> ---
>
> Key: HDFS-14850
> URL: https://issues.apache.org/jira/browse/HDFS-14850
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: httpfs, performance
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HDFS-14850.001.patch, HDFS-14850.002.patch, 
> HDFS-14850.003.patch, HDFS-14850.004(2).patch, HDFS-14850.004.patch, 
> HDFS-14850.005.patch
>
>
> {code:java}
>  @Override
>   public Configuration getFileSystemConfiguration() {
> Configuration conf = new Configuration(true);
> ConfigurationUtils.copy(serviceHadoopConf, conf);
> conf.setBoolean(FILE_SYSTEM_SERVICE_CREATED, true);
> // Force-clear server-side umask to make HttpFS match WebHDFS behavior
> conf.set(FsPermission.UMASK_LABEL, "000");
> return conf;
>   }
> {code}
> As above code,when call 
> FileSystemAccessService#getFileSystemConfiguration,current code  new 
> Configuration every time.  
> It is not necessary and affects performance. I think it only need to new 
> Configuration in FileSystemAccessService#init once and  
> FileSystemAccessService#getFileSystemConfiguration get it.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14850) Optimize FileSystemAccessService#getFileSystemConfiguration

2019-09-28 Thread Jira


[ 
https://issues.apache.org/jira/browse/HDFS-14850?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940182#comment-16940182
 ] 

Íñigo Goiri commented on HDFS-14850:


Thanks [~leosun08] for the patch.
Committed to trunk.

> Optimize FileSystemAccessService#getFileSystemConfiguration
> ---
>
> Key: HDFS-14850
> URL: https://issues.apache.org/jira/browse/HDFS-14850
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: httpfs, performance
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HDFS-14850.001.patch, HDFS-14850.002.patch, 
> HDFS-14850.003.patch, HDFS-14850.004(2).patch, HDFS-14850.004.patch, 
> HDFS-14850.005.patch
>
>
> {code:java}
>  @Override
>   public Configuration getFileSystemConfiguration() {
> Configuration conf = new Configuration(true);
> ConfigurationUtils.copy(serviceHadoopConf, conf);
> conf.setBoolean(FILE_SYSTEM_SERVICE_CREATED, true);
> // Force-clear server-side umask to make HttpFS match WebHDFS behavior
> conf.set(FsPermission.UMASK_LABEL, "000");
> return conf;
>   }
> {code}
> As above code,when call 
> FileSystemAccessService#getFileSystemConfiguration,current code  new 
> Configuration every time.  
> It is not necessary and affects performance. I think it only need to new 
> Configuration in FileSystemAccessService#init once and  
> FileSystemAccessService#getFileSystemConfiguration get it.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-14850) Optimize FileSystemAccessService#getFileSystemConfiguration

2019-09-28 Thread Jira


 [ 
https://issues.apache.org/jira/browse/HDFS-14850?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Íñigo Goiri updated HDFS-14850:
---
Fix Version/s: 3.3.0
 Hadoop Flags: Reviewed
   Resolution: Fixed
   Status: Resolved  (was: Patch Available)

> Optimize FileSystemAccessService#getFileSystemConfiguration
> ---
>
> Key: HDFS-14850
> URL: https://issues.apache.org/jira/browse/HDFS-14850
> Project: Hadoop HDFS
>  Issue Type: Improvement
>  Components: httpfs, performance
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Major
> Fix For: 3.3.0
>
> Attachments: HDFS-14850.001.patch, HDFS-14850.002.patch, 
> HDFS-14850.003.patch, HDFS-14850.004(2).patch, HDFS-14850.004.patch, 
> HDFS-14850.005.patch
>
>
> {code:java}
>  @Override
>   public Configuration getFileSystemConfiguration() {
> Configuration conf = new Configuration(true);
> ConfigurationUtils.copy(serviceHadoopConf, conf);
> conf.setBoolean(FILE_SYSTEM_SERVICE_CREATED, true);
> // Force-clear server-side umask to make HttpFS match WebHDFS behavior
> conf.set(FsPermission.UMASK_LABEL, "000");
> return conf;
>   }
> {code}
> As above code,when call 
> FileSystemAccessService#getFileSystemConfiguration,current code  new 
> Configuration every time.  
> It is not necessary and affects performance. I think it only need to new 
> Configuration in FileSystemAccessService#init once and  
> FileSystemAccessService#getFileSystemConfiguration get it.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2140) Add robot test for GDPR feature

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2140?focusedWorklogId=320052=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320052
 ]

ASF GitHub Bot logged work on HDDS-2140:


Author: ASF GitHub Bot
Created on: 28/Sep/19 19:57
Start Date: 28/Sep/19 19:57
Worklog Time Spent: 10m 
  Work Description: dineshchitlangia commented on issue #1542: HDDS-2140. 
Add robot test for GDPR feature
URL: https://github.com/apache/hadoop/pull/1542#issuecomment-536220912
 
 
   Failures are unrelated to the patch.
   Checkstyle issue is addressed by HDDS-2202
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320052)
Time Spent: 50m  (was: 40m)

> Add robot test for GDPR feature
> ---
>
> Key: HDDS-2140
> URL: https://issues.apache.org/jira/browse/HDDS-2140
> Project: Hadoop Distributed Data Store
>  Issue Type: Sub-task
>  Components: test
>Reporter: Dinesh Chitlangia
>Assignee: Dinesh Chitlangia
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> Add robot test for GDPR feature so it can be run during smoke tests.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2202) Remove unused import in OmUtils

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2202?focusedWorklogId=320053=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320053
 ]

ASF GitHub Bot logged work on HDDS-2202:


Author: ASF GitHub Bot
Created on: 28/Sep/19 19:57
Start Date: 28/Sep/19 19:57
Worklog Time Spent: 10m 
  Work Description: dineshchitlangia commented on issue #1543: HDDS-2202. 
Remove unused import in OmUtils
URL: https://github.com/apache/hadoop/pull/1543#issuecomment-536220963
 
 
   failures unrelated to patch
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320053)
Time Spent: 40m  (was: 0.5h)

> Remove unused import in OmUtils
> ---
>
> Key: HDDS-2202
> URL: https://issues.apache.org/jira/browse/HDDS-2202
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>Reporter: Dinesh Chitlangia
>Assignee: Dinesh Chitlangia
>Priority: Minor
>  Labels: newbie, pull-request-available
>  Time Spent: 40m
>  Remaining Estimate: 0h
>
> Fix hadoop-ozone/common/src/main/java/org/apache/hadoop/ozone/OmUtils.java
> Remove L49: Unused import - org.apache.hadoop.ozone.om.OMMetadataManager;
>  



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Hadoop QA (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940135#comment-16940135
 ] 

Hadoop QA commented on HDFS-14876:
--

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue}  0m 
47s{color} | {color:blue} Docker mode activated. {color} |
|| || || || {color:brown} Prechecks {color} ||
| {color:green}+1{color} | {color:green} @author {color} | {color:green}  0m  
0s{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:green}+1{color} | {color:green} test4tests {color} | {color:green}  0m 
 0s{color} | {color:green} The patch appears to include 1 new or modified test 
files. {color} |
|| || || || {color:brown} trunk Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 18m 
59s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  0m 
59s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
43s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m  
7s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
14m  4s{color} | {color:green} branch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  2m 
15s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  1m 
15s{color} | {color:green} trunk passed {color} |
|| || || || {color:brown} Patch Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  1m 
 0s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  0m 
53s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green}  0m 
53s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
37s{color} | {color:green} hadoop-hdfs-project/hadoop-hdfs: The patch generated 
0 new + 2 unchanged - 1 fixed = 2 total (was 3) {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  0m 
59s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green}  0m 
 0s{color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
13m  5s{color} | {color:green} patch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  2m 
21s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  1m 
10s{color} | {color:green} the patch passed {color} |
|| || || || {color:brown} Other Tests {color} ||
| {color:red}-1{color} | {color:red} unit {color} | {color:red} 96m 41s{color} 
| {color:red} hadoop-hdfs in the patch failed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green}  0m 
35s{color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black}157m 26s{color} | 
{color:black} {color} |
\\
\\
|| Reason || Tests ||
| Failed junit tests | hadoop.hdfs.tools.TestDFSAdminWithHA |
|   | hadoop.hdfs.server.namenode.ha.TestBootstrapStandby |
|   | hadoop.hdfs.qjournal.server.TestJournalNodeSync |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | Client=18.09.7 Server=18.09.7 Image:yetus/hadoop:efed4450bf1 |
| JIRA Issue | HDFS-14876 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/12981695/HDFS-14876.002.patch |
| Optional Tests |  dupname  asflicense  compile  javac  javadoc  mvninstall  
mvnsite  unit  shadedclient  findbugs  checkstyle  |
| uname | Linux 1200fb5b46e7 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 
11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
| Build tool | maven |
| Personality | /testptch/patchprocess/precommit/personality/provided.sh |
| git revision | trunk / c4c8d5f |
| maven | version: Apache Maven 3.3.9 |
| Default Java | 1.8.0_222 |
| findbugs | v3.1.0-RC1 |
| unit | 
https://builds.apache.org/job/PreCommit-HDFS-Build/27980/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt
 |
|  Test Results | 
https://builds.apache.org/job/PreCommit-HDFS-Build/27980/testReport/ |
| Max. process+thread count | 2728 (vs. ulimit of 5500) |
| modules | C: hadoop-hdfs-project/hadoop-hdfs U: 

[jira] [Work logged] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?focusedWorklogId=320040=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320040
 ]

ASF GitHub Bot logged work on HDDS-2203:


Author: ASF GitHub Bot
Created on: 28/Sep/19 18:08
Start Date: 28/Sep/19 18:08
Worklog Time Spent: 10m 
  Work Description: anuengineer commented on issue #1544: HDDS-2203 Race 
condition in ByteStringHelper.init()
URL: https://github.com/apache/hadoop/pull/1544#issuecomment-536212092
 
 
   Code changes look good to me. I have set the Ozone label, Let the Jenkins 
run once more and I will commit this. Thank you for the fix.
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320040)
Time Spent: 0.5h  (was: 20m)

> Race condition in ByteStringHelper.init()
> -
>
> Key: HDDS-2203
> URL: https://issues.apache.org/jira/browse/HDDS-2203
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>  Components: Ozone Client, SCM
>Reporter: Istvan Fajth
>Assignee: Istvan Fajth
>Priority: Critical
>  Labels: pull-request-available, pull-requests-available
>  Time Spent: 0.5h
>  Remaining Estimate: 0h
>
> The current init method:
> {code}
> public static void init(boolean isUnsafeByteOperation) {
>   final boolean set = INITIALIZED.compareAndSet(false, true);
>   if (set) {
> ByteStringHelper.isUnsafeByteOperationsEnabled =
>isUnsafeByteOperation;
>} else {
>  // already initialized, check values
>  Preconditions.checkState(isUnsafeByteOperationsEnabled
>== isUnsafeByteOperation);
>}
> }
> {code}
> In a scenario when two thread accesses this method, and the execution order 
> is the following, then the second thread runs into an exception from 
> PreCondition.checkState() in the else branch.
> In an unitialized state:
> - T1 thread arrives to the method with true as the parameter, the class 
> initialises the isUnsafeByteOperationsEnabled to false
> - T1 sets INITIALIZED true
> - T2 arrives to the method with true as the parameter
> - T2 reads the INITALIZED value and as it is not false goes to else branch
> - T2 tries to check if the internal boolean property is the same true as it 
> wanted to set, and as T1 still to set the value, the checkState throws an 
> IllegalArgumentException.
> This happens in certain Hive query cases, as it came from that testing, the 
> exception we see there is the following:
> {code}
> Error: Error while processing statement: FAILED: Execution Error, return code 
> 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, 
> vertexName=Map 2, vertexId=vertex_1569486223160_0334_1_02, 
> diagnostics=[Vertex vertex_1569486223160_0334_1_02 [Map 2] killed/failed
>  due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
> vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
> create RpcClient protocol
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
> at 
> org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
> at 
> org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:2002)
> at 
> 

[jira] [Commented] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Hudson (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940106#comment-16940106
 ] 

Hudson commented on HDFS-14876:
---

SUCCESS: Integrated in Jenkins build Hadoop-trunk-Commit #17411 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/17411/])
HDFS-14876. Remove unused imports from TestBlockMissingException.java 
(ayushsaxena: rev 22008716075b461ef48e801fc7049cfad6aade45)
* (edit) 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestClose.java
* (edit) 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java


> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Fix For: 3.3.0
>
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14849) Erasure Coding: the internal block is replicated many times when datanode is decommissioning

2019-09-28 Thread Hudson (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940099#comment-16940099
 ] 

Hudson commented on HDFS-14849:
---

SUCCESS: Integrated in Jenkins build Hadoop-trunk-Commit #17410 (See 
[https://builds.apache.org/job/Hadoop-trunk-Commit/17410/])
Revert "HDFS-14849. Erasure Coding: the internal block is replicated 
(ayushsaxena: rev 0d5d0b914ac959ce2c41f483ac5b74f58053cd00)
* (edit) 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
* (edit) 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockMissingException.java
* (edit) 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
HDFS-14849. Erasure Coding: the internal block is replicated many times 
(ayushsaxena: rev c4c8d5fd0e3c17ccdcf18ece8e005f510328b060)
* (edit) 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
* (edit) 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java


> Erasure Coding: the internal block is replicated many times when datanode is 
> decommissioning
> 
>
> Key: HDFS-14849
> URL: https://issues.apache.org/jira/browse/HDFS-14849
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: ec, erasure-coding
>Affects Versions: 3.3.0
>Reporter: HuangTao
>Assignee: HuangTao
>Priority: Major
>  Labels: EC, HDFS, NameNode
> Fix For: 3.3.0
>
> Attachments: HDFS-14849.001.patch, HDFS-14849.002.patch, 
> fsck-file.png, liveBlockIndices.png, scheduleReconstruction.png
>
>
> When the datanode keeping in DECOMMISSION_INPROGRESS status, the EC internal 
> block in that datanode will be replicated many times.
> // added 2019/09/19
> I reproduced this scenario in a 163 nodes cluster with decommission 100 nodes 
> simultaneously. 
>  !scheduleReconstruction.png! 
>  !fsck-file.png! 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Ayush Saxena (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ayush Saxena updated HDFS-14876:

Attachment: (was: HDFS-14876.002.patch)

> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Fix For: 3.3.0
>
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Ayush Saxena (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ayush Saxena updated HDFS-14876:

Fix Version/s: 3.3.0
 Hadoop Flags: Reviewed
   Resolution: Fixed
   Status: Resolved  (was: Patch Available)

> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Fix For: 3.3.0
>
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Ayush Saxena (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940088#comment-16940088
 ] 

Ayush Saxena commented on HDFS-14876:
-

Committed to trunk.
Thanx [~leosun08] for the contribution.

> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Ayush Saxena (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940087#comment-16940087
 ] 

Ayush Saxena commented on HDFS-14876:
-

[~leosun08] that went by mistake, I took it back. Committing v1. Sorry for the 
trouble.

> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch, 
> HDFS-14876.002.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14849) Erasure Coding: the internal block is replicated many times when datanode is decommissioning

2019-09-28 Thread Ayush Saxena (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14849?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940074#comment-16940074
 ] 

Ayush Saxena commented on HDFS-14849:
-

FYI. While commit, Got change from another JIRA too. Re-Commiting with the 
changes only here

> Erasure Coding: the internal block is replicated many times when datanode is 
> decommissioning
> 
>
> Key: HDFS-14849
> URL: https://issues.apache.org/jira/browse/HDFS-14849
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: ec, erasure-coding
>Affects Versions: 3.3.0
>Reporter: HuangTao
>Assignee: HuangTao
>Priority: Major
>  Labels: EC, HDFS, NameNode
> Fix For: 3.3.0
>
> Attachments: HDFS-14849.001.patch, HDFS-14849.002.patch, 
> fsck-file.png, liveBlockIndices.png, scheduleReconstruction.png
>
>
> When the datanode keeping in DECOMMISSION_INPROGRESS status, the EC internal 
> block in that datanode will be replicated many times.
> // added 2019/09/19
> I reproduced this scenario in a 163 nodes cluster with decommission 100 nodes 
> simultaneously. 
>  !scheduleReconstruction.png! 
>  !fsck-file.png! 



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Lisheng Sun (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940073#comment-16940073
 ] 

Lisheng Sun commented on HDFS-14876:


Hi [~ayushtkn]  HDFS-14849  remove unused imports in 
TestBlockMissingException.java. 

So i updated this patch and uploaded the v002 patch. Would you mind taking a 
review for patch? Thank you.

> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch, 
> HDFS-14876.002.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Lisheng Sun (Jira)


 [ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Lisheng Sun updated HDFS-14876:
---
Attachment: HDFS-14876.002.patch

> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch, 
> HDFS-14876.002.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated HDDS-2203:
-
Labels: pull-request-available pull-requests-available  (was: 
pull-requests-available)

> Race condition in ByteStringHelper.init()
> -
>
> Key: HDDS-2203
> URL: https://issues.apache.org/jira/browse/HDDS-2203
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>  Components: Ozone Client, SCM
>Reporter: Istvan Fajth
>Assignee: Istvan Fajth
>Priority: Critical
>  Labels: pull-request-available, pull-requests-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The current init method:
> {code}
> public static void init(boolean isUnsafeByteOperation) {
>   final boolean set = INITIALIZED.compareAndSet(false, true);
>   if (set) {
> ByteStringHelper.isUnsafeByteOperationsEnabled =
>isUnsafeByteOperation;
>} else {
>  // already initialized, check values
>  Preconditions.checkState(isUnsafeByteOperationsEnabled
>== isUnsafeByteOperation);
>}
> }
> {code}
> In a scenario when two thread accesses this method, and the execution order 
> is the following, then the second thread runs into an exception from 
> PreCondition.checkState() in the else branch.
> In an unitialized state:
> - T1 thread arrives to the method with true as the parameter, the class 
> initialises the isUnsafeByteOperationsEnabled to false
> - T1 sets INITIALIZED true
> - T2 arrives to the method with true as the parameter
> - T2 reads the INITALIZED value and as it is not false goes to else branch
> - T2 tries to check if the internal boolean property is the same true as it 
> wanted to set, and as T1 still to set the value, the checkState throws an 
> IllegalArgumentException.
> This happens in certain Hive query cases, as it came from that testing, the 
> exception we see there is the following:
> {code}
> Error: Error while processing statement: FAILED: Execution Error, return code 
> 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, 
> vertexName=Map 2, vertexId=vertex_1569486223160_0334_1_02, 
> diagnostics=[Vertex vertex_1569486223160_0334_1_02 [Map 2] killed/failed
>  due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
> vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
> create RpcClient protocol
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
> at 
> org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
> at 
> org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:2002)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:524)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:781)
> at 
> org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:243)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:278)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:269)
> 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:1876)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:269)
> at 
> 

[jira] [Work logged] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?focusedWorklogId=320026=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320026
 ]

ASF GitHub Bot logged work on HDDS-2203:


Author: ASF GitHub Bot
Created on: 28/Sep/19 15:56
Start Date: 28/Sep/19 15:56
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on issue #1544: HDDS-2203 Race 
condition in ByteStringHelper.init()
URL: https://github.com/apache/hadoop/pull/1544#issuecomment-536201609
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |::|--:|:|:|
   | 0 | reexec | 78 | Docker mode activated. |
   ||| _ Prechecks _ |
   | +1 | dupname | 0 | No case conflicting files found. |
   | +1 | @author | 0 | The patch does not contain any @author tags. |
   | -1 | test4tests | 0 | The patch doesn't appear to include any new or 
modified tests.  Please justify why no new tests are needed for this patch. 
Also please list what manual steps were performed to verify this patch. |
   ||| _ trunk Compile Tests _ |
   | -1 | mvninstall | 29 | hadoop-hdds in trunk failed. |
   | -1 | mvninstall | 42 | hadoop-ozone in trunk failed. |
   | -1 | compile | 21 | hadoop-hdds in trunk failed. |
   | -1 | compile | 16 | hadoop-ozone in trunk failed. |
   | -0 | checkstyle | 39 | The patch fails to run checkstyle in hadoop-ozone |
   | +1 | mvnsite | 0 | trunk passed |
   | +1 | shadedclient | 854 | branch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 25 | hadoop-hdds in trunk failed. |
   | -1 | javadoc | 20 | hadoop-ozone in trunk failed. |
   | 0 | spotbugs | 979 | Used deprecated FindBugs config; considering 
switching to SpotBugs. |
   | -1 | findbugs | 53 | hadoop-hdds in trunk failed. |
   | -1 | findbugs | 21 | hadoop-ozone in trunk failed. |
   ||| _ Patch Compile Tests _ |
   | -1 | mvninstall | 35 | hadoop-hdds in the patch failed. |
   | -1 | mvninstall | 38 | hadoop-ozone in the patch failed. |
   | -1 | compile | 24 | hadoop-hdds in the patch failed. |
   | -1 | compile | 20 | hadoop-ozone in the patch failed. |
   | -1 | javac | 24 | hadoop-hdds in the patch failed. |
   | -1 | javac | 20 | hadoop-ozone in the patch failed. |
   | -0 | checkstyle | 27 | hadoop-hdds: The patch generated 2 new + 0 
unchanged - 0 fixed = 2 total (was 0) |
   | -0 | checkstyle | 29 | The patch fails to run checkstyle in hadoop-ozone |
   | +1 | mvnsite | 0 | the patch passed |
   | +1 | whitespace | 0 | The patch has no whitespace issues. |
   | +1 | shadedclient | 729 | patch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 23 | hadoop-hdds in the patch failed. |
   | -1 | javadoc | 20 | hadoop-ozone in the patch failed. |
   | -1 | findbugs | 31 | hadoop-hdds in the patch failed. |
   | -1 | findbugs | 21 | hadoop-ozone in the patch failed. |
   ||| _ Other Tests _ |
   | -1 | unit | 27 | hadoop-hdds in the patch failed. |
   | -1 | unit | 26 | hadoop-ozone in the patch failed. |
   | +1 | asflicense | 33 | The patch does not generate ASF License warnings. |
   | | | 2431 | |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | Client=19.03.2 Server=19.03.2 base: 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1544 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux 609b5b23109c 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 
16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 14b4fbc |
   | Default Java | 1.8.0_222 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out/branch-mvninstall-hadoop-hdds.txt
 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out/branch-mvninstall-hadoop-ozone.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out/branch-compile-hadoop-hdds.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out/branch-compile-hadoop-ozone.txt
 |
   | checkstyle | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out//home/jenkins/jenkins-slave/workspace/hadoop-multibranch_PR-1544/out/maven-branch-checkstyle-hadoop-ozone.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out/branch-javadoc-hadoop-hdds.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1544/1/artifact/out/branch-javadoc-hadoop-ozone.txt
 |
   | findbugs | 

[jira] [Commented] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Hadoop QA (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940056#comment-16940056
 ] 

Hadoop QA commented on HDFS-14876:
--

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue}  0m  
0s{color} | {color:blue} Docker mode activated. {color} |
| {color:red}-1{color} | {color:red} patch {color} | {color:red}  0m  7s{color} 
| {color:red} HDFS-14876 does not apply to trunk. Rebase required? Wrong 
Branch? See https://wiki.apache.org/hadoop/HowToContribute for help. {color} |
\\
\\
|| Subsystem || Report/Notes ||
| JIRA Issue | HDFS-14876 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/12981564/HDFS-14876.001.patch |
| Console output | 
https://builds.apache.org/job/PreCommit-HDFS-Build/27979/console |
| Powered by | Apache Yetus 0.8.0   http://yetus.apache.org |


This message was automatically generated.



> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Commented] (HDFS-14876) Remove unused imports from TestBlockMissingException.java and TestClose.java

2019-09-28 Thread Ayush Saxena (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14876?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16940053#comment-16940053
 ] 

Ayush Saxena commented on HDFS-14876:
-

+1

> Remove unused imports from TestBlockMissingException.java and TestClose.java
> 
>
> Key: HDFS-14876
> URL: https://issues.apache.org/jira/browse/HDFS-14876
> Project: Hadoop HDFS
>  Issue Type: Bug
>  Components: test
>Reporter: Lisheng Sun
>Assignee: Lisheng Sun
>Priority: Minor
> Attachments: HDFS-14876.000.patch, HDFS-14876.001.patch
>
>
> There 3 unused imports in TestBlockMissingException.java and TestClose.java. 
> Let's clean them up.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Updated] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread Istvan Fajth (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Istvan Fajth updated HDDS-2203:
---
Component/s: SCM
 Ozone Client

> Race condition in ByteStringHelper.init()
> -
>
> Key: HDDS-2203
> URL: https://issues.apache.org/jira/browse/HDDS-2203
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>  Components: Ozone Client, SCM
>Reporter: Istvan Fajth
>Assignee: Istvan Fajth
>Priority: Critical
>  Labels: pull-requests-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The current init method:
> {code}
> public static void init(boolean isUnsafeByteOperation) {
>   final boolean set = INITIALIZED.compareAndSet(false, true);
>   if (set) {
> ByteStringHelper.isUnsafeByteOperationsEnabled =
>isUnsafeByteOperation;
>} else {
>  // already initialized, check values
>  Preconditions.checkState(isUnsafeByteOperationsEnabled
>== isUnsafeByteOperation);
>}
> }
> {code}
> In a scenario when two thread accesses this method, and the execution order 
> is the following, then the second thread runs into an exception from 
> PreCondition.checkState() in the else branch.
> In an unitialized state:
> - T1 thread arrives to the method with true as the parameter, the class 
> initialises the isUnsafeByteOperationsEnabled to false
> - T1 sets INITIALIZED true
> - T2 arrives to the method with true as the parameter
> - T2 reads the INITALIZED value and as it is not false goes to else branch
> - T2 tries to check if the internal boolean property is the same true as it 
> wanted to set, and as T1 still to set the value, the checkState throws an 
> IllegalArgumentException.
> This happens in certain Hive query cases, as it came from that testing, the 
> exception we see there is the following:
> {code}
> Error: Error while processing statement: FAILED: Execution Error, return code 
> 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, 
> vertexName=Map 2, vertexId=vertex_1569486223160_0334_1_02, 
> diagnostics=[Vertex vertex_1569486223160_0334_1_02 [Map 2] killed/failed
>  due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
> vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
> create RpcClient protocol
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
> at 
> org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
> at 
> org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:2002)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:524)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:781)
> at 
> org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:243)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:278)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:269)
> 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:1876)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:269)
> at 
> 

[jira] [Updated] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread Istvan Fajth (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Istvan Fajth updated HDDS-2203:
---
Labels: pull-requests-available  (was: pull-request-available)

> Race condition in ByteStringHelper.init()
> -
>
> Key: HDDS-2203
> URL: https://issues.apache.org/jira/browse/HDDS-2203
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>Reporter: Istvan Fajth
>Assignee: Istvan Fajth
>Priority: Critical
>  Labels: pull-requests-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The current init method:
> {code}
> public static void init(boolean isUnsafeByteOperation) {
>   final boolean set = INITIALIZED.compareAndSet(false, true);
>   if (set) {
> ByteStringHelper.isUnsafeByteOperationsEnabled =
>isUnsafeByteOperation;
>} else {
>  // already initialized, check values
>  Preconditions.checkState(isUnsafeByteOperationsEnabled
>== isUnsafeByteOperation);
>}
> }
> {code}
> In a scenario when two thread accesses this method, and the execution order 
> is the following, then the second thread runs into an exception from 
> PreCondition.checkState() in the else branch.
> In an unitialized state:
> - T1 thread arrives to the method with true as the parameter, the class 
> initialises the isUnsafeByteOperationsEnabled to false
> - T1 sets INITIALIZED true
> - T2 arrives to the method with true as the parameter
> - T2 reads the INITALIZED value and as it is not false goes to else branch
> - T2 tries to check if the internal boolean property is the same true as it 
> wanted to set, and as T1 still to set the value, the checkState throws an 
> IllegalArgumentException.
> This happens in certain Hive query cases, as it came from that testing, the 
> exception we see there is the following:
> {code}
> Error: Error while processing statement: FAILED: Execution Error, return code 
> 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, 
> vertexName=Map 2, vertexId=vertex_1569486223160_0334_1_02, 
> diagnostics=[Vertex vertex_1569486223160_0334_1_02 [Map 2] killed/failed
>  due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
> vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
> create RpcClient protocol
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
> at 
> org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
> at 
> org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:2002)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:524)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:781)
> at 
> org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:243)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:278)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:269)
> 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:1876)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:269)
> at 
> 

[jira] [Work logged] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?focusedWorklogId=320013=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-320013
 ]

ASF GitHub Bot logged work on HDDS-2203:


Author: ASF GitHub Bot
Created on: 28/Sep/19 14:47
Start Date: 28/Sep/19 14:47
Worklog Time Spent: 10m 
  Work Description: fapifta commented on pull request #1544: HDDS-2203 Race 
condition in ByteStringHelper.init()
URL: https://github.com/apache/hadoop/pull/1544
 
 
   ## NOTICE
   
   Please create an issue in ASF JIRA before opening a pull request,
   and you need to set the title of the pull request which starts with
   the corresponding JIRA issue number. (e.g. HADOOP-X. Fix a typo in YYY.)
   For more details, please see 
https://cwiki.apache.org/confluence/display/HADOOP/How+To+Contribute
   
 

This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
us...@infra.apache.org


Issue Time Tracking
---

Worklog Id: (was: 320013)
Remaining Estimate: 0h
Time Spent: 10m

> Race condition in ByteStringHelper.init()
> -
>
> Key: HDDS-2203
> URL: https://issues.apache.org/jira/browse/HDDS-2203
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>Reporter: Istvan Fajth
>Assignee: Istvan Fajth
>Priority: Critical
>  Labels: pull-requests-available
>  Time Spent: 10m
>  Remaining Estimate: 0h
>
> The current init method:
> {code}
> public static void init(boolean isUnsafeByteOperation) {
>   final boolean set = INITIALIZED.compareAndSet(false, true);
>   if (set) {
> ByteStringHelper.isUnsafeByteOperationsEnabled =
>isUnsafeByteOperation;
>} else {
>  // already initialized, check values
>  Preconditions.checkState(isUnsafeByteOperationsEnabled
>== isUnsafeByteOperation);
>}
> }
> {code}
> In a scenario when two thread accesses this method, and the execution order 
> is the following, then the second thread runs into an exception from 
> PreCondition.checkState() in the else branch.
> In an unitialized state:
> - T1 thread arrives to the method with true as the parameter, the class 
> initialises the isUnsafeByteOperationsEnabled to false
> - T1 sets INITIALIZED true
> - T2 arrives to the method with true as the parameter
> - T2 reads the INITALIZED value and as it is not false goes to else branch
> - T2 tries to check if the internal boolean property is the same true as it 
> wanted to set, and as T1 still to set the value, the checkState throws an 
> IllegalArgumentException.
> This happens in certain Hive query cases, as it came from that testing, the 
> exception we see there is the following:
> {code}
> Error: Error while processing statement: FAILED: Execution Error, return code 
> 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, 
> vertexName=Map 2, vertexId=vertex_1569486223160_0334_1_02, 
> diagnostics=[Vertex vertex_1569486223160_0334_1_02 [Map 2] killed/failed
>  due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
> vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
> create RpcClient protocol
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
> at 
> org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
> at 
> org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
> at 
> 

[jira] [Updated] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

ASF GitHub Bot updated HDDS-2203:
-
Labels: pull-request-available  (was: )

> Race condition in ByteStringHelper.init()
> -
>
> Key: HDDS-2203
> URL: https://issues.apache.org/jira/browse/HDDS-2203
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>Reporter: Istvan Fajth
>Assignee: Istvan Fajth
>Priority: Critical
>  Labels: pull-request-available
>
> The current init method:
> {code}
> public static void init(boolean isUnsafeByteOperation) {
>   final boolean set = INITIALIZED.compareAndSet(false, true);
>   if (set) {
> ByteStringHelper.isUnsafeByteOperationsEnabled =
>isUnsafeByteOperation;
>} else {
>  // already initialized, check values
>  Preconditions.checkState(isUnsafeByteOperationsEnabled
>== isUnsafeByteOperation);
>}
> }
> {code}
> In a scenario when two thread accesses this method, and the execution order 
> is the following, then the second thread runs into an exception from 
> PreCondition.checkState() in the else branch.
> In an unitialized state:
> - T1 thread arrives to the method with true as the parameter, the class 
> initialises the isUnsafeByteOperationsEnabled to false
> - T1 sets INITIALIZED true
> - T2 arrives to the method with true as the parameter
> - T2 reads the INITALIZED value and as it is not false goes to else branch
> - T2 tries to check if the internal boolean property is the same true as it 
> wanted to set, and as T1 still to set the value, the checkState throws an 
> IllegalArgumentException.
> This happens in certain Hive query cases, as it came from that testing, the 
> exception we see there is the following:
> {code}
> Error: Error while processing statement: FAILED: Execution Error, return code 
> 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, 
> vertexName=Map 2, vertexId=vertex_1569486223160_0334_1_02, 
> diagnostics=[Vertex vertex_1569486223160_0334_1_02 [Map 2] killed/failed
>  due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
> vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
> create RpcClient protocol
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
> at 
> org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
> at 
> org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:2002)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:524)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:781)
> at 
> org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:243)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:278)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:269)
> 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:1876)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:269)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:253)
> at 
> 

[jira] [Assigned] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread Istvan Fajth (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2203?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Istvan Fajth reassigned HDDS-2203:
--

Assignee: Istvan Fajth

> Race condition in ByteStringHelper.init()
> -
>
> Key: HDDS-2203
> URL: https://issues.apache.org/jira/browse/HDDS-2203
> Project: Hadoop Distributed Data Store
>  Issue Type: Bug
>Reporter: Istvan Fajth
>Assignee: Istvan Fajth
>Priority: Critical
>
> The current init method:
> {code}
> public static void init(boolean isUnsafeByteOperation) {
>   final boolean set = INITIALIZED.compareAndSet(false, true);
>   if (set) {
> ByteStringHelper.isUnsafeByteOperationsEnabled =
>isUnsafeByteOperation;
>} else {
>  // already initialized, check values
>  Preconditions.checkState(isUnsafeByteOperationsEnabled
>== isUnsafeByteOperation);
>}
> }
> {code}
> In a scenario when two thread accesses this method, and the execution order 
> is the following, then the second thread runs into an exception from 
> PreCondition.checkState() in the else branch.
> In an unitialized state:
> - T1 thread arrives to the method with true as the parameter, the class 
> initialises the isUnsafeByteOperationsEnabled to false
> - T1 sets INITIALIZED true
> - T2 arrives to the method with true as the parameter
> - T2 reads the INITALIZED value and as it is not false goes to else branch
> - T2 tries to check if the internal boolean property is the same true as it 
> wanted to set, and as T1 still to set the value, the checkState throws an 
> IllegalArgumentException.
> This happens in certain Hive query cases, as it came from that testing, the 
> exception we see there is the following:
> {code}
> Error: Error while processing statement: FAILED: Execution Error, return code 
> 2 from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, 
> vertexName=Map 2, vertexId=vertex_1569486223160_0334_1_02, 
> diagnostics=[Vertex vertex_1569486223160_0334_1_02 [Map 2] killed/failed
>  due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
> vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
> create RpcClient protocol
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
> at 
> org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
> at 
> org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
> at 
> org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
> at 
> org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
> at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
> at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
> at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
> at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
> at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
> at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
> at 
> org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:2002)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:524)
> at 
> org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:781)
> at 
> org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:243)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:278)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:269)
> 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:1876)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:269)
> at 
> org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:253)
> at 
> 

[jira] [Created] (HDDS-2203) Race condition in ByteStringHelper.init()

2019-09-28 Thread Istvan Fajth (Jira)
Istvan Fajth created HDDS-2203:
--

 Summary: Race condition in ByteStringHelper.init()
 Key: HDDS-2203
 URL: https://issues.apache.org/jira/browse/HDDS-2203
 Project: Hadoop Distributed Data Store
  Issue Type: Bug
Reporter: Istvan Fajth


The current init method:

{code}
public static void init(boolean isUnsafeByteOperation) {
  final boolean set = INITIALIZED.compareAndSet(false, true);
  if (set) {
ByteStringHelper.isUnsafeByteOperationsEnabled =
   isUnsafeByteOperation;
   } else {
 // already initialized, check values
 Preconditions.checkState(isUnsafeByteOperationsEnabled
   == isUnsafeByteOperation);
   }
}
{code}

In a scenario when two thread accesses this method, and the execution order is 
the following, then the second thread runs into an exception from 
PreCondition.checkState() in the else branch.

In an unitialized state:
- T1 thread arrives to the method with true as the parameter, the class 
initialises the isUnsafeByteOperationsEnabled to false
- T1 sets INITIALIZED true
- T2 arrives to the method with true as the parameter
- T2 reads the INITALIZED value and as it is not false goes to else branch
- T2 tries to check if the internal boolean property is the same true as it 
wanted to set, and as T1 still to set the value, the checkState throws an 
IllegalArgumentException.

This happens in certain Hive query cases, as it came from that testing, the 
exception we see there is the following:
{code}
Error: Error while processing statement: FAILED: Execution Error, return code 2 
from org.apache.hadoop.hive.ql.exec.tez.TezTask. Vertex failed, vertexName=Map 
2, vertexId=vertex_1569486223160_0334_1_02, diagnostics=[Vertex 
vertex_1569486223160_0334_1_02 [Map 2] killed/failed
 due to:ROOT_INPUT_INIT_FAILURE, Vertex Input: item initializer failed, 
vertex=vertex_1569486223160_0334_1_02 [Map 2], java.io.IOException: Couldn't 
create RpcClient protocol
at 
org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:263)
at 
org.apache.hadoop.ozone.client.OzoneClientFactory.getClientProtocol(OzoneClientFactory.java:239)
at 
org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:203)
at 
org.apache.hadoop.ozone.client.OzoneClientFactory.getRpcClient(OzoneClientFactory.java:165)
at 
org.apache.hadoop.fs.ozone.BasicOzoneClientAdapterImpl.(BasicOzoneClientAdapterImpl.java:158)
at 
org.apache.hadoop.fs.ozone.OzoneClientAdapterImpl.(OzoneClientAdapterImpl.java:50)
at 
org.apache.hadoop.fs.ozone.OzoneFileSystem.createAdapter(OzoneFileSystem.java:102)
at 
org.apache.hadoop.fs.ozone.BasicOzoneFileSystem.initialize(BasicOzoneFileSystem.java:155)
at org.apache.hadoop.fs.FileSystem.createFileSystem(FileSystem.java:3315)
at org.apache.hadoop.fs.FileSystem.access$200(FileSystem.java:136)
at org.apache.hadoop.fs.FileSystem$Cache.getInternal(FileSystem.java:3364)
at org.apache.hadoop.fs.FileSystem$Cache.get(FileSystem.java:3332)
at org.apache.hadoop.fs.FileSystem.get(FileSystem.java:491)
at org.apache.hadoop.fs.Path.getFileSystem(Path.java:361)
at 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.generateSplitsInfo(OrcInputFormat.java:1821)
at 
org.apache.hadoop.hive.ql.io.orc.OrcInputFormat.getSplits(OrcInputFormat.java:2002)
at 
org.apache.hadoop.hive.ql.io.HiveInputFormat.addSplitsForGroup(HiveInputFormat.java:524)
at 
org.apache.hadoop.hive.ql.io.HiveInputFormat.getSplits(HiveInputFormat.java:781)
at 
org.apache.hadoop.hive.ql.exec.tez.HiveSplitGenerator.initialize(HiveSplitGenerator.java:243)
at 
org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:278)
at 
org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable$1.run(RootInputInitializerManager.java:269)
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:1876)
at 
org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:269)
at 
org.apache.tez.dag.app.dag.RootInputInitializerManager$InputInitializerCallable.call(RootInputInitializerManager.java:253)
at 
com.google.common.util.concurrent.TrustedListenableFutureTask$TrustedFutureInterruptibleTask.runInterruptibly(TrustedListenableFutureTask.java:108)
at 
com.google.common.util.concurrent.InterruptibleTask.run(InterruptibleTask.java:41)
at 
com.google.common.util.concurrent.TrustedListenableFutureTask.run(TrustedListenableFutureTask.java:77)
at 
java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1149)
at 
java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:624)
 

[jira] [Commented] (HDFS-14730) Remove unused configuration dfs.web.authentication.filter

2019-09-28 Thread Hadoop QA (Jira)


[ 
https://issues.apache.org/jira/browse/HDFS-14730?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel=16939903#comment-16939903
 ] 

Hadoop QA commented on HDFS-14730:
--

| (x) *{color:red}-1 overall{color}* |
\\
\\
|| Vote || Subsystem || Runtime || Comment ||
| {color:blue}0{color} | {color:blue} reexec {color} | {color:blue}  0m 
44s{color} | {color:blue} Docker mode activated. {color} |
|| || || || {color:brown} Prechecks {color} ||
| {color:green}+1{color} | {color:green} @author {color} | {color:green}  0m  
0s{color} | {color:green} The patch does not contain any @author tags. {color} |
| {color:green}+1{color} | {color:green} test4tests {color} | {color:green}  0m 
 0s{color} | {color:green} The patch appears to include 1 new or modified test 
files. {color} |
|| || || || {color:brown} trunk Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green} 20m 
21s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
2s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
51s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m  
9s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
14m 51s{color} | {color:green} branch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  2m 
21s{color} | {color:green} trunk passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  1m 
18s{color} | {color:green} trunk passed {color} |
|| || || || {color:brown} Patch Compile Tests {color} ||
| {color:green}+1{color} | {color:green} mvninstall {color} | {color:green}  1m 
 8s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} compile {color} | {color:green}  1m  
3s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javac {color} | {color:green}  1m  
3s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} checkstyle {color} | {color:green}  0m 
49s{color} | {color:green} hadoop-hdfs-project/hadoop-hdfs: The patch generated 
0 new + 449 unchanged - 7 fixed = 449 total (was 456) {color} |
| {color:green}+1{color} | {color:green} mvnsite {color} | {color:green}  1m  
7s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} whitespace {color} | {color:green}  0m 
 0s{color} | {color:green} The patch has no whitespace issues. {color} |
| {color:green}+1{color} | {color:green} xml {color} | {color:green}  0m  
2s{color} | {color:green} The patch has no ill-formed XML file. {color} |
| {color:green}+1{color} | {color:green} shadedclient {color} | {color:green} 
13m 43s{color} | {color:green} patch has no errors when building and testing 
our client artifacts. {color} |
| {color:green}+1{color} | {color:green} findbugs {color} | {color:green}  2m 
30s{color} | {color:green} the patch passed {color} |
| {color:green}+1{color} | {color:green} javadoc {color} | {color:green}  1m 
12s{color} | {color:green} the patch passed {color} |
|| || || || {color:brown} Other Tests {color} ||
| {color:red}-1{color} | {color:red} unit {color} | {color:red}117m 39s{color} 
| {color:red} hadoop-hdfs in the patch failed. {color} |
| {color:green}+1{color} | {color:green} asflicense {color} | {color:green}  0m 
40s{color} | {color:green} The patch does not generate ASF License warnings. 
{color} |
| {color:black}{color} | {color:black} {color} | {color:black}182m 19s{color} | 
{color:black} {color} |
\\
\\
|| Reason || Tests ||
| Failed junit tests | hadoop.hdfs.TestDFSClientRetries |
|   | hadoop.hdfs.server.namenode.TestPersistentStoragePolicySatisfier |
|   | hadoop.hdfs.server.namenode.ha.TestHAAppend |
|   | hadoop.hdfs.TestRollingUpgrade |
\\
\\
|| Subsystem || Report/Notes ||
| Docker | Client=18.09.7 Server=18.09.7 Image:yetus/hadoop:efed4450bf1 |
| JIRA Issue | HDFS-14730 |
| JIRA Patch URL | 
https://issues.apache.org/jira/secure/attachment/12981442/HDFS-14730.002.patch |
| Optional Tests |  dupname  asflicense  compile  javac  javadoc  mvninstall  
mvnsite  unit  shadedclient  findbugs  checkstyle  xml  |
| uname | Linux 6f0f7c6d3f95 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 
11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
| Build tool | maven |
| Personality | /testptch/patchprocess/precommit/personality/provided.sh |
| git revision | trunk / 14b4fbc |
| maven | version: Apache Maven 3.3.9 |
| Default Java | 1.8.0_222 |
| findbugs | v3.1.0-RC1 |
| unit | 
https://builds.apache.org/job/PreCommit-HDFS-Build/27978/artifact/out/patch-unit-hadoop-hdfs-project_hadoop-hdfs.txt
 |

[jira] [Work logged] (HDDS-2140) Add robot test for GDPR feature

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2140?focusedWorklogId=319972=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-319972
 ]

ASF GitHub Bot logged work on HDDS-2140:


Author: ASF GitHub Bot
Created on: 28/Sep/19 06:49
Start Date: 28/Sep/19 06:49
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on issue #1542: HDDS-2140. Add 
robot test for GDPR feature
URL: https://github.com/apache/hadoop/pull/1542#issuecomment-536159862
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |::|--:|:|:|
   | 0 | reexec | 1727 | Docker mode activated. |
   ||| _ Prechecks _ |
   | +1 | dupname | 0 | No case conflicting files found. |
   | +1 | @author | 0 | The patch does not contain any @author tags. |
   | -1 | test4tests | 0 | The patch doesn't appear to include any new or 
modified tests.  Please justify why no new tests are needed for this patch. 
Also please list what manual steps were performed to verify this patch. |
   ||| _ trunk Compile Tests _ |
   | -1 | mvninstall | 30 | hadoop-hdds in trunk failed. |
   | -1 | mvninstall | 38 | hadoop-ozone in trunk failed. |
   | -1 | compile | 18 | hadoop-hdds in trunk failed. |
   | -1 | compile | 13 | hadoop-ozone in trunk failed. |
   | +1 | mvnsite | 0 | trunk passed |
   | +1 | shadedclient | 949 | branch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 18 | hadoop-hdds in trunk failed. |
   | -1 | javadoc | 16 | hadoop-ozone in trunk failed. |
   ||| _ Patch Compile Tests _ |
   | -1 | mvninstall | 30 | hadoop-hdds in the patch failed. |
   | -1 | mvninstall | 34 | hadoop-ozone in the patch failed. |
   | -1 | compile | 20 | hadoop-hdds in the patch failed. |
   | -1 | compile | 15 | hadoop-ozone in the patch failed. |
   | -1 | javac | 20 | hadoop-hdds in the patch failed. |
   | -1 | javac | 15 | hadoop-ozone in the patch failed. |
   | +1 | mvnsite | 0 | the patch passed |
   | +1 | whitespace | 0 | The patch has no whitespace issues. |
   | +1 | shadedclient | 777 | patch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 18 | hadoop-hdds in the patch failed. |
   | -1 | javadoc | 16 | hadoop-ozone in the patch failed. |
   ||| _ Other Tests _ |
   | -1 | unit | 23 | hadoop-hdds in the patch failed. |
   | -1 | unit | 22 | hadoop-ozone in the patch failed. |
   | +1 | asflicense | 29 | The patch does not generate ASF License warnings. |
   | | | 3845 | |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | Client=19.03.1 Server=19.03.1 base: 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1542 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient |
   | uname | Linux c181086e2738 4.15.0-60-generic #67-Ubuntu SMP Thu Aug 22 
16:55:30 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 14b4fbc |
   | Default Java | 1.8.0_222 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/branch-mvninstall-hadoop-hdds.txt
 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/branch-mvninstall-hadoop-ozone.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/branch-compile-hadoop-hdds.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/branch-compile-hadoop-ozone.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/branch-javadoc-hadoop-hdds.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/branch-javadoc-hadoop-ozone.txt
 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/patch-mvninstall-hadoop-hdds.txt
 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/patch-mvninstall-hadoop-ozone.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/patch-compile-hadoop-hdds.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/patch-compile-hadoop-ozone.txt
 |
   | javac | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/patch-compile-hadoop-hdds.txt
 |
   | javac | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/patch-compile-hadoop-ozone.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1542/1/artifact/out/patch-javadoc-hadoop-hdds.txt
 |
   | javadoc | 

[jira] [Commented] (HDDS-2175) Propagate System Exceptions from the OzoneManager

2019-09-28 Thread Arpit Agarwal (Jira)


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

Arpit Agarwal commented on HDDS-2175:
-

I feel that call stacks are invaluable when included in the bug report to the 
developer.

> Propagate System Exceptions from the OzoneManager
> -
>
> Key: HDDS-2175
> URL: https://issues.apache.org/jira/browse/HDDS-2175
> Project: Hadoop Distributed Data Store
>  Issue Type: Improvement
>  Components: Ozone Manager
>Reporter: Supratim Deka
>Assignee: Supratim Deka
>Priority: Major
>  Labels: pull-request-available
>  Time Spent: 50m
>  Remaining Estimate: 0h
>
> Exceptions encountered while processing requests on the OM are categorized as 
> business exceptions and system exceptions. All of the business exceptions are 
> captured as OMException and have an associated status code which is returned 
> to the client. The handling of these is not going to be changed.
> Currently system exceptions are returned as INTERNAL ERROR to the client with 
> a 1 line message string from the exception. The scope of this jira is to 
> capture system exceptions and propagate the related information(including the 
> complete stack trace) back to the client.
> There are 3 sub-tasks required to achieve this
> 1. Separate capture and handling for OMException and the other 
> exceptions(IOException). For system exceptions, use Hadoop IPC 
> ServiceException mechanism to send the stack trace to the client.
> 2. track and propagate exceptions inside Ratis OzoneManagerStateMachine and 
> propagate up to the OzoneManager layer (on the leader). Currently, these 
> exceptions are not being tracked.
> 3. Handle and propagate exceptions from Ratis.
> Will raise jira for each sub-task.
>   



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

-
To unsubscribe, e-mail: hdfs-issues-unsubscr...@hadoop.apache.org
For additional commands, e-mail: hdfs-issues-h...@hadoop.apache.org



[jira] [Work logged] (HDDS-2202) Remove unused import in OmUtils

2019-09-28 Thread ASF GitHub Bot (Jira)


 [ 
https://issues.apache.org/jira/browse/HDDS-2202?focusedWorklogId=319967=com.atlassian.jira.plugin.system.issuetabpanels:worklog-tabpanel#worklog-319967
 ]

ASF GitHub Bot logged work on HDDS-2202:


Author: ASF GitHub Bot
Created on: 28/Sep/19 06:28
Start Date: 28/Sep/19 06:28
Worklog Time Spent: 10m 
  Work Description: hadoop-yetus commented on issue #1543: HDDS-2202. 
Remove unused import in OmUtils
URL: https://github.com/apache/hadoop/pull/1543#issuecomment-536158743
 
 
   :broken_heart: **-1 overall**
   
   
   
   
   
   
   | Vote | Subsystem | Runtime | Comment |
   |::|--:|:|:|
   | 0 | reexec | 78 | Docker mode activated. |
   ||| _ Prechecks _ |
   | +1 | dupname | 0 | No case conflicting files found. |
   | +1 | @author | 0 | The patch does not contain any @author tags. |
   | -1 | test4tests | 0 | The patch doesn't appear to include any new or 
modified tests.  Please justify why no new tests are needed for this patch. 
Also please list what manual steps were performed to verify this patch. |
   ||| _ trunk Compile Tests _ |
   | -1 | mvninstall | 44 | hadoop-hdds in trunk failed. |
   | -1 | mvninstall | 40 | hadoop-ozone in trunk failed. |
   | -1 | compile | 19 | hadoop-hdds in trunk failed. |
   | -1 | compile | 13 | hadoop-ozone in trunk failed. |
   | +1 | checkstyle | 59 | trunk passed |
   | +1 | mvnsite | 0 | trunk passed |
   | +1 | shadedclient | 940 | branch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 20 | hadoop-hdds in trunk failed. |
   | -1 | javadoc | 16 | hadoop-ozone in trunk failed. |
   | 0 | spotbugs | 1028 | Used deprecated FindBugs config; considering 
switching to SpotBugs. |
   | -1 | findbugs | 31 | hadoop-hdds in trunk failed. |
   | -1 | findbugs | 17 | hadoop-ozone in trunk failed. |
   ||| _ Patch Compile Tests _ |
   | -1 | mvninstall | 32 | hadoop-hdds in the patch failed. |
   | -1 | mvninstall | 35 | hadoop-ozone in the patch failed. |
   | -1 | compile | 21 | hadoop-hdds in the patch failed. |
   | -1 | compile | 17 | hadoop-ozone in the patch failed. |
   | -1 | javac | 21 | hadoop-hdds in the patch failed. |
   | -1 | javac | 17 | hadoop-ozone in the patch failed. |
   | +1 | checkstyle | 28 | The patch passed checkstyle in hadoop-hdds |
   | +1 | checkstyle | 33 | hadoop-ozone: The patch generated 0 new + 0 
unchanged - 1 fixed = 0 total (was 1) |
   | +1 | mvnsite | 0 | the patch passed |
   | +1 | whitespace | 0 | The patch has no whitespace issues. |
   | +1 | shadedclient | 823 | patch has no errors when building and testing 
our client artifacts. |
   | -1 | javadoc | 21 | hadoop-hdds in the patch failed. |
   | -1 | javadoc | 17 | hadoop-ozone in the patch failed. |
   | -1 | findbugs | 30 | hadoop-hdds in the patch failed. |
   | -1 | findbugs | 19 | hadoop-ozone in the patch failed. |
   ||| _ Other Tests _ |
   | -1 | unit | 30 | hadoop-hdds in the patch failed. |
   | -1 | unit | 24 | hadoop-ozone in the patch failed. |
   | +1 | asflicense | 30 | The patch does not generate ASF License warnings. |
   | | | 2558 | |
   
   
   | Subsystem | Report/Notes |
   |--:|:-|
   | Docker | Client=19.03.0 Server=19.03.0 base: 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/Dockerfile
 |
   | GITHUB PR | https://github.com/apache/hadoop/pull/1543 |
   | Optional Tests | dupname asflicense compile javac javadoc mvninstall 
mvnsite unit shadedclient findbugs checkstyle |
   | uname | Linux eb0b87def2e8 4.15.0-58-generic #64-Ubuntu SMP Tue Aug 6 
11:12:41 UTC 2019 x86_64 x86_64 x86_64 GNU/Linux |
   | Build tool | maven |
   | Personality | personality/hadoop.sh |
   | git revision | trunk / 14b4fbc |
   | Default Java | 1.8.0_222 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-mvninstall-hadoop-hdds.txt
 |
   | mvninstall | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-mvninstall-hadoop-ozone.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-compile-hadoop-hdds.txt
 |
   | compile | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-compile-hadoop-ozone.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-javadoc-hadoop-hdds.txt
 |
   | javadoc | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-javadoc-hadoop-ozone.txt
 |
   | findbugs | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-findbugs-hadoop-hdds.txt
 |
   | findbugs | 
https://builds.apache.org/job/hadoop-multibranch/job/PR-1543/1/artifact/out/branch-findbugs-hadoop-ozone.txt
 |
   | mvninstall |