[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-22 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r599281959



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithValidator.java
##
@@ -0,0 +1,115 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ * http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+package org.apache.hadoop.hdfs;
+
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
+import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics;
+import org.junit.Assert;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This test extends {@link TestReconstructStripedFile} to test
+ * ec reconstruction validation.
+ */
+public class TestReconstructStripedFileWithValidator
+extends TestReconstructStripedFile {
+  private static final Logger LOG =
+  LoggerFactory.getLogger(TestReconstructStripedFileWithValidator.class);
+
+  public TestReconstructStripedFileWithValidator() {
+LOG.info("run {} with validator.",
+TestReconstructStripedFileWithValidator.class.getSuperclass()
+.getSimpleName());
+  }
+
+  /**
+   * This test injects data pollution into decoded outputs once.
+   * When validation enabled, the first reconstruction task should fail
+   * in the validation, but the data will be recovered correctly
+   * by the next task.
+   * On the other hand, when validation disabled, the first reconstruction task
+   * will succeed and then lead to data corruption.
+   */
+  @Test(timeout = 12)
+  public void testValidatorWithBadDecoding()
+  throws Exception {
+MiniDFSCluster cluster = getCluster();
+
+cluster.getDataNodes().stream()
+.map(DataNode::getMetrics)
+.map(DataNodeMetrics::getECInvalidReconstructionTasks)
+.forEach(n -> Assert.assertEquals(0, (long) n));
+
+DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
+DataNodeFaultInjector badDecodingInjector = new DataNodeFaultInjector() {
+  private final AtomicBoolean flag = new AtomicBoolean(false);
+
+  @Override
+  public void badDecoding(ByteBuffer[] outputs) {
+if (!flag.get()) {
+  for (ByteBuffer output : outputs) {
+output.mark();
+output.put((byte) (output.get(output.position()) + 1));
+output.reset();
+  }
+}
+flag.set(true);
+  }
+};
+DataNodeFaultInjector.set(badDecodingInjector);
+
+int fileLen =
+(getEcPolicy().getNumDataUnits() + getEcPolicy().getNumParityUnits())
+* getBlockSize() + getBlockSize() / 10;
+try {
+  assertFileBlocksReconstruction(
+  "/testValidatorWithBadDecoding",
+  fileLen,
+  ReconstructionType.DataOnly,
+  getEcPolicy().getNumParityUnits());
+
+  long count = cluster.getDataNodes().stream()
+  .map(DataNode::getMetrics)
+  .map(DataNodeMetrics::getECInvalidReconstructionTasks)
+  .filter(n -> n == 1).count();

Review comment:
   I think we should use `.sum()` instead of `.filter(n -> n == 1).count();`




-- 
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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-18 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r597351035



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithValidator.java
##
@@ -0,0 +1,98 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This test extends {@link TestReconstructStripedFile} to test
+ * ec reconstruction validation.
+ */
+public class TestReconstructStripedFileWithValidator
+extends TestReconstructStripedFile {
+  private static final Logger LOG =
+  LoggerFactory.getLogger(TestReconstructStripedFileWithValidator.class);
+
+  public TestReconstructStripedFileWithValidator() {
+LOG.info("run {} with validator.",
+TestReconstructStripedFileWithValidator.class.getSuperclass()
+.getSimpleName());
+  }
+
+  /**
+   * This test injects data pollution into decoded outputs once.
+   * When validation enabled, the first reconstruction task should fail
+   * in the validation, but the data will be recovered correctly
+   * by the next task.
+   * On the other hand, when validation disabled, the first reconstruction task
+   * will succeed and then lead to data corruption.
+   */
+  @Test(timeout = 12)
+  public void testValidatorWithBadDecoding()
+  throws Exception {
+DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
+DataNodeFaultInjector badDecodingInjector = new DataNodeFaultInjector() {
+  private final AtomicBoolean flag = new AtomicBoolean(false);
+
+  @Override
+  public void badDecoding(ByteBuffer[] outputs) {
+if (!flag.get()) {
+  for (ByteBuffer output : outputs) {
+output.mark();
+output.put((byte) (output.get(output.position()) + 1));
+output.reset();
+  }
+}
+flag.set(true);
+  }
+};
+DataNodeFaultInjector.set(badDecodingInjector);
+int fileLen =
+(getEcPolicy().getNumDataUnits() + getEcPolicy().getNumParityUnits())
+* getBlockSize() + getBlockSize() / 10;
+try {
+  assertFileBlocksReconstruction(
+  "/testValidatorWithBadDecoding",
+  fileLen,
+  ReconstructionType.DataOnly,
+  getEcPolicy().getNumParityUnits());

Review comment:
   +1 too




-- 
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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-18 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r597350941



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
##
@@ -126,12 +128,26 @@ private void reconstructTargets(int toReconstructLen) 
throws IOException {
 int[] erasedIndices = stripedWriter.getRealTargetIndices();
 ByteBuffer[] outputs = 
stripedWriter.getRealTargetBuffers(toReconstructLen);
 
+if (isValidationEnabled()) {
+  markBuffers(inputs);
+  decode(inputs, erasedIndices, outputs);
+  resetBuffers(inputs);
+
+  DataNodeFaultInjector.get().badDecoding(outputs);
+  getValidator().validate(inputs, erasedIndices, outputs);

Review comment:
   +1,I have no other suggestion.




-- 
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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595174859



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithValidator.java
##
@@ -0,0 +1,98 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This test extends {@link TestReconstructStripedFile} to test
+ * ec reconstruction validation.
+ */
+public class TestReconstructStripedFileWithValidator
+extends TestReconstructStripedFile {
+  private static final Logger LOG =
+  LoggerFactory.getLogger(TestReconstructStripedFileWithValidator.class);
+
+  public TestReconstructStripedFileWithValidator() {
+LOG.info("run {} with validator.",
+TestReconstructStripedFileWithValidator.class.getSuperclass()
+.getSimpleName());
+  }
+
+  /**
+   * This test injects data pollution into decoded outputs once.
+   * When validation enabled, the first reconstruction task should fail
+   * in the validation, but the data will be recovered correctly
+   * by the next task.
+   * On the other hand, when validation disabled, the first reconstruction task
+   * will succeed and then lead to data corruption.
+   */
+  @Test(timeout = 12)
+  public void testValidatorWithBadDecoding()
+  throws Exception {
+DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
+DataNodeFaultInjector badDecodingInjector = new DataNodeFaultInjector() {
+  private final AtomicBoolean flag = new AtomicBoolean(false);
+
+  @Override
+  public void badDecoding(ByteBuffer[] outputs) {
+if (!flag.get()) {
+  for (ByteBuffer output : outputs) {
+output.mark();
+output.put((byte) (output.get(output.position()) + 1));

Review comment:
   `output.position()) + 1` may cause IndexOutOfBoundsException





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595174859



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithValidator.java
##
@@ -0,0 +1,98 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This test extends {@link TestReconstructStripedFile} to test
+ * ec reconstruction validation.
+ */
+public class TestReconstructStripedFileWithValidator
+extends TestReconstructStripedFile {
+  private static final Logger LOG =
+  LoggerFactory.getLogger(TestReconstructStripedFileWithValidator.class);
+
+  public TestReconstructStripedFileWithValidator() {
+LOG.info("run {} with validator.",
+TestReconstructStripedFileWithValidator.class.getSuperclass()
+.getSimpleName());
+  }
+
+  /**
+   * This test injects data pollution into decoded outputs once.
+   * When validation enabled, the first reconstruction task should fail
+   * in the validation, but the data will be recovered correctly
+   * by the next task.
+   * On the other hand, when validation disabled, the first reconstruction task
+   * will succeed and then lead to data corruption.
+   */
+  @Test(timeout = 12)
+  public void testValidatorWithBadDecoding()
+  throws Exception {
+DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
+DataNodeFaultInjector badDecodingInjector = new DataNodeFaultInjector() {
+  private final AtomicBoolean flag = new AtomicBoolean(false);
+
+  @Override
+  public void badDecoding(ByteBuffer[] outputs) {
+if (!flag.get()) {
+  for (ByteBuffer output : outputs) {
+output.mark();
+output.put((byte) (output.get(output.position()) + 1));

Review comment:
   `output.position()) + 1` may cause IndexOutOfBoundsException





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595171264



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFileWithValidator.java
##
@@ -0,0 +1,98 @@
+/**
+ * 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.hdfs;
+
+import org.apache.hadoop.hdfs.server.datanode.DataNodeFaultInjector;
+import org.junit.Test;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+import java.nio.ByteBuffer;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+/**
+ * This test extends {@link TestReconstructStripedFile} to test
+ * ec reconstruction validation.
+ */
+public class TestReconstructStripedFileWithValidator
+extends TestReconstructStripedFile {
+  private static final Logger LOG =
+  LoggerFactory.getLogger(TestReconstructStripedFileWithValidator.class);
+
+  public TestReconstructStripedFileWithValidator() {
+LOG.info("run {} with validator.",
+TestReconstructStripedFileWithValidator.class.getSuperclass()
+.getSimpleName());
+  }
+
+  /**
+   * This test injects data pollution into decoded outputs once.
+   * When validation enabled, the first reconstruction task should fail
+   * in the validation, but the data will be recovered correctly
+   * by the next task.
+   * On the other hand, when validation disabled, the first reconstruction task
+   * will succeed and then lead to data corruption.
+   */
+  @Test(timeout = 12)
+  public void testValidatorWithBadDecoding()
+  throws Exception {
+DataNodeFaultInjector oldInjector = DataNodeFaultInjector.get();
+DataNodeFaultInjector badDecodingInjector = new DataNodeFaultInjector() {
+  private final AtomicBoolean flag = new AtomicBoolean(false);
+
+  @Override
+  public void badDecoding(ByteBuffer[] outputs) {
+if (!flag.get()) {
+  for (ByteBuffer output : outputs) {
+output.mark();
+output.put((byte) (output.get(output.position()) + 1));
+output.reset();
+  }
+}
+flag.set(true);
+  }
+};
+DataNodeFaultInjector.set(badDecodingInjector);
+int fileLen =
+(getEcPolicy().getNumDataUnits() + getEcPolicy().getNumParityUnits())
+* getBlockSize() + getBlockSize() / 10;
+try {
+  assertFileBlocksReconstruction(
+  "/testValidatorWithBadDecoding",
+  fileLen,
+  ReconstructionType.DataOnly,
+  getEcPolicy().getNumParityUnits());

Review comment:
   maybe add an assert here to indicate there is an InvalidDecodingException





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595161889



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReconstructStripedFile.java
##
@@ -84,6 +84,8 @@
   private int groupSize;
   private int dnNum;
 
+  private boolean isValidationEnabled;

Review comment:
   used in subclass to enable the validation





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595156529



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
##
@@ -126,12 +128,26 @@ private void reconstructTargets(int toReconstructLen) 
throws IOException {
 int[] erasedIndices = stripedWriter.getRealTargetIndices();
 ByteBuffer[] outputs = 
stripedWriter.getRealTargetBuffers(toReconstructLen);
 
+if (isValidationEnabled()) {
+  markBuffers(inputs);
+  decode(inputs, erasedIndices, outputs);
+  resetBuffers(inputs);
+
+  DataNodeFaultInjector.get().badDecoding(outputs);
+  getValidator().validate(inputs, erasedIndices, outputs);

Review comment:
   How about handle the InvalidDecodingException separately(currently 
handle it as a Throwable in run() method)? I think this situation need more 
attention.





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595156529



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockReconstructor.java
##
@@ -126,12 +128,26 @@ private void reconstructTargets(int toReconstructLen) 
throws IOException {
 int[] erasedIndices = stripedWriter.getRealTargetIndices();
 ByteBuffer[] outputs = 
stripedWriter.getRealTargetBuffers(toReconstructLen);
 
+if (isValidationEnabled()) {
+  markBuffers(inputs);
+  decode(inputs, erasedIndices, outputs);
+  resetBuffers(inputs);
+
+  DataNodeFaultInjector.get().badDecoding(outputs);
+  getValidator().validate(inputs, erasedIndices, outputs);

Review comment:
   How about handle the InvalidDecodingException separately? I think this 
situation need more attention.





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595018555



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockChecksumReconstructor.java
##
@@ -192,7 +193,16 @@ private void reconstructTargets(int toReconstructLen) 
throws IOException {
 for (int i = 0; i < targetIndices.length; i++) {
   tarIndices[i] = targetIndices[i];
 }
-getDecoder().decode(inputs, tarIndices, outputs);
+
+if (isValidationEnabled()) {
+  markBuffers(inputs);
+  getDecoder().decode(inputs, tarIndices, outputs);
+  resetBuffers(inputs);
+
+  getValidator().validate(inputs, tarIndices, outputs);

Review comment:
   Can we check the result and add a metric for the failed validation?





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595018555



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockChecksumReconstructor.java
##
@@ -192,7 +193,16 @@ private void reconstructTargets(int toReconstructLen) 
throws IOException {
 for (int i = 0; i < targetIndices.length; i++) {
   tarIndices[i] = targetIndices[i];
 }
-getDecoder().decode(inputs, tarIndices, outputs);
+
+if (isValidationEnabled()) {
+  markBuffers(inputs);
+  getDecoder().decode(inputs, tarIndices, outputs);
+  resetBuffers(inputs);
+
+  getValidator().validate(inputs, tarIndices, outputs);

Review comment:
   Can we check the result and add a metric for the failed validation?





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



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



[GitHub] [hadoop] runitao commented on a change in pull request #2585: HDFS-15759. EC: Verify EC reconstruction correctness on DataNode

2021-03-16 Thread GitBox


runitao commented on a change in pull request #2585:
URL: https://github.com/apache/hadoop/pull/2585#discussion_r595018555



##
File path: 
hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/StripedBlockChecksumReconstructor.java
##
@@ -192,7 +193,16 @@ private void reconstructTargets(int toReconstructLen) 
throws IOException {
 for (int i = 0; i < targetIndices.length; i++) {
   tarIndices[i] = targetIndices[i];
 }
-getDecoder().decode(inputs, tarIndices, outputs);
+
+if (isValidationEnabled()) {
+  markBuffers(inputs);
+  getDecoder().decode(inputs, tarIndices, outputs);
+  resetBuffers(inputs);
+
+  getValidator().validate(inputs, tarIndices, outputs);

Review comment:
   Can we check the result and add a metric for failed check?





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



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