hadoop-ozone-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From adorosz...@apache.org
Subject [hadoop-ozone] branch master updated: HDDS-2898. build failure due to hadoop-hdds-client test (#463)
Date Mon, 20 Jan 2020 14:43:52 GMT
This is an automated email from the ASF dual-hosted git repository.

adoroszlai pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hadoop-ozone.git


The following commit(s) were added to refs/heads/master by this push:
     new c29936a  HDDS-2898. build failure due to hadoop-hdds-client test (#463)
c29936a is described below

commit c29936adc07c809bb6db591b6bb893e917c472c5
Author: Elek, Márton <elek@users.noreply.github.com>
AuthorDate: Mon Jan 20 15:43:42 2020 +0100

    HDDS-2898. build failure due to hadoop-hdds-client test (#463)
---
 .../hdds/scm/storage/DummyBlockInputStream.java    |  92 ++++++++++++++++++
 .../storage/DummyBlockInputStreamWithRetry.java    |  78 +++++++++++++++
 .../hdds/scm/storage/DummyChunkInputStream.java    |  66 +++++++++++++
 .../hdds/scm/storage/TestBlockInputStream.java     | 106 ++-------------------
 .../hdds/scm/storage/TestChunkInputStream.java     |  52 +---------
 5 files changed, 248 insertions(+), 146 deletions(-)

diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java
b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java
new file mode 100644
index 0000000..5db722a
--- /dev/null
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStream.java
@@ -0,0 +1,92 @@
+/*
+ * 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.hdds.scm.storage;
+
+import java.io.IOException;
+import java.util.List;
+import java.util.Map;
+import java.util.function.Function;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * A dummy BlockInputStream to mock read block call to DN.
+ */
+class DummyBlockInputStream extends BlockInputStream {
+
+  private List<ChunkInfo> chunks;
+
+  private Map<String, byte[]> chunkDataMap;
+
+  @SuppressWarnings("parameternumber")
+  DummyBlockInputStream(
+      BlockID blockId,
+      long blockLen,
+      Pipeline pipeline,
+      Token<OzoneBlockTokenIdentifier> token,
+      boolean verifyChecksum,
+      XceiverClientManager xceiverClientManager,
+      List<ChunkInfo> chunkList,
+      Map<String, byte[]> chunkMap) {
+    super(blockId, blockLen, pipeline, token, verifyChecksum,
+        xceiverClientManager);
+    this.chunks = chunkList;
+    this.chunkDataMap = chunkMap;
+  }
+
+  @SuppressWarnings("parameternumber")
+  DummyBlockInputStream(
+      BlockID blockId,
+      long blockLen,
+      Pipeline pipeline,
+      Token<OzoneBlockTokenIdentifier> token,
+      boolean verifyChecksum,
+      XceiverClientManager xceiverClientManager,
+      Function<BlockID, Pipeline> refreshFunction,
+      List<ChunkInfo> chunkList,
+      Map<String, byte[]> chunks) {
+    super(blockId, blockLen, pipeline, token, verifyChecksum,
+        xceiverClientManager, refreshFunction);
+    this.chunkDataMap = chunks;
+    this.chunks = chunkList;
+
+  }
+
+  @Override
+  protected List<ChunkInfo> getChunkInfos() throws IOException {
+    return chunks;
+  }
+
+  @Override
+  protected void addStream(ChunkInfo chunkInfo) {
+    TestChunkInputStream testChunkInputStream = new TestChunkInputStream();
+    getChunkStreams().add(new DummyChunkInputStream(testChunkInputStream,
+        chunkInfo, null, null, false,
+        chunkDataMap.get(chunkInfo.getChunkName()).clone()));
+  }
+
+  @Override
+  protected synchronized void checkOpen() throws IOException {
+    // No action needed
+  }
+}
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java
b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java
new file mode 100644
index 0000000..1686ed4
--- /dev/null
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyBlockInputStreamWithRetry.java
@@ -0,0 +1,78 @@
+/*
+ * 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.hdds.scm.storage;
+
+import java.io.IOException;
+import java.util.Collections;
+import java.util.List;
+import java.util.Map;
+import java.util.concurrent.atomic.AtomicBoolean;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
+import org.apache.hadoop.hdds.scm.XceiverClientManager;
+import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
+import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
+import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
+import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
+import org.apache.hadoop.security.token.Token;
+
+/**
+ * A dummy BlockInputStream with pipeline refresh function to mock read
+ * block call to DN.
+ */
+final class DummyBlockInputStreamWithRetry
+    extends DummyBlockInputStream {
+
+  private int getChunkInfoCount = 0;
+
+  @SuppressWarnings("parameternumber")
+  DummyBlockInputStreamWithRetry(
+      BlockID blockId,
+      long blockLen,
+      Pipeline pipeline,
+      Token<OzoneBlockTokenIdentifier> token,
+      boolean verifyChecksum,
+      XceiverClientManager xceiverClientManager,
+      List<ChunkInfo> chunkList,
+      Map<String, byte[]> chunkMap,
+      AtomicBoolean isRerfreshed) {
+    super(blockId, blockLen, pipeline, token, verifyChecksum,
+        xceiverClientManager, blockID -> {
+          isRerfreshed.set(true);
+          return Pipeline.newBuilder()
+              .setState(Pipeline.PipelineState.OPEN)
+              .setId(PipelineID.randomId())
+              .setType(HddsProtos.ReplicationType.STAND_ALONE)
+              .setFactor(HddsProtos.ReplicationFactor.ONE)
+              .setNodes(Collections.emptyList())
+              .build();
+        }, chunkList, chunkMap);
+  }
+
+  @Override
+  protected List<ChunkInfo> getChunkInfos() throws IOException {
+    if (getChunkInfoCount == 0) {
+      getChunkInfoCount++;
+      throw new ContainerNotFoundException("Exception encountered");
+    } else {
+      return super.getChunkInfos();
+    }
+  }
+}
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyChunkInputStream.java
b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyChunkInputStream.java
new file mode 100644
index 0000000..8405f43
--- /dev/null
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/DummyChunkInputStream.java
@@ -0,0 +1,66 @@
+/*
+ * 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.hdds.scm.storage;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.hdds.client.BlockID;
+import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
+import org.apache.hadoop.hdds.scm.XceiverClientSpi;
+
+import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+
+/**
+ * A dummy ChunkInputStream to mock read chunk calls to DN.
+ */
+public class DummyChunkInputStream extends ChunkInputStream {
+
+  private byte[] chunkData;
+
+  // Stores the read chunk data in each readChunk call
+  private List<ByteString> readByteBuffers = new ArrayList<>();
+
+  public DummyChunkInputStream(TestChunkInputStream testChunkInputStream,
+      ChunkInfo chunkInfo,
+      BlockID blockId,
+      XceiverClientSpi xceiverClient,
+      boolean verifyChecksum,
+      byte[] data) {
+    super(chunkInfo, blockId, xceiverClient, verifyChecksum);
+    this.chunkData = data;
+  }
+
+  @Override
+  protected ByteString readChunk(ChunkInfo readChunkInfo) {
+    ByteString byteString = ByteString.copyFrom(chunkData,
+        (int) readChunkInfo.getOffset(),
+        (int) readChunkInfo.getLen());
+    getReadByteBuffers().add(byteString);
+    return byteString;
+  }
+
+  @Override
+  protected void checkOpen() {
+    // No action needed
+  }
+
+  public List<ByteString> getReadByteBuffers() {
+    return readByteBuffers;
+  }
+}
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java
b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java
index 6c47ef6..3f5e12a 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestBlockInputStream.java
@@ -23,27 +23,19 @@ import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.client.ContainerBlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
-import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
-import org.apache.hadoop.hdds.scm.XceiverClientManager;
-import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
-import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
-import org.apache.hadoop.hdds.scm.pipeline.PipelineID;
-import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
 import org.apache.hadoop.ozone.common.Checksum;
-import org.apache.hadoop.security.token.Token;
+
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.EOFException;
-import java.io.IOException;
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.HashMap;
 import java.util.List;
 import java.util.Map;
 import java.util.Random;
-import java.util.function.Function;
+import java.util.concurrent.atomic.AtomicBoolean;
 
 import static org.apache.hadoop.hdds.scm.storage.TestChunkInputStream.generateRandomData;
 
@@ -60,7 +52,7 @@ public class TestBlockInputStream {
   private int blockSize;
   private List<ChunkInfo> chunks;
   private Map<String, byte[]> chunkDataMap;
-  private boolean refreshFunctionFlag = false;
+  private AtomicBoolean isRefreshed = new AtomicBoolean();
 
   @Before
   public void setup() throws Exception {
@@ -69,7 +61,7 @@ public class TestBlockInputStream {
     createChunkList(5);
 
     blockStream = new DummyBlockInputStream(blockID, blockSize, null, null,
-        false, null);
+        false, null, chunks, chunkDataMap);
   }
 
   /**
@@ -109,51 +101,6 @@ public class TestBlockInputStream {
     }
   }
 
-  /**
-   * A dummy BlockInputStream to mock read block call to DN.
-   */
-  private class DummyBlockInputStream extends BlockInputStream {
-
-    DummyBlockInputStream(BlockID blockId,
-        long blockLen,
-        Pipeline pipeline,
-        Token<OzoneBlockTokenIdentifier> token,
-        boolean verifyChecksum,
-        XceiverClientManager xceiverClientManager) {
-      super(blockId, blockLen, pipeline, token, verifyChecksum,
-          xceiverClientManager);
-    }
-
-    DummyBlockInputStream(BlockID blockId,
-                          long blockLen,
-                          Pipeline pipeline,
-                          Token<OzoneBlockTokenIdentifier> token,
-                          boolean verifyChecksum,
-                          XceiverClientManager xceiverClientManager,
-                          Function<BlockID, Pipeline> refreshFunction) {
-      super(blockId, blockLen, pipeline, token, verifyChecksum,
-          xceiverClientManager, refreshFunction);
-    }
-
-    @Override
-    protected List<ChunkInfo> getChunkInfos() throws IOException {
-      return chunks;
-    }
-
-    @Override
-    protected void addStream(ChunkInfo chunkInfo) {
-      TestChunkInputStream testChunkInputStream = new TestChunkInputStream();
-      getChunkStreams().add(testChunkInputStream.new DummyChunkInputStream(
-          chunkInfo, null, null, false,
-          chunkDataMap.get(chunkInfo.getChunkName()).clone()));
-    }
-
-    @Override
-    protected synchronized void checkOpen() throws IOException {
-      // No action needed
-    }
-  }
-
   private void seekAndVerify(int pos) throws Exception {
     blockStream.seek(pos);
     Assert.assertEquals("Current position of buffer does not match with the " +
@@ -249,57 +196,18 @@ public class TestBlockInputStream {
     matchWithInputData(b2, 150, 100);
   }
 
-  /**
-   * A dummy BlockInputStream with pipeline refresh function to mock read
-   * block call to DN.
-   */
-  private final class DummyBlockInputStreamWithRetry
-      extends DummyBlockInputStream {
-
-    private int getChunkInfoCount = 0;
-
-    private DummyBlockInputStreamWithRetry(BlockID blockId,
-                                   long blockLen,
-                                   Pipeline pipeline,
-                                   Token<OzoneBlockTokenIdentifier> token,
-                                   boolean verifyChecksum,
-                                   XceiverClientManager xceiverClientManager) {
-      super(blockId, blockLen, pipeline, token, verifyChecksum,
-          xceiverClientManager, blockID -> {
-            refreshFunctionFlag = true;
-            return Pipeline.newBuilder()
-                .setState(Pipeline.PipelineState.OPEN)
-                .setId(PipelineID.randomId())
-                .setType(HddsProtos.ReplicationType.STAND_ALONE)
-                .setFactor(HddsProtos.ReplicationFactor.ONE)
-                .setNodes(Collections.emptyList())
-                .build();
-          });
-    }
-
-    @Override
-    protected List<ChunkInfo> getChunkInfos() throws IOException {
-      if (getChunkInfoCount == 0) {
-        getChunkInfoCount++;
-        throw new ContainerNotFoundException("Exception encountered");
-      } else {
-        return super.getChunkInfos();
-      }
-    }
-  }
-
   @Test
   public void testRefreshPipelineFunction() throws Exception {
     BlockID blockID = new BlockID(new ContainerBlockID(1, 1));
     createChunkList(5);
     BlockInputStream blockInputStreamWithRetry =
         new DummyBlockInputStreamWithRetry(blockID, blockSize, null, null,
-        false, null);
+            false, null, chunks, chunkDataMap, isRefreshed);
 
-    Assert.assertFalse(refreshFunctionFlag);
+    Assert.assertFalse(isRefreshed.get());
     seekAndVerify(50);
     byte[] b = new byte[200];
     blockInputStreamWithRetry.read(b, 0, 200);
-    Assert.assertTrue(refreshFunctionFlag);
+    Assert.assertTrue(isRefreshed.get());
   }
 }
diff --git a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestChunkInputStream.java
b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestChunkInputStream.java
index a5fe26b..94ec157 100644
--- a/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestChunkInputStream.java
+++ b/hadoop-hdds/client/src/test/java/org/apache/hadoop/hdds/scm/storage/TestChunkInputStream.java
@@ -18,21 +18,17 @@
 
 package org.apache.hadoop.hdds.scm.storage;
 
-import org.apache.hadoop.hdds.client.BlockID;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
 import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
-import org.apache.hadoop.hdds.scm.XceiverClientSpi;
 import org.apache.hadoop.ozone.OzoneConfigKeys;
 import org.apache.hadoop.ozone.common.Checksum;
 import org.apache.hadoop.test.GenericTestUtils;
-import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
+
 import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.EOFException;
-import java.util.ArrayList;
-import java.util.List;
 import java.util.Random;
 
 /**
@@ -66,7 +62,8 @@ public class TestChunkInputStream {
             chunkData, 0, CHUNK_SIZE).getProtoBufMessage())
         .build();
 
-    chunkStream = new DummyChunkInputStream(chunkInfo, null, null, true);
+    chunkStream =
+        new DummyChunkInputStream(this, chunkInfo, null, null, true, chunkData);
   }
 
   static byte[] generateRandomData(int length) {
@@ -76,45 +73,6 @@ public class TestChunkInputStream {
   }
 
   /**
-   * A dummy ChunkInputStream to mock read chunk calls to DN.
-   */
-  public class DummyChunkInputStream extends ChunkInputStream {
-
-    // Stores the read chunk data in each readChunk call
-    private List<ByteString> readByteBuffers = new ArrayList<>();
-
-    DummyChunkInputStream(ChunkInfo chunkInfo,
-        BlockID blockId,
-        XceiverClientSpi xceiverClient,
-        boolean verifyChecksum) {
-      super(chunkInfo, blockId, xceiverClient, verifyChecksum);
-    }
-
-    public DummyChunkInputStream(ChunkInfo chunkInfo,
-        BlockID blockId,
-        XceiverClientSpi xceiverClient,
-        boolean verifyChecksum,
-        byte[] data) {
-      super(chunkInfo, blockId, xceiverClient, verifyChecksum);
-      chunkData = data;
-    }
-
-    @Override
-    protected ByteString readChunk(ChunkInfo readChunkInfo) {
-      ByteString byteString = ByteString.copyFrom(chunkData,
-          (int) readChunkInfo.getOffset(),
-          (int) readChunkInfo.getLen());
-      readByteBuffers.add(byteString);
-      return byteString;
-    }
-
-    @Override
-    protected void checkOpen() {
-      // No action needed
-    }
-  }
-
-  /**
    * Match readData with the chunkData byte-wise.
    * @param readData Data read through ChunkInputStream
    * @param inputDataStartIndex first index (inclusive) in chunkData to compare
@@ -159,7 +117,7 @@ public class TestChunkInputStream {
     // chunk from offset 0 to 60 as the checksum boundary is at every 20
     // bytes. Verify that 60 bytes of chunk data are read and stored in the
     // buffers.
-    matchWithInputData(chunkStream.readByteBuffers.get(0).toByteArray(),
+    matchWithInputData(chunkStream.getReadByteBuffers().get(0).toByteArray(),
         0, 60);
 
   }
@@ -187,7 +145,7 @@ public class TestChunkInputStream {
     byte[] b = new byte[30];
     chunkStream.read(b, 0, 30);
     matchWithInputData(b, 25, 30);
-    matchWithInputData(chunkStream.readByteBuffers.get(0).toByteArray(),
+    matchWithInputData(chunkStream.getReadByteBuffers().get(0).toByteArray(),
         20, 40);
 
     // After read, the position of the chunkStream is evaluated from the


---------------------------------------------------------------------
To unsubscribe, e-mail: ozone-commits-unsubscribe@hadoop.apache.org
For additional commands, e-mail: ozone-commits-help@hadoop.apache.org


Mime
View raw message