You are viewing a plain text version of this content. The canonical link for it is here.
Posted to common-commits@hadoop.apache.org by zh...@apache.org on 2015/04/13 20:01:28 UTC

[01/50] [abbrv] hadoop git commit: HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng

Repository: hadoop
Updated Branches:
  refs/heads/HDFS-7285 39799da0c -> 01af25fbe (forced update)


HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b70d5846
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b70d5846
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b70d5846

Branch: refs/heads/HDFS-7285
Commit: b70d5846ae30cd77e2ee432516efe1ba57cc6303
Parents: 43f6bd6
Author: drankye <dr...@gmail.com>
Authored: Thu Feb 12 21:12:44 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:28:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   4 +
 .../io/erasurecode/rawcoder/JRSRawDecoder.java  |  69 +++
 .../io/erasurecode/rawcoder/JRSRawEncoder.java  |  78 +++
 .../erasurecode/rawcoder/RawErasureCoder.java   |   2 +-
 .../erasurecode/rawcoder/util/GaloisField.java  | 497 +++++++++++++++++++
 .../io/erasurecode/rawcoder/util/RSUtil.java    |  22 +
 .../hadoop/io/erasurecode/TestCoderBase.java    |  28 +-
 .../erasurecode/rawcoder/TestJRSRawCoder.java   |  93 ++++
 .../erasurecode/rawcoder/TestRawCoderBase.java  |   5 +-
 .../erasurecode/rawcoder/TestXorRawCoder.java   |   1 -
 10 files changed, 786 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 9728f97..7bbacf7 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -8,3 +8,7 @@
 
     HADOOP-11541. Raw XOR coder
     ( Kai Zheng )
+
+    HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng
+    ( Kai Zheng )
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
new file mode 100644
index 0000000..dbb689e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
@@ -0,0 +1,69 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw erasure decoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible.
+ */
+public class JRSRawDecoder extends AbstractRawErasureDecoder {
+  // To describe and calculate the needed Vandermonde matrix
+  private int[] errSignature;
+  private int[] primitivePower;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
+    super.initialize(numDataUnits, numParityUnits, chunkSize);
+    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+
+    this.errSignature = new int[getNumParityUnits()];
+    this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
+        getNumParityUnits());
+  }
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+    }
+
+    int dataLen = inputs[0].remaining();
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
+        erasedIndexes.length, dataLen);
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
+                          byte[][] outputs) {
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+    }
+
+    int dataLen = inputs[0].length;
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
+        erasedIndexes.length, dataLen);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
new file mode 100644
index 0000000..6ea7551
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw erasure encoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible.
+ */
+public class JRSRawEncoder extends AbstractRawErasureEncoder {
+  private int[] generatingPolynomial;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
+    super.initialize(numDataUnits, numParityUnits, chunkSize);
+    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+
+    int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
+        getNumParityUnits());
+    // compute generating polynomial
+    int[] gen = {1};
+    int[] poly = new int[2];
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      poly[0] = primitivePower[i];
+      poly[1] = 1;
+      gen = RSUtil.GF.multiply(gen, poly);
+    }
+    // generating polynomial has all generating roots
+    generatingPolynomial = gen;
+  }
+
+  @Override
+  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    ByteBuffer[] data = new ByteBuffer[getNumDataUnits() + getNumParityUnits()];
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      data[i] = outputs[i];
+    }
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      data[i + getNumParityUnits()] = inputs[i];
+    }
+
+    // Compute the remainder
+    RSUtil.GF.remainder(data, generatingPolynomial);
+  }
+
+  @Override
+  protected void doEncode(byte[][] inputs, byte[][] outputs) {
+    byte[][] data = new byte[getNumDataUnits() + getNumParityUnits()][];
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      data[i] = outputs[i];
+    }
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      data[i + getNumParityUnits()] = inputs[i];
+    }
+
+    // Compute the remainder
+    RSUtil.GF.remainder(data, generatingPolynomial);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 91a9abf..6e07cf1 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -71,7 +71,7 @@ public interface RawErasureCoder {
   public boolean preferNativeBuffer();
 
   /**
-   * Should be called when release this coder. Good chance to release encoding
+   * Should be called when release this blockcoder. Good chance to release encoding
    * or decoding buffers
    */
   public void release();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
new file mode 100644
index 0000000..77544c6
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java
@@ -0,0 +1,497 @@
+/**
+ * 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.io.erasurecode.rawcoder.util;
+
+import java.nio.ByteBuffer;
+import java.util.HashMap;
+import java.util.Map;
+
+/**
+ * Implementation of Galois field arithmetic with 2^p elements. The input must
+ * be unsigned integers. It's ported from HDFS-RAID, slightly adapted.
+ */
+public class GaloisField {
+
+  // Field size 256 is good for byte based system
+  private static final int DEFAULT_FIELD_SIZE = 256;
+  // primitive polynomial 1 + X^2 + X^3 + X^4 + X^8 (substitute 2)
+  private static final int DEFAULT_PRIMITIVE_POLYNOMIAL = 285;
+  static private final Map<Integer, GaloisField> instances =
+      new HashMap<Integer, GaloisField>();
+  private final int[] logTable;
+  private final int[] powTable;
+  private final int[][] mulTable;
+  private final int[][] divTable;
+  private final int fieldSize;
+  private final int primitivePeriod;
+  private final int primitivePolynomial;
+
+  private GaloisField(int fieldSize, int primitivePolynomial) {
+    assert fieldSize > 0;
+    assert primitivePolynomial > 0;
+
+    this.fieldSize = fieldSize;
+    this.primitivePeriod = fieldSize - 1;
+    this.primitivePolynomial = primitivePolynomial;
+    logTable = new int[fieldSize];
+    powTable = new int[fieldSize];
+    mulTable = new int[fieldSize][fieldSize];
+    divTable = new int[fieldSize][fieldSize];
+    int value = 1;
+    for (int pow = 0; pow < fieldSize - 1; pow++) {
+      powTable[pow] = value;
+      logTable[value] = pow;
+      value = value * 2;
+      if (value >= fieldSize) {
+        value = value ^ primitivePolynomial;
+      }
+    }
+    // building multiplication table
+    for (int i = 0; i < fieldSize; i++) {
+      for (int j = 0; j < fieldSize; j++) {
+        if (i == 0 || j == 0) {
+          mulTable[i][j] = 0;
+          continue;
+        }
+        int z = logTable[i] + logTable[j];
+        z = z >= primitivePeriod ? z - primitivePeriod : z;
+        z = powTable[z];
+        mulTable[i][j] = z;
+      }
+    }
+    // building division table
+    for (int i = 0; i < fieldSize; i++) {
+      for (int j = 1; j < fieldSize; j++) {
+        if (i == 0) {
+          divTable[i][j] = 0;
+          continue;
+        }
+        int z = logTable[i] - logTable[j];
+        z = z < 0 ? z + primitivePeriod : z;
+        z = powTable[z];
+        divTable[i][j] = z;
+      }
+    }
+  }
+
+  /**
+   * Get the object performs Galois field arithmetics
+   *
+   * @param fieldSize           size of the field
+   * @param primitivePolynomial a primitive polynomial corresponds to the size
+   */
+  public static GaloisField getInstance(int fieldSize,
+                                        int primitivePolynomial) {
+    int key = ((fieldSize << 16) & 0xFFFF0000)
+        + (primitivePolynomial & 0x0000FFFF);
+    GaloisField gf;
+    synchronized (instances) {
+      gf = instances.get(key);
+      if (gf == null) {
+        gf = new GaloisField(fieldSize, primitivePolynomial);
+        instances.put(key, gf);
+      }
+    }
+    return gf;
+  }
+
+  /**
+   * Get the object performs Galois field arithmetic with default setting
+   */
+  public static GaloisField getInstance() {
+    return getInstance(DEFAULT_FIELD_SIZE, DEFAULT_PRIMITIVE_POLYNOMIAL);
+  }
+
+  /**
+   * Return number of elements in the field
+   *
+   * @return number of elements in the field
+   */
+  public int getFieldSize() {
+    return fieldSize;
+  }
+
+  /**
+   * Return the primitive polynomial in GF(2)
+   *
+   * @return primitive polynomial as a integer
+   */
+  public int getPrimitivePolynomial() {
+    return primitivePolynomial;
+  }
+
+  /**
+   * Compute the sum of two fields
+   *
+   * @param x input field
+   * @param y input field
+   * @return result of addition
+   */
+  public int add(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+    return x ^ y;
+  }
+
+  /**
+   * Compute the multiplication of two fields
+   *
+   * @param x input field
+   * @param y input field
+   * @return result of multiplication
+   */
+  public int multiply(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y >= 0 && y < getFieldSize());
+    return mulTable[x][y];
+  }
+
+  /**
+   * Compute the division of two fields
+   *
+   * @param x input field
+   * @param y input field
+   * @return x/y
+   */
+  public int divide(int x, int y) {
+    assert (x >= 0 && x < getFieldSize() && y > 0 && y < getFieldSize());
+    return divTable[x][y];
+  }
+
+  /**
+   * Compute power n of a field
+   *
+   * @param x input field
+   * @param n power
+   * @return x^n
+   */
+  public int power(int x, int n) {
+    assert (x >= 0 && x < getFieldSize());
+    if (n == 0) {
+      return 1;
+    }
+    if (x == 0) {
+      return 0;
+    }
+    x = logTable[x] * n;
+    if (x < primitivePeriod) {
+      return powTable[x];
+    }
+    x = x % primitivePeriod;
+    return powTable[x];
+  }
+
+  /**
+   * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+   * that Vz=y. The output z will be placed in y.
+   *
+   * @param x the vector which describe the Vandermonde matrix
+   * @param y right-hand side of the Vandermonde system equation. will be
+   *          replaced the output in this vector
+   */
+  public void solveVandermondeSystem(int[] x, int[] y) {
+    solveVandermondeSystem(x, y, x.length);
+  }
+
+  /**
+   * Given a Vandermonde matrix V[i][j]=x[j]^i and vector y, solve for z such
+   * that Vz=y. The output z will be placed in y.
+   *
+   * @param x   the vector which describe the Vandermonde matrix
+   * @param y   right-hand side of the Vandermonde system equation. will be
+   *            replaced the output in this vector
+   * @param len consider x and y only from 0...len-1
+   */
+  public void solveVandermondeSystem(int[] x, int[] y, int len) {
+    assert (x.length <= len && y.length <= len);
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        y[j] = y[j] ^ mulTable[x[i]][y[j - 1]];
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        y[j] = divTable[y[j]][x[j] ^ x[j - i - 1]];
+      }
+      for (int j = i; j < len - 1; j++) {
+        y[j] = y[j] ^ y[j + 1];
+      }
+    }
+  }
+
+  /**
+   * A "bulk" version to the solving of Vandermonde System
+   */
+  public void solveVandermondeSystem(int[] x, byte[][] y,
+                                     int len, int dataLen) {
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j][k] = (byte) (y[j][k] ^ mulTable[x[i]][y[j - 1][k] &
+              0x000000FF]);
+        }
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j][k] = (byte) (divTable[y[j][k] & 0x000000FF][x[j] ^
+              x[j - i - 1]]);
+        }
+      }
+      for (int j = i; j < len - 1; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j][k] = (byte) (y[j][k] ^ y[j + 1][k]);
+        }
+      }
+    }
+  }
+
+  /**
+   * A "bulk" version of the solveVandermondeSystem, using ByteBuffer.
+   */
+  public void solveVandermondeSystem(int[] x, ByteBuffer[] y,
+                                     int len, int dataLen) {
+    for (int i = 0; i < len - 1; i++) {
+      for (int j = len - 1; j > i; j--) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j].put(k, (byte) (y[j].get(k) ^ mulTable[x[i]][y[j - 1].get(k) &
+              0x000000FF]));
+        }
+      }
+    }
+    for (int i = len - 1; i >= 0; i--) {
+      for (int j = i + 1; j < len; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j].put(k, (byte) (divTable[y[j].get(k) & 0x000000FF][x[j] ^
+              x[j - i - 1]]));
+        }
+      }
+      for (int j = i; j < len - 1; j++) {
+        for (int k = 0; k < dataLen; k++) {
+          y[j].put(k, (byte) (y[j].get(k) ^ y[j + 1].get(k)));
+        }
+      }
+    }
+  }
+
+  /**
+   * Compute the multiplication of two polynomials. The index in the array
+   * corresponds to the power of the entry. For example p[0] is the constant
+   * term of the polynomial p.
+   *
+   * @param p input polynomial
+   * @param q input polynomial
+   * @return polynomial represents p*q
+   */
+  public int[] multiply(int[] p, int[] q) {
+    int len = p.length + q.length - 1;
+    int[] result = new int[len];
+    for (int i = 0; i < len; i++) {
+      result[i] = 0;
+    }
+    for (int i = 0; i < p.length; i++) {
+
+      for (int j = 0; j < q.length; j++) {
+        result[i + j] = add(result[i + j], multiply(p[i], q[j]));
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Compute the remainder of a dividend and divisor pair. The index in the
+   * array corresponds to the power of the entry. For example p[0] is the
+   * constant term of the polynomial p.
+   *
+   * @param dividend dividend polynomial, the remainder will be placed
+   *                 here when return
+   * @param divisor  divisor polynomial
+   */
+  public void remainder(int[] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      int ratio = divTable[dividend[i +
+          divisor.length - 1]][divisor[divisor.length - 1]];
+      for (int j = 0; j < divisor.length; j++) {
+        int k = j + i;
+        dividend[k] = dividend[k] ^ mulTable[ratio][divisor[j]];
+      }
+    }
+  }
+
+  /**
+   * Compute the sum of two polynomials. The index in the array corresponds to
+   * the power of the entry. For example p[0] is the constant term of the
+   * polynomial p.
+   *
+   * @param p input polynomial
+   * @param q input polynomial
+   * @return polynomial represents p+q
+   */
+  public int[] add(int[] p, int[] q) {
+    int len = Math.max(p.length, q.length);
+    int[] result = new int[len];
+    for (int i = 0; i < len; i++) {
+      if (i < p.length && i < q.length) {
+        result[i] = add(p[i], q[i]);
+      } else if (i < p.length) {
+        result[i] = p[i];
+      } else {
+        result[i] = q[i];
+      }
+    }
+    return result;
+  }
+
+  /**
+   * Substitute x into polynomial p(x).
+   *
+   * @param p input polynomial
+   * @param x input field
+   * @return p(x)
+   */
+  public int substitute(int[] p, int x) {
+    int result = 0;
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      result = result ^ mulTable[p[i]][y];
+      y = mulTable[x][y];
+    }
+    return result;
+  }
+
+  /**
+   * A "bulk" version of the substitute.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param q store the return result
+   * @param x input field
+   */
+  public void substitute(byte[][] p, byte[] q, int x) {
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      byte[] pi = p[i];
+      for (int j = 0; j < pi.length; j++) {
+        int pij = pi[j] & 0x000000FF;
+        q[j] = (byte) (q[j] ^ mulTable[pij][y]);
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * A "bulk" version of the substitute, using ByteBuffer.
+   * Tends to be 2X faster than the "int" substitute in a loop.
+   *
+   * @param p input polynomial
+   * @param q store the return result
+   * @param x input field
+   */
+  public void substitute(ByteBuffer[] p, ByteBuffer q, int x) {
+    int y = 1;
+    for (int i = 0; i < p.length; i++) {
+      ByteBuffer pi = p[i];
+      int len = pi.remaining();
+      for (int j = 0; j < len; j++) {
+        int pij = pi.get(j) & 0x000000FF;
+        q.put(j, (byte) (q.get(j) ^ mulTable[pij][y]));
+      }
+      y = mulTable[x][y];
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(byte[][] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      for (int j = 0; j < divisor.length; j++) {
+        for (int k = 0; k < dividend[i].length; k++) {
+          int ratio = divTable[dividend[i + divisor.length - 1][k] &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i][k] = (byte) ((dividend[j + i][k] & 0x00FF) ^
+              mulTable[ratio][divisor[j]]);
+        }
+      }
+    }
+  }
+
+  /**
+   * The "bulk" version of the remainder, using ByteBuffer.
+   * Warning: This function will modify the "dividend" inputs.
+   */
+  public void remainder(ByteBuffer[] dividend, int[] divisor) {
+    for (int i = dividend.length - divisor.length; i >= 0; i--) {
+      int width = dividend[i].remaining();
+      for (int j = 0; j < divisor.length; j++) {
+        for (int k = 0; k < width; k++) {
+          int ratio = divTable[dividend[i + divisor.length - 1].get(k) &
+              0x00FF][divisor[divisor.length - 1]];
+          dividend[j + i].put(k, (byte) ((dividend[j + i].get(k) & 0x00FF) ^
+              mulTable[ratio][divisor[j]]));
+        }
+      }
+    }
+  }
+
+  /**
+   * Perform Gaussian elimination on the given matrix. This matrix has to be a
+   * fat matrix (number of rows > number of columns).
+   */
+  public void gaussianElimination(int[][] matrix) {
+    assert(matrix != null && matrix.length > 0 && matrix[0].length > 0
+        && matrix.length < matrix[0].length);
+    int height = matrix.length;
+    int width = matrix[0].length;
+    for (int i = 0; i < height; i++) {
+      boolean pivotFound = false;
+      // scan the column for a nonzero pivot and swap it to the diagonal
+      for (int j = i; j < height; j++) {
+        if (matrix[i][j] != 0) {
+          int[] tmp = matrix[i];
+          matrix[i] = matrix[j];
+          matrix[j] = tmp;
+          pivotFound = true;
+          break;
+        }
+      }
+      if (!pivotFound) {
+        continue;
+      }
+      int pivot = matrix[i][i];
+      for (int j = i; j < width; j++) {
+        matrix[i][j] = divide(matrix[i][j], pivot);
+      }
+      for (int j = i + 1; j < height; j++) {
+        int lead = matrix[j][i];
+        for (int k = i; k < width; k++) {
+          matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+        }
+      }
+    }
+    for (int i = height - 1; i >=0; i--) {
+      for (int j = 0; j < i; j++) {
+        int lead = matrix[j][i];
+        for (int k = i; k < width; k++) {
+          matrix[j][k] = add(matrix[j][k], multiply(lead, matrix[i][k]));
+        }
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
new file mode 100644
index 0000000..33ba561
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
@@ -0,0 +1,22 @@
+package org.apache.hadoop.io.erasurecode.rawcoder.util;
+
+/**
+ * Some utilities for Reed-Solomon coding.
+ */
+public class RSUtil {
+
+  // We always use the byte system (with symbol size 8, field size 256,
+  // primitive polynomial 285, and primitive root 2).
+  public static GaloisField GF = GaloisField.getInstance();
+  public static final int PRIMITIVE_ROOT = 2;
+
+  public static int[] getPrimitivePower(int numDataUnits, int numParityUnits) {
+    int[] primitivePower = new int[numDataUnits + numParityUnits];
+    // compute powers of the primitive root
+    for (int i = 0; i < numDataUnits + numParityUnits; i++) {
+      primitivePower[i] = GF.power(PRIMITIVE_ROOT, i);
+    }
+    return primitivePower;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 9482b43..3c4288c 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -18,9 +18,11 @@
 package org.apache.hadoop.io.erasurecode;
 
 import java.nio.ByteBuffer;
+import java.util.Arrays;
 import java.util.Random;
 
 import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertTrue;
 
 /**
  * Test base of common utilities for tests not only raw coders but also block
@@ -41,6 +43,14 @@ public abstract class TestCoderBase {
   // may go to different coding implementations.
   protected boolean usingDirectBuffer = true;
 
+  protected void prepare(int numDataUnits, int numParityUnits,
+                         int[] erasedIndexes) {
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+    this.erasedDataIndexes = erasedIndexes != null ?
+        erasedIndexes : new int[] {0};
+  }
+
   /**
    * Compare and verify if erased chunks are equal to recovered chunks
    * @param erasedChunks
@@ -50,10 +60,8 @@ public abstract class TestCoderBase {
                                   ECChunk[] recoveredChunks) {
     byte[][] erased = ECChunk.toArray(erasedChunks);
     byte[][] recovered = ECChunk.toArray(recoveredChunks);
-    for (int i = 0; i < erasedChunks.length; ++i) {
-      assertArrayEquals("Decoding and comparing failed.", erased[i],
-          recovered[i]);
-    }
+    boolean result = Arrays.deepEquals(erased, recovered);
+    assertTrue("Decoding and comparing failed.", result);
   }
 
   /**
@@ -63,7 +71,7 @@ public abstract class TestCoderBase {
    */
   protected int[] getErasedIndexesForDecoding() {
     int[] erasedIndexesForDecoding = new int[erasedDataIndexes.length];
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
       erasedIndexesForDecoding[i] = erasedDataIndexes[i] + numParityUnits;
     }
     return erasedIndexesForDecoding;
@@ -100,7 +108,7 @@ public abstract class TestCoderBase {
     ECChunk[] copiedChunks = new ECChunk[erasedDataIndexes.length];
 
     int j = 0;
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
       copiedChunks[j ++] = cloneChunkWithData(dataChunks[erasedDataIndexes[i]]);
     }
 
@@ -112,7 +120,7 @@ public abstract class TestCoderBase {
    * @param dataChunks
    */
   protected void eraseSomeDataBlocks(ECChunk[] dataChunks) {
-    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+    for (int i = 0; i < erasedDataIndexes.length; i++) {
       eraseDataFromChunk(dataChunks[erasedDataIndexes[i]]);
     }
   }
@@ -122,7 +130,7 @@ public abstract class TestCoderBase {
    * @param chunks
    */
   protected void eraseDataFromChunks(ECChunk[] chunks) {
-    for (int i = 0; i < chunks.length; ++i) {
+    for (int i = 0; i < chunks.length; i++) {
       eraseDataFromChunk(chunks[i]);
     }
   }
@@ -135,7 +143,7 @@ public abstract class TestCoderBase {
     ByteBuffer chunkBuffer = chunk.getBuffer();
     // erase the data
     chunkBuffer.position(0);
-    for (int i = 0; i < chunkSize; ++i) {
+    for (int i = 0; i < chunkSize; i++) {
       chunkBuffer.put((byte) 0);
     }
     chunkBuffer.flip();
@@ -150,7 +158,7 @@ public abstract class TestCoderBase {
    */
   protected static ECChunk[] cloneChunksWithData(ECChunk[] chunks) {
     ECChunk[] results = new ECChunk[chunks.length];
-    for (int i = 0; i < chunks.length; ++i) {
+    for (int i = 0; i < chunks.length; i++) {
       results[i] = cloneChunkWithData(chunks[i]);
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
new file mode 100644
index 0000000..e54f647
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
@@ -0,0 +1,93 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Test raw Reed-solomon encoding and decoding.
+ */
+public class TestJRSRawCoder extends TestRawCoderBase {
+
+  private static int symbolSize = 0;
+  private static int symbolMax = 0;
+
+  static {
+    symbolSize = (int) Math.round(Math.log(
+        RSUtil.GF.getFieldSize()) / Math.log(2));
+    symbolMax = (int) Math.pow(2, symbolSize);
+  }
+
+  @Before
+  public void setup() {
+    this.encoderClass = JRSRawEncoder.class;
+    this.decoderClass = JRSRawDecoder.class;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_10x4() {
+    prepare(10, 4, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4() {
+    prepare(10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+    prepare(10, 4, new int[] {2, 4});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_all() {
+    prepare(10, 4, new int[] {0, 1, 2, 3});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3() {
+    prepare(3, 3, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3() {
+    prepare(3, 3, null);
+    testCoding(true);
+  }
+
+  @Override
+  protected ECChunk generateDataChunk() {
+    ByteBuffer buffer = allocateOutputBuffer();
+    for (int i = 0; i < chunkSize; i++) {
+      buffer.put((byte) RAND.nextInt(symbolMax));
+    }
+    buffer.flip();
+
+    return new ECChunk(buffer);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
index 9119211..5f6ccda 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -31,10 +31,13 @@ public abstract class TestRawCoderBase extends TestCoderBase {
    * Generating source data, encoding, recovering and then verifying.
    * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
    * it supports two kinds of ByteBuffers, one is array backed, the other is
-   * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test.
+   * direct ByteBuffer. Use usingDirectBuffer indicate which case to test.
+   *
    * @param usingDirectBuffer
    */
   protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+
     // Generate data and encode
     ECChunk[] dataChunks = prepareDataChunksForEncoding();
     ECChunk[] parityChunks = prepareParityChunksForEncoding();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b70d5846/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
index 8e59b8a..ff48586 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
@@ -26,7 +26,6 @@ import java.util.Random;
  * Test XOR encoding and decoding.
  */
 public class TestXorRawCoder extends TestRawCoderBase {
-  private static Random RAND = new Random();
 
   @Before
   public void setup() {


[18/50] [abbrv] hadoop git commit: HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks. Contributed by GAO Rui.

Posted by zh...@apache.org.
HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks. Contributed by GAO Rui.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ffd75cc4
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ffd75cc4
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ffd75cc4

Branch: refs/heads/HDFS-7285
Commit: ffd75cc4d5b0bc5b2c47d303b395d3fc70dee889
Parents: 43df926
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 23 15:06:53 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:51 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockIdManager.java     |  6 ++++++
 .../hdfs/server/blockmanagement/BlockManager.java  | 12 +++++++-----
 .../hdfs/server/blockmanagement/BlocksMap.java     |  2 +-
 .../hadoop/hdfs/server/namenode/FSNamesystem.java  | 17 ++++++++++++-----
 .../hadoop/hdfs/server/namenode/SafeMode.java      |  5 +++--
 .../java/org/apache/hadoop/hdfs/TestSafeMode.java  | 15 +++++++++++++--
 6 files changed, 42 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd75cc4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 1d69d74..187f8c9 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -233,6 +233,12 @@ public class BlockIdManager {
     return id < 0;
   }
 
+  /**
+   * The last 4 bits of HdfsConstants.BLOCK_GROUP_INDEX_MASK(15) is 1111,
+   * so the last 4 bits of (~HdfsConstants.BLOCK_GROUP_INDEX_MASK) is 0000
+   * and the other 60 bits are 1. Group ID is the first 60 bits of any
+   * data/parity block id in the same striped block group.
+   */
   public static long convertToStripedID(long id) {
     return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd75cc4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 4f81e3a..7eb1861 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -685,8 +685,10 @@ public class BlockManager {
     // a "forced" completion when a file is getting closed by an
     // OP_CLOSE edit on the standby).
     namesystem.adjustSafeModeBlockTotals(0, 1);
+    final int minStorage = curBlock.isStriped() ?
+        ((BlockInfoStriped) curBlock).getDataBlockNum() : minReplication;
     namesystem.incrementSafeBlockCount(
-        Math.min(numNodes, minReplication));
+        Math.min(numNodes, minStorage), curBlock);
     
     // replace block in the blocksMap
     return blocksMap.replaceBlock(completeBlock);
@@ -2211,7 +2213,7 @@ public class BlockManager {
         // refer HDFS-5283
         if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
           int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock);
-          namesystem.incrementSafeBlockCount(numOfReplicas);
+          namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock);
         }
         //and fall through to next clause
       }      
@@ -2592,14 +2594,14 @@ public class BlockManager {
       // only complete blocks are counted towards that.
       // In the case that the block just became complete above, completeBlock()
       // handles the safe block count maintenance.
-      namesystem.incrementSafeBlockCount(numCurrentReplica);
+      namesystem.incrementSafeBlockCount(numCurrentReplica, storedBlock);
     }
   }
 
   /**
    * Modify (block-->datanode) map. Remove block from set of
    * needed replications if this takes care of the problem.
-   * @return the block that is stored in blockMap.
+   * @return the block that is stored in blocksMap.
    */
   private Block addStoredBlock(final BlockInfo block,
                                final Block reportedBlock,
@@ -2668,7 +2670,7 @@ public class BlockManager {
       // Is no-op if not in safe mode.
       // In the case that the block just became complete above, completeBlock()
       // handles the safe block count maintenance.
-      namesystem.incrementSafeBlockCount(numCurrentReplica);
+      namesystem.incrementSafeBlockCount(numCurrentReplica, storedBlock);
     }
     
     // if file is under construction, then done for now

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd75cc4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index 0cf5fe6..e4d7b32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -140,7 +140,7 @@ class BlocksMap {
     }
   }
   
-  /** Returns the block object it it exists in the map. */
+  /** Returns the block object if it exists in the map. */
   BlockInfo getStoredBlock(Block b) {
     return blocks.get(b);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd75cc4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 8398ea7..76ac2aa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -213,6 +213,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStatistics;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption;
@@ -5378,10 +5379,16 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     /**
      * Increment number of safe blocks if current block has 
      * reached minimal replication.
-     * @param replication current replication 
+     * @param storageNum current number of replicas or number of internal blocks
+     *                   of a striped block group
+     * @param storedBlock current storedBlock which is either a
+     *                    BlockInfoContiguous or a BlockInfoStriped
      */
-    private synchronized void incrementSafeBlockCount(short replication) {
-      if (replication == safeReplication) {
+    private synchronized void incrementSafeBlockCount(short storageNum,
+        BlockInfo storedBlock) {
+      final int safe = storedBlock.isStriped() ?
+          ((BlockInfoStriped) storedBlock).getDataBlockNum() : safeReplication;
+      if (storageNum == safe) {
         this.blockSafe++;
 
         // Report startup progress only if we haven't completed startup yet.
@@ -5674,12 +5681,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public void incrementSafeBlockCount(int replication) {
+  public void incrementSafeBlockCount(int storageNum, BlockInfo storedBlock) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null)
       return;
-    safeMode.incrementSafeBlockCount((short)replication);
+    safeMode.incrementSafeBlockCount((short) storageNum, storedBlock);
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd75cc4/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 0debb1f..e26e727 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@ -45,9 +45,10 @@ public interface SafeMode {
     
   /**
    * Increment number of blocks that reached minimal replication.
-   * @param replication current replication 
+   * @param replication current replication
+   * @param storedBlock current stored Block
    */
-  public void incrementSafeBlockCount(int replication);
+  public void incrementSafeBlockCount(int replication, BlockInfo storedBlock);
 
   /** Decrement number of blocks that reached minimal replication. */
   public void decrementSafeBlockCount(BlockInfo b);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ffd75cc4/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
index 80fe9ee..a43e371 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeMode.java
@@ -552,7 +552,18 @@ public class TestSafeMode {
       if(cluster!= null) cluster.shutdown();
     }
   }
-  
+
+  //TODO : test should be added to check safeMode with stripedBloks after stripedBlock related functions have been added in class MiniDFSCluster
+  @Test
+  public void testSafeModeWithCorruptSripedBlock() throws IOException {
+    try {
+
+    } finally {
+      if(fs != null) fs.close();
+      if(cluster!= null) cluster.shutdown();
+    }
+  }
+
   void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOException {
     FileStatus stat = fs.getFileStatus(fileName);
     try {  
@@ -560,7 +571,7 @@ public class TestSafeMode {
     } catch (SafeModeException e) {
       assertTrue("Should have not got safemode exception", false);
     } catch (RemoteException re) {
-      assertTrue("Should have not got safemode exception", false);   
+      assertTrue("Should have not got remote exception", false);
     }    
   }
 }


[22/50] [abbrv] hadoop git commit: HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage. Contributed by Hui Zheng.

Posted by zh...@apache.org.
HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage. Contributed by Hui Zheng.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6a8afd35
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6a8afd35
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6a8afd35

Branch: refs/heads/HDFS-7285
Commit: 6a8afd35b526cea4f46fac9275bb1237f9eb603f
Parents: ffd75cc
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 23 15:10:10 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:52 2015 -0700

----------------------------------------------------------------------
 .../blockmanagement/BlockInfoStriped.java       |  11 +-
 .../hdfs/server/namenode/FSImageFormat.java     |  62 ++++++--
 .../server/namenode/FSImageSerialization.java   |  78 +++++++---
 .../blockmanagement/TestBlockInfoStriped.java   |  34 +++++
 .../hdfs/server/namenode/TestFSImage.java       | 148 ++++++++++++++++++-
 5 files changed, 300 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a8afd35/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index cef8318..30b5ee7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -20,6 +20,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import java.io.DataOutput;
+import java.io.IOException;
 
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
@@ -206,6 +208,13 @@ public class BlockInfoStriped extends BlockInfo {
     return num;
   }
 
+  @Override
+  public void write(DataOutput out) throws IOException {
+    out.writeShort(dataBlockNum);
+    out.writeShort(parityBlockNum);
+    super.write(out);
+  }
+
   /**
    * Convert a complete block to an under construction block.
    * @return BlockInfoUnderConstruction -  an under construction block.
@@ -215,7 +224,7 @@ public class BlockInfoStriped extends BlockInfo {
     final BlockInfoStripedUnderConstruction ucBlock;
     if(isComplete()) {
       ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(),
-              getParityBlockNum(),  s, targets);
+              getParityBlockNum(), s, targets);
       ucBlock.setBlockCollection(getBlockCollection());
     } else {
       // the block is already under construction

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a8afd35/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index 2e6e741..ad96863 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -47,13 +47,16 @@ import org.apache.hadoop.fs.PathIsNotDirectoryException;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.DFSUtil;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.common.InconsistentFSStateException;
@@ -752,16 +755,31 @@ public class FSImageFormat {
       atime = in.readLong();
     }
     final long blockSize = in.readLong();
+    final boolean isStriped = NameNodeLayoutVersion.supports(
+            NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
+            && (in.readBoolean());
     final int numBlocks = in.readInt();
 
     if (numBlocks >= 0) {
       // file
       
       // read blocks
-      BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
-      for (int j = 0; j < numBlocks; j++) {
-        blocks[j] = new BlockInfoContiguous(replication);
-        blocks[j].readFields(in);
+      Block[] blocks;
+      if (isStriped) {
+        blocks = new Block[numBlocks];
+        for (int j = 0; j < numBlocks; j++) {
+          short dataBlockNum = in.readShort();
+          short parityBlockNum = in.readShort();
+          blocks[j] = new BlockInfoStriped(new Block(),
+                  dataBlockNum, parityBlockNum);
+          blocks[j].readFields(in);
+        }
+      } else {
+        blocks = new BlockInfoContiguous[numBlocks];
+        for (int j = 0; j < numBlocks; j++) {
+          blocks[j] = new BlockInfoContiguous(replication);
+          blocks[j].readFields(in);
+        }
       }
 
       String clientName = "";
@@ -780,9 +798,18 @@ public class FSImageFormat {
             clientMachine = FSImageSerialization.readString(in);
             // convert the last block to BlockUC
             if (blocks.length > 0) {
-              BlockInfoContiguous lastBlk = blocks[blocks.length - 1];
-              blocks[blocks.length - 1] = new BlockInfoContiguousUnderConstruction(
-                  lastBlk, replication);
+              Block lastBlk = blocks[blocks.length - 1];
+              if (isStriped){
+                BlockInfoStriped lastStripedBlk = (BlockInfoStriped) lastBlk;
+                blocks[blocks.length - 1]
+                        = new BlockInfoStripedUnderConstruction(lastBlk,
+                                lastStripedBlk.getDataBlockNum(),
+                                lastStripedBlk.getParityBlockNum());
+              } else {
+                blocks[blocks.length - 1]
+                        = new BlockInfoContiguousUnderConstruction(lastBlk,
+                                replication);
+              }
             }
           }
         }
@@ -795,14 +822,25 @@ public class FSImageFormat {
         counter.increment();
       }
 
-      final INodeFile file = new INodeFile(inodeId, localName, permissions,
-          modificationTime, atime, blocks, replication, blockSize);
+      INodeFile file;
+      if (isStriped) {
+        file = new INodeFile(inodeId, localName, permissions, modificationTime,
+            atime, new BlockInfoContiguous[0], (short) 0, blockSize);
+        file.addStripedBlocksFeature();
+        for (Block block : blocks) {
+          file.getStripedBlocksFeature().addBlock((BlockInfoStriped) block);
+        }
+      } else {
+        file = new INodeFile(inodeId, localName, permissions,
+            modificationTime, atime, (BlockInfoContiguous[]) blocks,
+            replication, blockSize);
+      }
       if (underConstruction) {
         file.toUnderConstruction(clientName, clientMachine);
       }
-        return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
-      } else if (numBlocks == -1) {
-        //directory
+      return fileDiffs == null ? file : new INodeFile(file, fileDiffs);
+    } else if (numBlocks == -1) {
+      //directory
       
       //read quotas
       final long nsQuota = in.readLong();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a8afd35/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index 1888d87..1e58858 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -35,6 +35,8 @@ import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat.ReferenceMap;
@@ -124,21 +126,48 @@ public class FSImageSerialization {
     short blockReplication = in.readShort();
     long modificationTime = in.readLong();
     long preferredBlockSize = in.readLong();
+    final boolean isStriped = NameNodeLayoutVersion.supports(
+        NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)
+        && (in.readBoolean());
   
     int numBlocks = in.readInt();
-    BlockInfoContiguous[] blocks = new BlockInfoContiguous[numBlocks];
-    Block blk = new Block();
-    int i = 0;
-    for (; i < numBlocks-1; i++) {
-      blk.readFields(in);
-      blocks[i] = new BlockInfoContiguous(blk, blockReplication);
-    }
-    // last block is UNDER_CONSTRUCTION
-    if(numBlocks > 0) {
-      blk.readFields(in);
-      blocks[i] = new BlockInfoContiguousUnderConstruction(
-        blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
+
+    final BlockInfoContiguous[] blocksContiguous;
+    BlockInfoStriped[] blocksStriped = null;
+    if (isStriped) {
+      blocksContiguous = new BlockInfoContiguous[0];
+      blocksStriped = new BlockInfoStriped[numBlocks];
+      int i = 0;
+      for (; i < numBlocks - 1; i++) {
+        short dataBlockNum = in.readShort();
+        short parityBlockNum = in.readShort();
+        blocksStriped[i] = new BlockInfoStriped(new Block(), dataBlockNum,
+            parityBlockNum);
+        blocksStriped[i].readFields(in);
+      }
+      if (numBlocks > 0) {
+        short dataBlockNum = in.readShort();
+        short parityBlockNum = in.readShort();
+        blocksStriped[i] = new BlockInfoStripedUnderConstruction(new Block(),
+            dataBlockNum, parityBlockNum, BlockUCState.UNDER_CONSTRUCTION, null);
+        blocksStriped[i].readFields(in);
+      }
+    } else {
+      blocksContiguous = new BlockInfoContiguous[numBlocks];
+      Block blk = new Block();
+      int i = 0;
+      for (; i < numBlocks-1; i++) {
+        blk.readFields(in);
+        blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication);
+      }
+      // last block is UNDER_CONSTRUCTION
+      if(numBlocks > 0) {
+        blk.readFields(in);
+        blocksContiguous[i] = new BlockInfoContiguousUnderConstruction(
+                blk, blockReplication, BlockUCState.UNDER_CONSTRUCTION, null);
+      }
     }
+
     PermissionStatus perm = PermissionStatus.read(in);
     String clientName = readString(in);
     String clientMachine = readString(in);
@@ -150,8 +179,19 @@ public class FSImageSerialization {
 
     // Images in the pre-protobuf format will not have the lazyPersist flag,
     // so it is safe to pass false always.
-    INodeFile file = new INodeFile(inodeId, name, perm, modificationTime,
-        modificationTime, blocks, blockReplication, preferredBlockSize);
+    INodeFile file;
+    if (isStriped) {
+      file = new INodeFile(inodeId, name, perm, modificationTime,
+          modificationTime, blocksContiguous, (short) 0, preferredBlockSize);
+      file.addStripedBlocksFeature();
+      for (int i = 0; i < numBlocks; i++) {
+        file.getStripedBlocksFeature().addBlock(blocksStriped[i]);
+      }
+    } else {
+      file = new INodeFile(inodeId, name, perm, modificationTime,
+          modificationTime, blocksContiguous, blockReplication,
+          preferredBlockSize);
+    }
     file.toUnderConstruction(clientName, clientMachine);
     return file;
   }
@@ -166,7 +206,8 @@ public class FSImageSerialization {
     out.writeShort(cons.getFileReplication());
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
-
+    // whether the file has striped blocks
+    out.writeBoolean(cons.isWithStripedBlocks());
     writeBlocks(cons.getBlocks(), out);
     cons.getPermissionStatus().write(out);
 
@@ -179,9 +220,9 @@ public class FSImageSerialization {
 
   /**
    * Serialize a {@link INodeFile} node
-   * @param node The node to write
+   * @param file The node to write
    * @param out The {@link DataOutputStream} where the fields are written
-   * @param writeBlock Whether to write block information
+   * @param writeUnderConstruction Whether to write block information
    */
   public static void writeINodeFile(INodeFile file, DataOutput out,
       boolean writeUnderConstruction) throws IOException {
@@ -191,7 +232,8 @@ public class FSImageSerialization {
     out.writeLong(file.getModificationTime());
     out.writeLong(file.getAccessTime());
     out.writeLong(file.getPreferredBlockSize());
-
+    // whether the file has striped blocks
+    out.writeBoolean(file.isWithStripedBlocks());
     writeBlocks(file.getBlocks(), out);
     SnapshotFSImageFormat.saveFileDiffList(file, out);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a8afd35/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
index 74ddac0..c4db5d4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java
@@ -25,8 +25,16 @@ import org.junit.Before;
 import org.junit.Test;
 import org.mockito.internal.util.reflection.Whitebox;
 
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.ByteArrayOutputStream;
+import java.nio.ByteBuffer;
+
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_PARITY_BLOCKS;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.fail;
 
 /**
  * Test {@link BlockInfoStriped}
@@ -216,4 +224,30 @@ public class TestBlockInfoStriped {
       Assert.assertNull(newBlockInfo.getNext());
     }
   }
+
+  @Test
+  public void testWrite() {
+    long blkID = 1;
+    long numBytes = 1;
+    long generationStamp = 1;
+    short dataBlockNum = 6;
+    short parityBlockNum = 3;
+    ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE/Byte.SIZE*3
+            + Short.SIZE/Byte.SIZE*2);
+    byteBuffer.putShort(dataBlockNum).putShort(parityBlockNum)
+            .putLong(blkID).putLong(numBytes).putLong(generationStamp);
+
+    ByteArrayOutputStream byteStream = new ByteArrayOutputStream();
+    DataOutput out = new DataOutputStream(byteStream);
+    BlockInfoStriped blk = new BlockInfoStriped(new Block(1,1,1),
+            (short)6,(short)3);
+    try {
+      blk.write(out);
+    } catch(Exception ex) {
+      fail("testWrite error:" + ex.getMessage());
+    }
+    assertEquals(byteBuffer.array().length, byteStream.toByteArray().length);
+    assertArrayEquals(byteBuffer.array(), byteStream.toByteArray());
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6a8afd35/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 4d42911..71dc978 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -17,18 +17,28 @@
  */
 package org.apache.hadoop.hdfs.server.namenode;
 
+import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
+import java.io.DataOutput;
+import java.io.DataOutputStream;
+import java.io.DataInput;
+import java.io.DataInputStream;
+import java.io.ByteArrayOutputStream;
+import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
+import org.apache.hadoop.fs.permission.PermissionStatus;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -42,8 +52,8 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
 import org.apache.hadoop.test.GenericTestUtils;
@@ -120,6 +130,140 @@ public class TestFSImage {
     }
   }
 
+  private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf,
+      boolean isUC) throws IOException{
+    // contruct a INode with StripedBlock for saving and loading
+    long id = 123456789;
+    byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
+    PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
+            "testuser_groups", new FsPermission((short)0x755));
+    long mtime = 1426222916-3600;
+    long atime = 1426222916;
+    BlockInfoContiguous[] blks = new BlockInfoContiguous[0];
+    short replication = 3;
+    long preferredBlockSize = 128*1024*1024;
+    byte storagePolicyID = HdfsConstants.EC_STORAGE_POLICY_ID;
+    INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
+        blks, replication, preferredBlockSize, storagePolicyID);
+    ByteArrayOutputStream bs = new ByteArrayOutputStream();
+    file.addStripedBlocksFeature();
+
+    //construct StripedBlocks for the INode
+    BlockInfoStriped[] stripedBlks = new BlockInfoStriped[3];
+    long stripedBlkId = 10000001;
+    long timestamp = mtime+3600;
+    for (int i = 0; i < stripedBlks.length; i++) {
+      stripedBlks[i] = new BlockInfoStriped(
+              new Block(stripedBlkId + i, preferredBlockSize, timestamp),
+              (short) 6, (short) 3);
+      file.getStripedBlocksFeature().addBlock(stripedBlks[i]);
+    }
+
+    final String client = "testClient";
+    final String clientMachine = "testClientMachine";
+    final String path = "testUnderConstructionPath";
+
+    //save the INode to byte array
+    DataOutput out = new DataOutputStream(bs);
+    if (isUC) {
+      file.toUnderConstruction(client, clientMachine);
+      FSImageSerialization.writeINodeUnderConstruction((DataOutputStream) out,
+          file, path);
+    } else {
+      FSImageSerialization.writeINodeFile(file, out, false);
+    }
+    DataInput in = new DataInputStream(
+            new ByteArrayInputStream(bs.toByteArray()));
+
+    // load the INode from the byte array
+    INodeFile fileByLoaded;
+    if (isUC) {
+      fileByLoaded = FSImageSerialization.readINodeUnderConstruction(in,
+              fsn, fsn.getFSImage().getLayoutVersion());
+    } else {
+      fileByLoaded = (INodeFile) new FSImageFormat.Loader(conf, fsn)
+              .loadINodeWithLocalName(false, in, false);
+    }
+
+    assertEquals(id, fileByLoaded.getId() );
+    assertArrayEquals(isUC ? path.getBytes() : name,
+        fileByLoaded.getLocalName().getBytes());
+    assertEquals(permissionStatus.getUserName(),
+        fileByLoaded.getPermissionStatus().getUserName());
+    assertEquals(permissionStatus.getGroupName(),
+        fileByLoaded.getPermissionStatus().getGroupName());
+    assertEquals(permissionStatus.getPermission(),
+        fileByLoaded.getPermissionStatus().getPermission());
+    assertEquals(mtime, fileByLoaded.getModificationTime());
+    assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
+    assertEquals(0, fileByLoaded.getContiguousBlocks().length);
+    assertEquals(0, fileByLoaded.getBlockReplication());
+    assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
+
+    //check the BlockInfoStriped
+    BlockInfoStriped[] stripedBlksByLoaded =
+        fileByLoaded.getStripedBlocksFeature().getBlocks();
+    assertEquals(3, stripedBlksByLoaded.length);
+    for (int i = 0; i < 3; i++) {
+      assertEquals(stripedBlks[i].getBlockId(),
+          stripedBlksByLoaded[i].getBlockId());
+      assertEquals(stripedBlks[i].getNumBytes(),
+          stripedBlksByLoaded[i].getNumBytes());
+      assertEquals(stripedBlks[i].getGenerationStamp(),
+          stripedBlksByLoaded[i].getGenerationStamp());
+      assertEquals(stripedBlks[i].getDataBlockNum(),
+          stripedBlksByLoaded[i].getDataBlockNum());
+      assertEquals(stripedBlks[i].getParityBlockNum(),
+          stripedBlksByLoaded[i].getParityBlockNum());
+    }
+
+    if (isUC) {
+      assertEquals(client,
+          fileByLoaded.getFileUnderConstructionFeature().getClientName());
+      assertEquals(clientMachine,
+          fileByLoaded.getFileUnderConstructionFeature().getClientMachine());
+    }
+  }
+
+  /**
+   * Test if a INodeFile with BlockInfoStriped can be saved by
+   * FSImageSerialization and loaded by FSImageFormat#Loader.
+   */
+  @Test
+  public void testSaveAndLoadInodeFile() throws IOException{
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, false);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  /**
+   * Test if a INodeFileUnderConstruction with BlockInfoStriped can be
+   * saved and loaded by FSImageSerialization
+   */
+  @Test
+  public void testSaveAndLoadInodeFileUC() throws IOException{
+    // construct a INode with StripedBlock for saving and loading
+    Configuration conf = new Configuration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).build();
+      cluster.waitActive();
+      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, true);
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
   /**
    * Ensure that the digest written by the saver equals to the digest of the
    * file.


[37/50] [abbrv] hadoop git commit: Updated CHANGES-HDFS-EC-7285.txt

Posted by zh...@apache.org.
Updated CHANGES-HDFS-EC-7285.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d20e16e2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d20e16e2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d20e16e2

Branch: refs/heads/HDFS-7285
Commit: d20e16e2c178f1cc214a987bf53c4563186c0d96
Parents: 0993f53
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Apr 8 01:31:46 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:15:35 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 3 +++
 1 file changed, 3 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d20e16e2/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 01280db..68d1d32 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -32,3 +32,6 @@
 
     HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
     ( Xinwei Qin via Kai Zheng )
+
+    HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng
+    ( Kai Zheng )


[12/50] [abbrv] hadoop git commit: HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped blocks. Contributed by Kai Sasaki.

Posted by zh...@apache.org.
HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped blocks. Contributed by Kai Sasaki.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5c7ab8b0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5c7ab8b0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5c7ab8b0

Branch: refs/heads/HDFS-7285
Commit: 5c7ab8b0afba15d0861ee680d2bf1fa40b1e7d93
Parents: 8336831
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 16 16:37:08 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:41:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  3 +
 .../blockmanagement/BlockInfoStriped.java       | 12 ++-
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 89 +++++++++++++++++---
 3 files changed, 90 insertions(+), 14 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c7ab8b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 245b630..07b72e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -186,4 +186,7 @@ public class HdfsConstants {
   public static final byte NUM_PARITY_BLOCKS = 2;
   public static final long BLOCK_GROUP_INDEX_MASK = 15;
   public static final byte MAX_BLOCKS_IN_GROUP = 16;
+
+  // The chunk size for striped block which is used by erasure coding
+  public static final int BLOCK_STRIPED_CHUNK_SIZE = 64 * 1024;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c7ab8b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 84c3be6..cef8318 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
@@ -34,6 +35,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
  * array to record the block index for each triplet.
  */
 public class BlockInfoStriped extends BlockInfo {
+  private final int   chunkSize = HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
   private final short dataBlockNum;
   private final short parityBlockNum;
   /**
@@ -56,7 +58,7 @@ public class BlockInfoStriped extends BlockInfo {
     this.setBlockCollection(b.getBlockCollection());
   }
 
-  short getTotalBlockNum() {
+  public short getTotalBlockNum() {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
@@ -178,6 +180,14 @@ public class BlockInfoStriped extends BlockInfo {
     }
   }
 
+  public long spaceConsumed() {
+    // In case striped blocks, total usage by this striped blocks should
+    // be the total of data blocks and parity blocks because
+    // `getNumBytes` is the total of actual data block size.
+    return ((getNumBytes() - 1) / (dataBlockNum * chunkSize) + 1)
+        * chunkSize * parityBlockNum + getNumBytes();
+  }
+
   @Override
   public final boolean isStriped() {
     return true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5c7ab8b0/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 22d61bc..9cd7ddd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
@@ -649,6 +650,9 @@ public class INodeFile extends INodeWithAdditionalFields
     long nsDelta = 1;
     final long ssDeltaNoReplication;
     short replication;
+    if (isWithStripedBlocks()) {
+      return computeQuotaUsageWithStriped(bsps, counts);
+    }
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
     if (sf != null) {
       FileDiffList fileDiffList = sf.getDiffs();
@@ -686,6 +690,23 @@ public class INodeFile extends INodeWithAdditionalFields
     return counts;
   }
 
+  /**
+   * Compute quota of striped file
+   * @param bsps
+   * @param counts
+   * @param useCache
+   * @param lastSnapshotId
+   * @return quota counts
+   */
+  public final QuotaCounts computeQuotaUsageWithStriped(
+      BlockStoragePolicySuite bsps, QuotaCounts counts) {
+    long nsDelta = 1;
+    final long ssDelta = storagespaceConsumed();
+    counts.addNameSpace(nsDelta);
+    counts.addStorageSpace(ssDelta);
+    return counts;
+  }
+
   @Override
   public final ContentSummaryComputationContext computeContentSummary(
       final ContentSummaryComputationContext summary) {
@@ -763,23 +784,37 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return file size
    */
   public final long computeFileSize(boolean includesLastUcBlock,
-      boolean usePreferredBlockSize4LastUcBlock) {
-    if (blocks == null || blocks.length == 0) {
+                                    boolean usePreferredBlockSize4LastUcBlock) {
+    BlockInfo[] blockInfos = getBlocks();
+    // In case of contiguous blocks
+    if (blockInfos == null || blockInfos.length == 0) {
       return 0;
     }
-    final int last = blocks.length - 1;
+    final int last = blockInfos.length - 1;
     //check if the last block is BlockInfoUnderConstruction
-    long size = blocks[last].getNumBytes();
-    if (blocks[last] instanceof BlockInfoContiguousUnderConstruction) {
-       if (!includesLastUcBlock) {
-         size = 0;
-       } else if (usePreferredBlockSize4LastUcBlock) {
-         size = getPreferredBlockSize();
-       }
+    long size = blockInfos[last].getNumBytes();
+    if (blockInfos[last] instanceof BlockInfoContiguousUnderConstruction) {
+      if (!includesLastUcBlock) {
+        size = 0;
+      } else if (usePreferredBlockSize4LastUcBlock) {
+        size = getPreferredBlockSize();
+      }
+    } else if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
+      if (!includesLastUcBlock) {
+        size = 0;
+      } else if (usePreferredBlockSize4LastUcBlock) {
+        // Striped blocks keeps block group which counts
+        // (data blocks num + parity blocks num). When you
+        // count actual used size by BlockInfoStripedUC must
+        // be multiplied by these blocks number.
+        BlockInfoStripedUnderConstruction blockInfoStripedUC
+            = (BlockInfoStripedUnderConstruction) blockInfos[last];
+        size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
+      }
     }
     //sum other blocks
-    for(int i = 0; i < last; i++) {
-      size += blocks[i].getNumBytes();
+    for (int i = 0; i < last; i++) {
+      size += blockInfos[i].getNumBytes();
     }
     return size;
   }
@@ -790,7 +825,35 @@ public class INodeFile extends INodeWithAdditionalFields
    * Use preferred block size for the last block if it is under construction.
    */
   public final long storagespaceConsumed() {
-    return storagespaceConsumedNoReplication() * getBlockReplication();
+    if (isWithStripedBlocks()) {
+      return storagespaceConsumedWithStriped();
+    } else {
+      return storagespaceConsumedNoReplication() * getBlockReplication();
+    }
+  }
+
+  /**
+   * Compute size consumed by striped blocks.
+   * @return long
+   */
+  public final long storagespaceConsumedWithStriped() {
+    BlockInfo[] blockInfos = getBlocks();
+    long size = 0;
+    final int last = blockInfos.length - 1;
+    if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
+      BlockInfoStripedUnderConstruction blockInfoStripedUC
+          =(BlockInfoStripedUnderConstruction)blockInfos[last];
+      size = getPreferredBlockSize() * blockInfoStripedUC.getTotalBlockNum();
+    } else {
+      // In case of last block is complete
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[last];
+      size = blockInfoStriped.spaceConsumed();
+    }
+    for (int i = 0; i < last; i++) {
+      BlockInfoStriped blockInfoStriped = (BlockInfoStriped)blockInfos[i];
+      size += blockInfoStriped.spaceConsumed();
+    }
+    return size;
   }
 
   public final long storagespaceConsumedNoReplication() {


[42/50] [abbrv] hadoop git commit: HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B) Added missed file

Posted by zh...@apache.org.
HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B)
Added missed file


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/228c6b42
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/228c6b42
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/228c6b42

Branch: refs/heads/HDFS-7285
Commit: 228c6b42b6cbbd3075178f7e47073018a37d2087
Parents: bc69d45
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 8 14:23:03 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:56 2015 -0700

----------------------------------------------------------------------
 .../org/apache/hadoop/hdfs/protocol/ECInfo.java | 41 ++++++++++++++++++++
 1 file changed, 41 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/228c6b42/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
new file mode 100644
index 0000000..ca642c2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ECInfo.java
@@ -0,0 +1,41 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * Class to provide information, such as ECSchema, for a file/block.
+ */
+public class ECInfo {
+  private final String src;
+  private final ECSchema schema;
+
+  public ECInfo(String src, ECSchema schema) {
+    this.src = src;
+    this.schema = schema;
+  }
+
+  public String getSrc() {
+    return src;
+  }
+
+  public ECSchema getSchema() {
+    return schema;
+  }
+}


[28/50] [abbrv] hadoop git commit: HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin

Posted by zh...@apache.org.
HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/784f9d3a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/784f9d3a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/784f9d3a

Branch: refs/heads/HDFS-7285
Commit: 784f9d3ad97a553d2e4bbaa4205f7ecd37511301
Parents: ade2d75
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Apr 2 05:12:35 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:48:56 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt     | 3 +++
 .../src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java | 4 ++--
 2 files changed, 5 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/784f9d3a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index b69e69a..01280db 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -29,3 +29,6 @@
 
     HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng
     ( Kai Zheng )
+
+    HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
+    ( Xinwei Qin via Kai Zheng )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/784f9d3a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
index 8dc3f45..27be00e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -64,7 +64,7 @@ public class ECSchema {
       }
     } catch (NumberFormatException e) {
       throw new IllegalArgumentException("Option value " +
-          options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          options.get(NUM_DATA_UNITS_KEY) + " for " + NUM_DATA_UNITS_KEY +
           " is found. It should be an integer");
     }
 
@@ -74,7 +74,7 @@ public class ECSchema {
       }
     } catch (NumberFormatException e) {
       throw new IllegalArgumentException("Option value " +
-          options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          options.get(NUM_PARITY_UNITS_KEY) + " for " + NUM_PARITY_UNITS_KEY +
           " is found. It should be an integer");
     }
 


[36/50] [abbrv] hadoop git commit: HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/0993f530
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/0993f530
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/0993f530

Branch: refs/heads/HDFS-7285
Commit: 0993f5305d0c127ee3ba46a7377433369d2f2a99
Parents: 9426f64
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Apr 8 01:26:40 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:15:35 2015 -0700

----------------------------------------------------------------------
 .../io/erasurecode/coder/RSErasureDecoder.java  |  8 +-
 .../io/erasurecode/coder/RSErasureEncoder.java  |  4 +-
 .../io/erasurecode/coder/XORErasureDecoder.java | 78 ++++++++++++++++
 .../io/erasurecode/coder/XORErasureEncoder.java | 45 ++++++++++
 .../io/erasurecode/coder/XorErasureDecoder.java | 78 ----------------
 .../io/erasurecode/coder/XorErasureEncoder.java | 45 ----------
 .../io/erasurecode/rawcoder/JRSRawDecoder.java  | 69 ---------------
 .../io/erasurecode/rawcoder/JRSRawEncoder.java  | 78 ----------------
 .../rawcoder/JRSRawErasureCoderFactory.java     | 34 -------
 .../io/erasurecode/rawcoder/RSRawDecoder.java   | 69 +++++++++++++++
 .../io/erasurecode/rawcoder/RSRawEncoder.java   | 78 ++++++++++++++++
 .../rawcoder/RSRawErasureCoderFactory.java      | 34 +++++++
 .../io/erasurecode/rawcoder/XORRawDecoder.java  | 81 +++++++++++++++++
 .../io/erasurecode/rawcoder/XORRawEncoder.java  | 61 +++++++++++++
 .../rawcoder/XORRawErasureCoderFactory.java     | 34 +++++++
 .../io/erasurecode/rawcoder/XorRawDecoder.java  | 81 -----------------
 .../io/erasurecode/rawcoder/XorRawEncoder.java  | 61 -------------
 .../rawcoder/XorRawErasureCoderFactory.java     | 34 -------
 .../erasurecode/coder/TestRSErasureCoder.java   |  4 +-
 .../io/erasurecode/coder/TestXORCoder.java      | 50 +++++++++++
 .../io/erasurecode/coder/TestXorCoder.java      | 50 -----------
 .../erasurecode/rawcoder/TestJRSRawCoder.java   | 93 --------------------
 .../io/erasurecode/rawcoder/TestRSRawCoder.java | 93 ++++++++++++++++++++
 .../erasurecode/rawcoder/TestXORRawCoder.java   | 49 +++++++++++
 .../erasurecode/rawcoder/TestXorRawCoder.java   | 51 -----------
 25 files changed, 680 insertions(+), 682 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
index ba32f04..e2c5051 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -4,9 +4,9 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawDecoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
@@ -56,7 +56,7 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
       rsRawDecoder = createRawDecoder(
           CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
       if (rsRawDecoder == null) {
-        rsRawDecoder = new JRSRawDecoder();
+        rsRawDecoder = new RSRawDecoder();
       }
       rsRawDecoder.initialize(getNumDataUnits(),
           getNumParityUnits(), getChunkSize());
@@ -66,7 +66,7 @@ public class RSErasureDecoder extends AbstractErasureDecoder {
 
   private RawErasureDecoder checkCreateXorRawDecoder() {
     if (xorRawDecoder == null) {
-      xorRawDecoder = new XorRawDecoder();
+      xorRawDecoder = new XORRawDecoder();
       xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
     }
     return xorRawDecoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
index 430749d..a7d02b5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -3,7 +3,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 
 /**
@@ -30,7 +30,7 @@ public class RSErasureEncoder extends AbstractErasureEncoder {
       rawEncoder = createRawEncoder(
           CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
       if (rawEncoder == null) {
-        rawEncoder = new JRSRawEncoder();
+        rawEncoder = new RSRawEncoder();
       }
       rawEncoder.initialize(getNumDataUnits(),
           getNumParityUnits(), getChunkSize());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
new file mode 100644
index 0000000..6f4b423
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
+
+/**
+ * Xor erasure decoder that decodes a block group.
+ *
+ * It implements {@link ErasureDecoder}.
+ */
+public class XORErasureDecoder extends AbstractErasureDecoder {
+
+  @Override
+  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
+    // May be configured
+    RawErasureDecoder rawDecoder = new XORRawDecoder();
+    rawDecoder.initialize(getNumDataUnits(),
+        getNumParityUnits(), getChunkSize());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureDecodingStep(inputBlocks,
+        getErasedIndexes(inputBlocks),
+        getOutputBlocks(blockGroup), rawDecoder);
+  }
+
+  /**
+   * Which blocks were erased ? For XOR it's simple we only allow and return one
+   * erased block, either data or parity.
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  @Override
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    /**
+     * If more than one blocks (either data or parity) erased, then it's not
+     * edible to recover. We don't have the check here since it will be done
+     * by upper level: ErasreCoder call can be avoid if not possible to recover
+     * at all.
+     */
+    int erasedNum = getNumErasedBlocks(blockGroup);
+    ECBlock[] outputBlocks = new ECBlock[erasedNum];
+
+    int idx = 0;
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      if (blockGroup.getParityBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
+      }
+    }
+
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      if (blockGroup.getDataBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
+      }
+    }
+
+    return outputBlocks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
new file mode 100644
index 0000000..9011857
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java
@@ -0,0 +1,45 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XORRawEncoder;
+
+/**
+ * Xor erasure encoder that encodes a block group.
+ *
+ * It implements {@link ErasureEncoder}.
+ */
+public class XORErasureEncoder extends AbstractErasureEncoder {
+
+  @Override
+  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
+    // May be configured
+    RawErasureEncoder rawEncoder = new XORRawEncoder();
+    rawEncoder.initialize(getNumDataUnits(),
+        getNumParityUnits(), getChunkSize());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
deleted file mode 100644
index 33f5386..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * 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.io.erasurecode.coder;
-
-import org.apache.hadoop.io.erasurecode.ECBlock;
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
-
-/**
- * Xor erasure decoder that decodes a block group.
- *
- * It implements {@link ErasureDecoder}.
- */
-public class XorErasureDecoder extends AbstractErasureDecoder {
-
-  @Override
-  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
-    // May be configured
-    RawErasureDecoder rawDecoder = new XorRawDecoder();
-    rawDecoder.initialize(getNumDataUnits(),
-        getNumParityUnits(), getChunkSize());
-
-    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
-
-    return new ErasureDecodingStep(inputBlocks,
-        getErasedIndexes(inputBlocks),
-        getOutputBlocks(blockGroup), rawDecoder);
-  }
-
-  /**
-   * Which blocks were erased ? For XOR it's simple we only allow and return one
-   * erased block, either data or parity.
-   * @param blockGroup
-   * @return output blocks to recover
-   */
-  @Override
-  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
-    /**
-     * If more than one blocks (either data or parity) erased, then it's not
-     * edible to recover. We don't have the check here since it will be done
-     * by upper level: ErasreCoder call can be avoid if not possible to recover
-     * at all.
-     */
-    int erasedNum = getNumErasedBlocks(blockGroup);
-    ECBlock[] outputBlocks = new ECBlock[erasedNum];
-
-    int idx = 0;
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      if (blockGroup.getParityBlocks()[i].isErased()) {
-        outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
-      }
-    }
-
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      if (blockGroup.getDataBlocks()[i].isErased()) {
-        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
-      }
-    }
-
-    return outputBlocks;
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
deleted file mode 100644
index f8d67c3..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
+++ /dev/null
@@ -1,45 +0,0 @@
-/**
- * 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.io.erasurecode.coder;
-
-import org.apache.hadoop.io.erasurecode.ECBlock;
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
-import org.apache.hadoop.io.erasurecode.rawcoder.XorRawEncoder;
-
-/**
- * Xor erasure encoder that encodes a block group.
- *
- * It implements {@link ErasureEncoder}.
- */
-public class XorErasureEncoder extends AbstractErasureEncoder {
-
-  @Override
-  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
-    // May be configured
-    RawErasureEncoder rawEncoder = new XorRawEncoder();
-    rawEncoder.initialize(getNumDataUnits(),
-        getNumParityUnits(), getChunkSize());
-
-    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
-
-    return new ErasureEncodingStep(inputBlocks,
-        getOutputBlocks(blockGroup), rawEncoder);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
deleted file mode 100644
index dbb689e..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawDecoder.java
+++ /dev/null
@@ -1,69 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw erasure decoder in RS code scheme in pure Java in case native one
- * isn't available in some environment. Please always use native implementations
- * when possible.
- */
-public class JRSRawDecoder extends AbstractRawErasureDecoder {
-  // To describe and calculate the needed Vandermonde matrix
-  private int[] errSignature;
-  private int[] primitivePower;
-
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
-    super.initialize(numDataUnits, numParityUnits, chunkSize);
-    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
-
-    this.errSignature = new int[getNumParityUnits()];
-    this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
-        getNumParityUnits());
-  }
-
-  @Override
-  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
-                          ByteBuffer[] outputs) {
-    for (int i = 0; i < erasedIndexes.length; i++) {
-      errSignature[i] = primitivePower[erasedIndexes[i]];
-      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
-    }
-
-    int dataLen = inputs[0].remaining();
-    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
-        erasedIndexes.length, dataLen);
-  }
-
-  @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
-    for (int i = 0; i < erasedIndexes.length; i++) {
-      errSignature[i] = primitivePower[erasedIndexes[i]];
-      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
-    }
-
-    int dataLen = inputs[0].length;
-    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
-        erasedIndexes.length, dataLen);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
deleted file mode 100644
index 6ea7551..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawEncoder.java
+++ /dev/null
@@ -1,78 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw erasure encoder in RS code scheme in pure Java in case native one
- * isn't available in some environment. Please always use native implementations
- * when possible.
- */
-public class JRSRawEncoder extends AbstractRawErasureEncoder {
-  private int[] generatingPolynomial;
-
-  @Override
-  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
-    super.initialize(numDataUnits, numParityUnits, chunkSize);
-    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
-
-    int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
-        getNumParityUnits());
-    // compute generating polynomial
-    int[] gen = {1};
-    int[] poly = new int[2];
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      poly[0] = primitivePower[i];
-      poly[1] = 1;
-      gen = RSUtil.GF.multiply(gen, poly);
-    }
-    // generating polynomial has all generating roots
-    generatingPolynomial = gen;
-  }
-
-  @Override
-  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    ByteBuffer[] data = new ByteBuffer[getNumDataUnits() + getNumParityUnits()];
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      data[i] = outputs[i];
-    }
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      data[i + getNumParityUnits()] = inputs[i];
-    }
-
-    // Compute the remainder
-    RSUtil.GF.remainder(data, generatingPolynomial);
-  }
-
-  @Override
-  protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    byte[][] data = new byte[getNumDataUnits() + getNumParityUnits()][];
-    for (int i = 0; i < getNumParityUnits(); i++) {
-      data[i] = outputs[i];
-    }
-    for (int i = 0; i < getNumDataUnits(); i++) {
-      data[i + getNumParityUnits()] = inputs[i];
-    }
-
-    // Compute the remainder
-    RSUtil.GF.remainder(data, generatingPolynomial);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
deleted file mode 100644
index d6b40aa..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-/**
- * A raw coder factory for raw Reed-Solomon coder in Java.
- */
-public class JRSRawErasureCoderFactory implements RawErasureCoderFactory {
-
-  @Override
-  public RawErasureEncoder createEncoder() {
-    return new JRSRawEncoder();
-  }
-
-  @Override
-  public RawErasureDecoder createDecoder() {
-    return new JRSRawDecoder();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
new file mode 100644
index 0000000..24fa637
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java
@@ -0,0 +1,69 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw erasure decoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible.
+ */
+public class RSRawDecoder extends AbstractRawErasureDecoder {
+  // To describe and calculate the needed Vandermonde matrix
+  private int[] errSignature;
+  private int[] primitivePower;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
+    super.initialize(numDataUnits, numParityUnits, chunkSize);
+    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+
+    this.errSignature = new int[getNumParityUnits()];
+    this.primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
+        getNumParityUnits());
+  }
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+    }
+
+    int dataLen = inputs[0].remaining();
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
+        erasedIndexes.length, dataLen);
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
+                          byte[][] outputs) {
+    for (int i = 0; i < erasedIndexes.length; i++) {
+      errSignature[i] = primitivePower[erasedIndexes[i]];
+      RSUtil.GF.substitute(inputs, outputs[i], primitivePower[i]);
+    }
+
+    int dataLen = inputs[0].length;
+    RSUtil.GF.solveVandermondeSystem(errSignature, outputs,
+        erasedIndexes.length, dataLen);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java
new file mode 100644
index 0000000..7b501ce
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw erasure encoder in RS code scheme in pure Java in case native one
+ * isn't available in some environment. Please always use native implementations
+ * when possible.
+ */
+public class RSRawEncoder extends AbstractRawErasureEncoder {
+  private int[] generatingPolynomial;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize) {
+    super.initialize(numDataUnits, numParityUnits, chunkSize);
+    assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize());
+
+    int[] primitivePower = RSUtil.getPrimitivePower(getNumDataUnits(),
+        getNumParityUnits());
+    // compute generating polynomial
+    int[] gen = {1};
+    int[] poly = new int[2];
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      poly[0] = primitivePower[i];
+      poly[1] = 1;
+      gen = RSUtil.GF.multiply(gen, poly);
+    }
+    // generating polynomial has all generating roots
+    generatingPolynomial = gen;
+  }
+
+  @Override
+  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    ByteBuffer[] data = new ByteBuffer[getNumDataUnits() + getNumParityUnits()];
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      data[i] = outputs[i];
+    }
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      data[i + getNumParityUnits()] = inputs[i];
+    }
+
+    // Compute the remainder
+    RSUtil.GF.remainder(data, generatingPolynomial);
+  }
+
+  @Override
+  protected void doEncode(byte[][] inputs, byte[][] outputs) {
+    byte[][] data = new byte[getNumDataUnits() + getNumParityUnits()][];
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      data[i] = outputs[i];
+    }
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      data[i + getNumParityUnits()] = inputs[i];
+    }
+
+    // Compute the remainder
+    RSUtil.GF.remainder(data, generatingPolynomial);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
new file mode 100644
index 0000000..19a95af
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawErasureCoderFactory.java
@@ -0,0 +1,34 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+/**
+ * A raw coder factory for raw Reed-Solomon coder in Java.
+ */
+public class RSRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder() {
+    return new RSRawEncoder();
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder() {
+    return new RSRawDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
new file mode 100644
index 0000000..b6b1633
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java
@@ -0,0 +1,81 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ */
+public class XORRawDecoder extends AbstractRawErasureDecoder {
+
+  @Override
+  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
+                          ByteBuffer[] outputs) {
+    assert(erasedIndexes.length == outputs.length);
+    assert(erasedIndexes.length <= 1);
+
+    int bufSize = inputs[0].remaining();
+    int erasedIdx = erasedIndexes[0];
+
+    // Set the output to zeros.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0].put(j, (byte) 0);
+    }
+
+    // Process the inputs.
+    for (int i = 0; i < inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (int j = 0; j < bufSize; j++) {
+        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
+      }
+    }
+  }
+
+  @Override
+  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
+                          byte[][] outputs) {
+    assert(erasedIndexes.length == outputs.length);
+    assert(erasedIndexes.length <= 1);
+
+    int bufSize = inputs[0].length;
+    int erasedIdx = erasedIndexes[0];
+
+    // Set the output to zeros.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0][j] = 0;
+    }
+
+    // Process the inputs.
+    for (int i = 0; i < inputs.length; i++) {
+      // Skip the erased location.
+      if (i == erasedIdx) {
+        continue;
+      }
+
+      for (int j = 0; j < bufSize; j++) {
+        outputs[0][j] ^= inputs[i][j];
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
new file mode 100644
index 0000000..dbfab5d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.java
@@ -0,0 +1,61 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import java.nio.ByteBuffer;
+
+/**
+ * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
+ */
+public class XORRawEncoder extends AbstractRawErasureEncoder {
+
+  @Override
+  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
+    int bufSize = inputs[0].remaining();
+
+    // Get the first buffer's data.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0].put(j, inputs[0].get(j));
+    }
+
+    // XOR with everything else.
+    for (int i = 1; i < inputs.length; i++) {
+      for (int j = 0; j < bufSize; j++) {
+        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
+      }
+    }
+  }
+
+  @Override
+  protected void doEncode(byte[][] inputs, byte[][] outputs) {
+    int bufSize = inputs[0].length;
+
+    // Get the first buffer's data.
+    for (int j = 0; j < bufSize; j++) {
+      outputs[0][j] = inputs[0][j];
+    }
+
+    // XOR with everything else.
+    for (int i = 1; i < inputs.length; i++) {
+      for (int j = 0; j < bufSize; j++) {
+        outputs[0][j] ^= inputs[i][j];
+      }
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
new file mode 100644
index 0000000..67f45c1
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawErasureCoderFactory.java
@@ -0,0 +1,34 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+/**
+ * A raw coder factory for raw XOR coder.
+ */
+public class XORRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder() {
+    return new XORRawEncoder();
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder() {
+    return new XORRawDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java
deleted file mode 100644
index 98307a7..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawDecoder.java
+++ /dev/null
@@ -1,81 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw decoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
- */
-public class XorRawDecoder extends AbstractRawErasureDecoder {
-
-  @Override
-  protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes,
-                          ByteBuffer[] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
-
-    int bufSize = inputs[0].remaining();
-    int erasedIdx = erasedIndexes[0];
-
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0].put(j, (byte) 0);
-    }
-
-    // Process the inputs.
-    for (int i = 0; i < inputs.length; i++) {
-      // Skip the erased location.
-      if (i == erasedIdx) {
-        continue;
-      }
-
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
-      }
-    }
-  }
-
-  @Override
-  protected void doDecode(byte[][] inputs, int[] erasedIndexes,
-                          byte[][] outputs) {
-    assert(erasedIndexes.length == outputs.length);
-    assert(erasedIndexes.length <= 1);
-
-    int bufSize = inputs[0].length;
-    int erasedIdx = erasedIndexes[0];
-
-    // Set the output to zeros.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0][j] = 0;
-    }
-
-    // Process the inputs.
-    for (int i = 0; i < inputs.length; i++) {
-      // Skip the erased location.
-      if (i == erasedIdx) {
-        continue;
-      }
-
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0][j] ^= inputs[i][j];
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java
deleted file mode 100644
index 99b20b9..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawEncoder.java
+++ /dev/null
@@ -1,61 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-import java.nio.ByteBuffer;
-
-/**
- * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID.
- */
-public class XorRawEncoder extends AbstractRawErasureEncoder {
-
-  @Override
-  protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) {
-    int bufSize = inputs[0].remaining();
-
-    // Get the first buffer's data.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0].put(j, inputs[0].get(j));
-    }
-
-    // XOR with everything else.
-    for (int i = 1; i < inputs.length; i++) {
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0].put(j, (byte) (outputs[0].get(j) ^ inputs[i].get(j)));
-      }
-    }
-  }
-
-  @Override
-  protected void doEncode(byte[][] inputs, byte[][] outputs) {
-    int bufSize = inputs[0].length;
-
-    // Get the first buffer's data.
-    for (int j = 0; j < bufSize; j++) {
-      outputs[0][j] = inputs[0][j];
-    }
-
-    // XOR with everything else.
-    for (int i = 1; i < inputs.length; i++) {
-      for (int j = 0; j < bufSize; j++) {
-        outputs[0][j] ^= inputs[i][j];
-      }
-    }
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java
deleted file mode 100644
index 751d16f..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java
+++ /dev/null
@@ -1,34 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-/**
- * A raw coder factory for raw XOR coder.
- */
-public class XorRawErasureCoderFactory implements RawErasureCoderFactory {
-
-  @Override
-  public RawErasureEncoder createEncoder() {
-    return new XorRawEncoder();
-  }
-
-  @Override
-  public RawErasureDecoder createDecoder() {
-    return new XorRawDecoder();
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
index 8a7561c..3507dd2 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
@@ -19,7 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
-import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawErasureCoderFactory;
 import org.junit.Before;
 import org.junit.Test;
 
@@ -58,7 +58,7 @@ public class TestRSErasureCoder extends TestErasureCoderBase {
      */
     Configuration conf = new Configuration();
     conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
-        JRSRawErasureCoderFactory.class.getCanonicalName());
+        RSRawErasureCoderFactory.class.getCanonicalName());
     conf.setBoolean(
         CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
     prepare(conf, 10, 4, null);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
new file mode 100644
index 0000000..109e46e
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java
@@ -0,0 +1,50 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test XOR encoding and decoding.
+ */
+public class TestXORCoder extends TestErasureCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = XORErasureEncoder.class;
+    this.decoderClass = XORErasureDecoder.class;
+
+    this.numDataUnits = 10;
+    this.numParityUnits = 1;
+    this.erasedDataIndexes = new int[] {0};
+
+    this.numChunksInBlock = 10;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer() {
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer() {
+    testCoding(true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java
deleted file mode 100644
index d46fe06..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java
+++ /dev/null
@@ -1,50 +0,0 @@
-/**
- * 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.io.erasurecode.coder;
-
-import org.junit.Before;
-import org.junit.Test;
-
-/**
- * Test XOR encoding and decoding.
- */
-public class TestXorCoder extends TestErasureCoderBase {
-
-  @Before
-  public void setup() {
-    this.encoderClass = XorErasureEncoder.class;
-    this.decoderClass = XorErasureDecoder.class;
-
-    this.numDataUnits = 10;
-    this.numParityUnits = 1;
-    this.erasedDataIndexes = new int[] {0};
-
-    this.numChunksInBlock = 10;
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer() {
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer() {
-    testCoding(true);
-  }
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
deleted file mode 100644
index 39e5deb..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
+++ /dev/null
@@ -1,93 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-import org.apache.hadoop.io.erasurecode.ECChunk;
-import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.nio.ByteBuffer;
-
-/**
- * Test raw Reed-solomon encoding and decoding.
- */
-public class TestJRSRawCoder extends TestRawCoderBase {
-
-  private static int symbolSize = 0;
-  private static int symbolMax = 0;
-
-  static {
-    symbolSize = (int) Math.round(Math.log(
-        RSUtil.GF.getFieldSize()) / Math.log(2));
-    symbolMax = (int) Math.pow(2, symbolSize);
-  }
-
-  @Before
-  public void setup() {
-    this.encoderClass = JRSRawEncoder.class;
-    this.decoderClass = JRSRawDecoder.class;
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4() {
-    prepare(null, 10, 4, null);
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
-    prepare(null, 10, 4, new int[] {2, 4});
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_10x4_erasing_all() {
-    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
-    testCoding(true);
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer_3x3() {
-    prepare(null, 3, 3, null);
-    testCoding(true);
-  }
-
-  @Override
-  protected ECChunk generateDataChunk() {
-    ByteBuffer buffer = allocateOutputBuffer();
-    for (int i = 0; i < chunkSize; i++) {
-      buffer.put((byte) RAND.nextInt(symbolMax));
-    }
-    buffer.flip();
-
-    return new ECChunk(buffer);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
new file mode 100644
index 0000000..8bb5d0f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java
@@ -0,0 +1,93 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.util.RSUtil;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.nio.ByteBuffer;
+
+/**
+ * Test raw Reed-solomon encoding and decoding.
+ */
+public class TestRSRawCoder extends TestRawCoderBase {
+
+  private static int symbolSize = 0;
+  private static int symbolMax = 0;
+
+  static {
+    symbolSize = (int) Math.round(Math.log(
+        RSUtil.GF.getFieldSize()) / Math.log(2));
+    symbolMax = (int) Math.pow(2, symbolSize);
+  }
+
+  @Before
+  public void setup() {
+    this.encoderClass = RSRawEncoder.class;
+    this.decoderClass = RSRawDecoder.class;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+    prepare(null, 10, 4, new int[] {2, 4});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_all() {
+    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(true);
+  }
+
+  @Override
+  protected ECChunk generateDataChunk() {
+    ByteBuffer buffer = allocateOutputBuffer();
+    for (int i = 0; i < chunkSize; i++) {
+      buffer.put((byte) RAND.nextInt(symbolMax));
+    }
+    buffer.flip();
+
+    return new ECChunk(buffer);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
new file mode 100644
index 0000000..e66e48d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java
@@ -0,0 +1,49 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test XOR encoding and decoding.
+ */
+public class TestXORRawCoder extends TestRawCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = XORRawEncoder.class;
+    this.decoderClass = XORRawDecoder.class;
+
+    this.numDataUnits = 10;
+    this.numParityUnits = 1;
+
+    this.erasedDataIndexes = new int[] {0};
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer() {
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer() {
+    testCoding(true);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/0993f530/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
deleted file mode 100644
index ff48586..0000000
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXorRawCoder.java
+++ /dev/null
@@ -1,51 +0,0 @@
-/**
- * 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.io.erasurecode.rawcoder;
-
-import org.junit.Before;
-import org.junit.Test;
-
-import java.util.Random;
-
-/**
- * Test XOR encoding and decoding.
- */
-public class TestXorRawCoder extends TestRawCoderBase {
-
-  @Before
-  public void setup() {
-    this.encoderClass = XorRawEncoder.class;
-    this.decoderClass = XorRawDecoder.class;
-
-    this.numDataUnits = 10;
-    this.numParityUnits = 1;
-
-    this.erasedDataIndexes = new int[] {0};
-  }
-
-  @Test
-  public void testCodingNoDirectBuffer() {
-    testCoding(false);
-  }
-
-  @Test
-  public void testCodingDirectBuffer() {
-    testCoding(true);
-  }
-
-}


[14/50] [abbrv] hadoop git commit: Updated CHANGES-HDFS-EC-7285.txt accordingly

Posted by zh...@apache.org.
Updated CHANGES-HDFS-EC-7285.txt accordingly


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e692325c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e692325c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e692325c

Branch: refs/heads/HDFS-7285
Commit: e692325c4c63b73e8381e2d783046c708d7d5a57
Parents: 068a5d5
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Mar 18 19:24:24 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:42:00 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 5 ++++-
 1 file changed, 4 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e692325c/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index a97dc34..e27ff5c 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -19,6 +19,9 @@
     ( Kai Zheng via vinayakumarb )
 
     HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng
-    ( Kai Zheng )    
+    ( Kai Zheng )
+
+    HADOOP-11706. Refine a little bit erasure coder API. Contributed by Kai Zheng
+    ( Kai Zheng )
 
 


[11/50] [abbrv] hadoop git commit: HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7912. Erasure Coding: track BlockInfo instead of Block in UnderReplicatedBlocks and PendingReplicationBlocks. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c3b03388
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c3b03388
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c3b03388

Branch: refs/heads/HDFS-7285
Commit: c3b03388d9f3bd3928b2dac808329e77e799f863
Parents: 5c7ab8b
Author: Jing Zhao <ji...@apache.org>
Authored: Tue Mar 17 10:18:50 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:41:59 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 47 ++++++++---------
 .../PendingReplicationBlocks.java               | 51 +++++++++----------
 .../blockmanagement/UnderReplicatedBlocks.java  | 49 +++++++++---------
 .../hdfs/server/namenode/FSDirAttrOp.java       | 10 ++--
 .../hdfs/server/namenode/FSNamesystem.java      | 21 ++++----
 .../hadoop/hdfs/server/namenode/INode.java      | 12 ++---
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  4 +-
 .../hdfs/server/namenode/NamenodeFsck.java      | 10 ++--
 .../hadoop/hdfs/server/namenode/SafeMode.java   |  3 +-
 .../blockmanagement/BlockManagerTestUtil.java   |  5 +-
 .../blockmanagement/TestBlockManager.java       |  8 +--
 .../server/blockmanagement/TestNodeCount.java   |  3 +-
 .../TestOverReplicatedBlocks.java               |  5 +-
 .../blockmanagement/TestPendingReplication.java | 19 ++++---
 .../TestRBWBlockInvalidation.java               |  4 +-
 .../blockmanagement/TestReplicationPolicy.java  | 53 +++++++++++---------
 .../TestUnderReplicatedBlockQueues.java         | 16 +++---
 .../datanode/TestReadOnlySharedStorage.java     |  9 ++--
 .../namenode/TestProcessCorruptBlocks.java      |  5 +-
 19 files changed, 180 insertions(+), 154 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index b0ac1c3..25a2446 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1336,7 +1336,7 @@ public class BlockManager {
    * @return number of blocks scheduled for replication during this iteration.
    */
   int computeReplicationWork(int blocksToProcess) {
-    List<List<Block>> blocksToReplicate = null;
+    List<List<BlockInfo>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
       // Choose the blocks to be replicated
@@ -1354,7 +1354,7 @@ public class BlockManager {
    * @return the number of blocks scheduled for replication
    */
   @VisibleForTesting
-  int computeReplicationWorkForBlocks(List<List<Block>> blocksToReplicate) {
+  int computeReplicationWorkForBlocks(List<List<BlockInfo>> blocksToReplicate) {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
     DatanodeDescriptor srcNode;
@@ -1368,7 +1368,7 @@ public class BlockManager {
     try {
       synchronized (neededReplications) {
         for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
-          for (Block block : blocksToReplicate.get(priority)) {
+          for (BlockInfo block : blocksToReplicate.get(priority)) {
             // block should belong to a file
             bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
@@ -1452,7 +1452,7 @@ public class BlockManager {
         }
 
         synchronized (neededReplications) {
-          Block block = rw.block;
+          BlockInfo block = rw.block;
           int priority = rw.priority;
           // Recheck since global lock was released
           // block should belong to a file
@@ -1714,7 +1714,7 @@ public class BlockManager {
    * and put them back into the neededReplication queue
    */
   private void processPendingReplications() {
-    Block[] timedOutItems = pendingReplications.getTimedOutBlocks();
+    BlockInfo[] timedOutItems = pendingReplications.getTimedOutBlocks();
     if (timedOutItems != null) {
       namesystem.writeLock();
       try {
@@ -2887,13 +2887,13 @@ public class BlockManager {
   
   /** Set replication for the blocks. */
   public void setReplication(final short oldRepl, final short newRepl,
-      final String src, final Block... blocks) {
+      final String src, final BlockInfoContiguous... blocks) {
     if (newRepl == oldRepl) {
       return;
     }
 
     // update needReplication priority queues
-    for(Block b : blocks) {
+    for(BlockInfoContiguous b : blocks) {
       updateNeededReplications(b, 0, newRepl-oldRepl);
     }
       
@@ -2901,7 +2901,7 @@ public class BlockManager {
       // old replication > the new one; need to remove copies
       LOG.info("Decreasing replication from " + oldRepl + " to " + newRepl
           + " for " + src);
-      for(Block b : blocks) {
+      for(BlockInfoContiguous b : blocks) {
         processOverReplicatedBlock(b, newRepl, null, null);
       }
     } else { // replication factor is increased
@@ -3073,7 +3073,8 @@ public class BlockManager {
     blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node);
     assert (namesystem.hasWriteLock());
     {
-      if (!blocksMap.removeNode(block, node)) {
+      BlockInfo storedBlock = getStoredBlock(block);
+      if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
         blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
             " removed from node {}", block, node);
         return;
@@ -3087,8 +3088,8 @@ public class BlockManager {
       //
       BlockCollection bc = blocksMap.getBlockCollection(block);
       if (bc != null) {
-        namesystem.decrementSafeBlockCount(block);
-        updateNeededReplications(block, -1, 0);
+        namesystem.decrementSafeBlockCount(storedBlock);
+        updateNeededReplications(storedBlock, -1, 0);
       }
 
       //
@@ -3162,7 +3163,7 @@ public class BlockManager {
     //
     // Modify the blocks->datanode map and node's map.
     //
-    pendingReplications.decrement(block, node);
+    pendingReplications.decrement(getStoredBlock(block), node);
     processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED,
         delHintNode);
   }
@@ -3277,7 +3278,7 @@ public class BlockManager {
    * For a striped block, this includes nodes storing blocks belonging to the
    * striped block group.
    */
-  public NumberReplicas countNodes(Block b) {
+  public NumberReplicas countNodes(BlockInfo b) {
     int decommissioned = 0;
     int decommissioning = 0;
     int live = 0;
@@ -3310,11 +3311,11 @@ public class BlockManager {
   }
 
   /** 
-   * Simpler, faster form of {@link #countNodes(Block)} that only returns the number
+   * Simpler, faster form of {@link #countNodes} that only returns the number
    * of live nodes.  If in startup safemode (or its 30-sec extension period),
    * then it gains speed by ignoring issues of excess replicas or nodes
    * that are decommissioned or in process of becoming decommissioned.
-   * If not in startup, then it calls {@link #countNodes(Block)} instead.
+   * If not in startup, then it calls {@link #countNodes} instead.
    * 
    * @param b - the block being tested
    * @return count of live nodes for this block
@@ -3344,10 +3345,10 @@ public class BlockManager {
     if (!namesystem.isPopulatingReplQueues()) {
       return;
     }
-    final Iterator<? extends Block> it = srcNode.getBlockIterator();
+    final Iterator<BlockInfo> it = srcNode.getBlockIterator();
     int numOverReplicated = 0;
     while(it.hasNext()) {
-      final Block block = it.next();
+      final BlockInfo block = it.next();
       BlockCollection bc = blocksMap.getBlockCollection(block);
       short expectedReplication = bc.getBlockReplication();
       NumberReplicas num = countNodes(block);
@@ -3411,7 +3412,7 @@ public class BlockManager {
     return blocksMap.size();
   }
 
-  public void removeBlock(Block block) {
+  public void removeBlock(BlockInfo block) {
     assert namesystem.hasWriteLock();
     // No need to ACK blocks that are being removed entirely
     // from the namespace, since the removal of the associated
@@ -3440,7 +3441,7 @@ public class BlockManager {
   }
 
   /** updates a block in under replication queue */
-  private void updateNeededReplications(final Block block,
+  private void updateNeededReplications(final BlockInfo block,
       final int curReplicasDelta, int expectedReplicasDelta) {
     namesystem.writeLock();
     try {
@@ -3472,7 +3473,7 @@ public class BlockManager {
    */
   public void checkReplication(BlockCollection bc) {
     final short expected = bc.getBlockReplication();
-    for (Block block : bc.getBlocks()) {
+    for (BlockInfo block : bc.getBlocks()) {
       final NumberReplicas n = countNodes(block);
       if (isNeededReplication(block, expected, n.liveReplicas())) { 
         neededReplications.add(block, n.liveReplicas(),
@@ -3649,7 +3650,7 @@ public class BlockManager {
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
    */
-  public Iterator<Block> getCorruptReplicaBlockIterator() {
+  public Iterator<BlockInfo> getCorruptReplicaBlockIterator() {
     return neededReplications.iterator(
         UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS);
   }
@@ -3763,7 +3764,7 @@ public class BlockManager {
 
   private static class ReplicationWork {
 
-    private final Block block;
+    private final BlockInfo block;
     private final BlockCollection bc;
 
     private final DatanodeDescriptor srcNode;
@@ -3774,7 +3775,7 @@ public class BlockManager {
     private DatanodeStorageInfo targets[];
     private final int priority;
 
-    public ReplicationWork(Block block,
+    public ReplicationWork(BlockInfo block,
         BlockCollection bc,
         DatanodeDescriptor srcNode,
         List<DatanodeDescriptor> containingNodes,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
index 796b878..04232cf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/PendingReplicationBlocks.java
@@ -23,6 +23,7 @@ import java.io.PrintWriter;
 import java.sql.Time;
 import java.util.ArrayList;
 import java.util.Arrays;
+import java.util.Collections;
 import java.util.HashMap;
 import java.util.Iterator;
 import java.util.List;
@@ -46,8 +47,8 @@ import org.slf4j.Logger;
 class PendingReplicationBlocks {
   private static final Logger LOG = BlockManager.LOG;
 
-  private final Map<Block, PendingBlockInfo> pendingReplications;
-  private final ArrayList<Block> timedOutItems;
+  private final Map<BlockInfo, PendingBlockInfo> pendingReplications;
+  private final ArrayList<BlockInfo> timedOutItems;
   Daemon timerThread = null;
   private volatile boolean fsRunning = true;
 
@@ -62,8 +63,8 @@ class PendingReplicationBlocks {
     if ( timeoutPeriod > 0 ) {
       this.timeout = timeoutPeriod;
     }
-    pendingReplications = new HashMap<Block, PendingBlockInfo>();
-    timedOutItems = new ArrayList<Block>();
+    pendingReplications = new HashMap<>();
+    timedOutItems = new ArrayList<>();
   }
 
   void start() {
@@ -76,7 +77,7 @@ class PendingReplicationBlocks {
    * @param block The corresponding block
    * @param targets The DataNodes where replicas of the block should be placed
    */
-  void increment(Block block, DatanodeDescriptor[] targets) {
+  void increment(BlockInfo block, DatanodeDescriptor[] targets) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found == null) {
@@ -93,9 +94,9 @@ class PendingReplicationBlocks {
    * Decrement the number of pending replication requests
    * for this block.
    * 
-   * @param The DataNode that finishes the replication
+   * @param dn The DataNode that finishes the replication
    */
-  void decrement(Block block, DatanodeDescriptor dn) {
+  void decrement(BlockInfo block, DatanodeDescriptor dn) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -115,7 +116,7 @@ class PendingReplicationBlocks {
    * @param block The given block whose pending replication requests need to be
    *              removed
    */
-  void remove(Block block) {
+  void remove(BlockInfo block) {
     synchronized (pendingReplications) {
       pendingReplications.remove(block);
     }
@@ -138,7 +139,7 @@ class PendingReplicationBlocks {
   /**
    * How many copies of this block is pending replication?
    */
-  int getNumReplicas(Block block) {
+  int getNumReplicas(BlockInfo block) {
     synchronized (pendingReplications) {
       PendingBlockInfo found = pendingReplications.get(block);
       if (found != null) {
@@ -153,13 +154,13 @@ class PendingReplicationBlocks {
    * replication requests. Returns null if no blocks have
    * timed out.
    */
-  Block[] getTimedOutBlocks() {
+  BlockInfo[] getTimedOutBlocks() {
     synchronized (timedOutItems) {
       if (timedOutItems.size() <= 0) {
         return null;
       }
-      Block[] blockList = timedOutItems.toArray(
-          new Block[timedOutItems.size()]);
+      BlockInfo[] blockList = timedOutItems.toArray(
+          new BlockInfo[timedOutItems.size()]);
       timedOutItems.clear();
       return blockList;
     }
@@ -179,7 +180,7 @@ class PendingReplicationBlocks {
     PendingBlockInfo(DatanodeDescriptor[] targets) {
       this.timeStamp = monotonicNow();
       this.targets = targets == null ? new ArrayList<DatanodeDescriptor>()
-          : new ArrayList<DatanodeDescriptor>(Arrays.asList(targets));
+          : new ArrayList<>(Arrays.asList(targets));
     }
 
     long getTimeStamp() {
@@ -192,9 +193,7 @@ class PendingReplicationBlocks {
 
     void incrementReplicas(DatanodeDescriptor... newTargets) {
       if (newTargets != null) {
-        for (DatanodeDescriptor dn : newTargets) {
-          targets.add(dn);
-        }
+        Collections.addAll(targets, newTargets);
       }
     }
 
@@ -232,17 +231,17 @@ class PendingReplicationBlocks {
      */
     void pendingReplicationCheck() {
       synchronized (pendingReplications) {
-        Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
+        Iterator<Map.Entry<BlockInfo, PendingBlockInfo>> iter =
                                     pendingReplications.entrySet().iterator();
         long now = monotonicNow();
         if(LOG.isDebugEnabled()) {
           LOG.debug("PendingReplicationMonitor checking Q");
         }
         while (iter.hasNext()) {
-          Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+          Map.Entry<BlockInfo, PendingBlockInfo> entry = iter.next();
           PendingBlockInfo pendingBlock = entry.getValue();
           if (now > pendingBlock.getTimeStamp() + timeout) {
-            Block block = entry.getKey();
+            BlockInfo block = entry.getKey();
             synchronized (timedOutItems) {
               timedOutItems.add(block);
             }
@@ -275,16 +274,14 @@ class PendingReplicationBlocks {
     synchronized (pendingReplications) {
       out.println("Metasave: Blocks being replicated: " +
                   pendingReplications.size());
-      Iterator<Map.Entry<Block, PendingBlockInfo>> iter =
-                                  pendingReplications.entrySet().iterator();
-      while (iter.hasNext()) {
-        Map.Entry<Block, PendingBlockInfo> entry = iter.next();
+      for (Map.Entry<BlockInfo, PendingBlockInfo> entry :
+          pendingReplications.entrySet()) {
         PendingBlockInfo pendingBlock = entry.getValue();
         Block block = entry.getKey();
-        out.println(block + 
-                    " StartTime: " + new Time(pendingBlock.timeStamp) +
-                    " NumReplicaInProgress: " + 
-                    pendingBlock.getNumReplicas());
+        out.println(block +
+            " StartTime: " + new Time(pendingBlock.timeStamp) +
+            " NumReplicaInProgress: " +
+            pendingBlock.getNumReplicas());
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
index 1daa0ee..f9bce26 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/UnderReplicatedBlocks.java
@@ -20,7 +20,6 @@ package org.apache.hadoop.hdfs.server.blockmanagement;
 import java.util.ArrayList;
 import java.util.Iterator;
 import java.util.List;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.util.LightWeightLinkedSet;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
@@ -35,7 +34,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *
  * <p/>
  * The policy for choosing which priority to give added blocks
- * is implemented in {@link #getPriority(Block, int, int, int)}.
+ * is implemented in {@link #getPriority(int, int, int)}.
  * </p>
  * <p>The queue order is as follows:</p>
  * <ol>
@@ -62,7 +61,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  *   blocks that are not corrupt higher priority.</li>
  * </ol>
  */
-class UnderReplicatedBlocks implements Iterable<Block> {
+class UnderReplicatedBlocks implements Iterable<BlockInfo> {
   /** The total number of queues : {@value} */
   static final int LEVEL = 5;
   /** The queue with the highest priority: {@value} */
@@ -78,8 +77,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** The queue for corrupt blocks: {@value} */
   static final int QUEUE_WITH_CORRUPT_BLOCKS = 4;
   /** the queues themselves */
-  private final List<LightWeightLinkedSet<Block>> priorityQueues
-      = new ArrayList<LightWeightLinkedSet<Block>>(LEVEL);
+  private final List<LightWeightLinkedSet<BlockInfo>> priorityQueues
+      = new ArrayList<>(LEVEL);
 
   /** The number of corrupt blocks with replication factor 1 */
   private int corruptReplOneBlocks = 0;
@@ -87,7 +86,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /** Create an object. */
   UnderReplicatedBlocks() {
     for (int i = 0; i < LEVEL; i++) {
-      priorityQueues.add(new LightWeightLinkedSet<Block>());
+      priorityQueues.add(new LightWeightLinkedSet<BlockInfo>());
     }
   }
 
@@ -131,8 +130,8 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Check if a block is in the neededReplication queue */
-  synchronized boolean contains(Block block) {
-    for(LightWeightLinkedSet<Block> set : priorityQueues) {
+  synchronized boolean contains(BlockInfo block) {
+    for(LightWeightLinkedSet<BlockInfo> set : priorityQueues) {
       if (set.contains(block)) {
         return true;
       }
@@ -141,13 +140,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** Return the priority of a block
-   * @param block a under replicated block
    * @param curReplicas current number of replicas of the block
    * @param expectedReplicas expected number of replicas of the block
    * @return the priority for the blocks, between 0 and ({@link #LEVEL}-1)
    */
-  private int getPriority(Block block,
-                          int curReplicas, 
+  private int getPriority(int curReplicas,
                           int decommissionedReplicas,
                           int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
@@ -183,12 +180,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param expectedReplicas expected number of replicas of the block
    * @return true if the block was added to a queue.
    */
-  synchronized boolean add(Block block,
+  synchronized boolean add(BlockInfo block,
                            int curReplicas, 
                            int decomissionedReplicas,
                            int expectedReplicas) {
     assert curReplicas >= 0 : "Negative replicas!";
-    int priLevel = getPriority(block, curReplicas, decomissionedReplicas,
+    int priLevel = getPriority(curReplicas, decomissionedReplicas,
                                expectedReplicas);
     if(priorityQueues.get(priLevel).add(block)) {
       if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS &&
@@ -207,11 +204,11 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   }
 
   /** remove a block from a under replication queue */
-  synchronized boolean remove(Block block, 
+  synchronized boolean remove(BlockInfo block,
                               int oldReplicas, 
                               int decommissionedReplicas,
                               int oldExpectedReplicas) {
-    int priLevel = getPriority(block, oldReplicas, 
+    int priLevel = getPriority(oldReplicas,
                                decommissionedReplicas,
                                oldExpectedReplicas);
     boolean removedBlock = remove(block, priLevel);
@@ -241,7 +238,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param priLevel expected privilege level
    * @return true if the block was found and removed from one of the priority queues
    */
-  boolean remove(Block block, int priLevel) {
+  boolean remove(BlockInfo block, int priLevel) {
     if(priLevel >= 0 && priLevel < LEVEL 
         && priorityQueues.get(priLevel).remove(block)) {
       NameNode.blockStateChangeLog.debug(
@@ -279,14 +276,14 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @param curReplicasDelta the change in the replicate count from before
    * @param expectedReplicasDelta the change in the expected replica count from before
    */
-  synchronized void update(Block block, int curReplicas,
+  synchronized void update(BlockInfo block, int curReplicas,
                            int decommissionedReplicas,
                            int curExpectedReplicas,
                            int curReplicasDelta, int expectedReplicasDelta) {
     int oldReplicas = curReplicas-curReplicasDelta;
     int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta;
-    int curPri = getPriority(block, curReplicas, decommissionedReplicas, curExpectedReplicas);
-    int oldPri = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas);
+    int curPri = getPriority(curReplicas, decommissionedReplicas, curExpectedReplicas);
+    int oldPri = getPriority(oldReplicas, decommissionedReplicas, oldExpectedReplicas);
     if(NameNode.stateChangeLog.isDebugEnabled()) {
       NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 
         block +
@@ -336,12 +333,12 @@ class UnderReplicatedBlocks implements Iterable<Block> {
    * @return Return a list of block lists to be replicated. The block list index
    *         represents its replication priority.
    */
-  public synchronized List<List<Block>> chooseUnderReplicatedBlocks(
+  public synchronized List<List<BlockInfo>> chooseUnderReplicatedBlocks(
       int blocksToProcess) {
     // initialize data structure for the return value
-    List<List<Block>> blocksToReplicate = new ArrayList<List<Block>>(LEVEL);
+    List<List<BlockInfo>> blocksToReplicate = new ArrayList<>(LEVEL);
     for (int i = 0; i < LEVEL; i++) {
-      blocksToReplicate.add(new ArrayList<Block>());
+      blocksToReplicate.add(new ArrayList<BlockInfo>());
     }
 
     if (size() == 0) { // There are no blocks to collect.
@@ -364,7 +361,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
       // Loop through all remaining blocks in the list.
       while (blockCount < blocksToProcess
           && neededReplicationsIterator.hasNext()) {
-        Block block = neededReplicationsIterator.next();
+        BlockInfo block = neededReplicationsIterator.next();
         blocksToReplicate.get(priority).add(block);
         blockCount++;
       }
@@ -396,10 +393,10 @@ class UnderReplicatedBlocks implements Iterable<Block> {
   /**
    * An iterator over blocks.
    */
-  class BlockIterator implements Iterator<Block> {
+  class BlockIterator implements Iterator<BlockInfo> {
     private int level;
     private boolean isIteratorForLevel = false;
-    private final List<Iterator<Block>> iterators = new ArrayList<Iterator<Block>>();
+    private final List<Iterator<BlockInfo>> iterators = new ArrayList<>();
 
     /**
      * Construct an iterator over all queues.
@@ -431,7 +428,7 @@ class UnderReplicatedBlocks implements Iterable<Block> {
     }
 
     @Override
-    public Block next() {
+    public BlockInfo next() {
       if (isIteratorForLevel) {
         return iterators.get(0).next();
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
index a3881b8..eefbcce 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAttrOp.java
@@ -31,6 +31,8 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.util.EnumCounters;
@@ -148,8 +150,8 @@ public class FSDirAttrOp {
       }
 
       final short[] blockRepls = new short[2]; // 0: old, 1: new
-      final Block[] blocks = unprotectedSetReplication(fsd, src, replication,
-                                                       blockRepls);
+      final BlockInfoContiguous[] blocks = unprotectedSetReplication(fsd, src,
+          replication, blockRepls);
       isFile = blocks != null;
       if (isFile) {
         fsd.getEditLog().logSetReplication(src, replication);
@@ -375,7 +377,7 @@ public class FSDirAttrOp {
     }
   }
 
-  static Block[] unprotectedSetReplication(
+  static BlockInfoContiguous[] unprotectedSetReplication(
       FSDirectory fsd, String src, short replication, short[] blockRepls)
       throws QuotaExceededException, UnresolvedLinkException,
              SnapshotAccessControlException {
@@ -410,7 +412,7 @@ public class FSDirAttrOp {
       blockRepls[0] = oldBR;
       blockRepls[1] = newBR;
     }
-    return file.getBlocks();
+    return file.getContiguousBlocks();
   }
 
   static void unprotectedSetStoragePolicy(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 14e97a1..f1ccb1a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3742,8 +3742,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    *          of blocks that need to be removed from blocksMap
    */
   void removeBlocks(BlocksMapUpdateInfo blocks) {
-    List<Block> toDeleteList = blocks.getToDeleteList();
-    Iterator<Block> iter = toDeleteList.iterator();
+    List<BlockInfo> toDeleteList = blocks.getToDeleteList();
+    Iterator<BlockInfo> iter = toDeleteList.iterator();
     while (iter.hasNext()) {
       writeLock();
       try {
@@ -3798,12 +3798,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean trackBlockCounts = isSafeModeTrackingBlocks();
     int numRemovedComplete = 0, numRemovedSafe = 0;
 
-    for (Block b : blocks.getToDeleteList()) {
+    for (BlockInfo b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfo bi = getStoredBlock(b);
-        if (bi.isComplete()) {
+        if (b.isComplete()) {
           numRemovedComplete++;
-          if (blockManager.checkMinStorage(bi, bi.numNodes())) {
+          if (blockManager.checkMinStorage(b, b.numNodes())) {
             numRemovedSafe++;
           }
         }
@@ -4732,7 +4731,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       writeLock();
 
       try {
-        final Iterator<Block> it = blockManager.getCorruptReplicaBlockIterator();
+        final Iterator<BlockInfo> it =
+            blockManager.getCorruptReplicaBlockIterator();
 
         while (it.hasNext()) {
           Block b = it.next();
@@ -5682,7 +5682,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   }
 
   @Override
-  public void decrementSafeBlockCount(Block b) {
+  public void decrementSafeBlockCount(BlockInfo b) {
     // safeMode is volatile, and may be set to null at any time
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
@@ -6547,7 +6547,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
       // print a limited # of corrupt files per call
 
-      final Iterator<Block> blkIterator = blockManager.getCorruptReplicaBlockIterator();
+      final Iterator<BlockInfo> blkIterator =
+          blockManager.getCorruptReplicaBlockIterator();
 
       int skip = getIntCookie(cookieTab[0]);
       for (int i = 0; i < skip && blkIterator.hasNext(); i++) {
@@ -6555,7 +6556,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
 
       while (blkIterator.hasNext()) {
-        Block blk = blkIterator.next();
+        BlockInfo blk = blkIterator.next();
         final INode inode = (INode)blockManager.getBlockCollection(blk);
         skip++;
         if (inode != null && blockManager.countNodes(blk).liveReplicas() == 0) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
index e629441..63a1a32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INode.java
@@ -30,9 +30,9 @@ import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.DFSUtil;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.DstReference;
 import org.apache.hadoop.hdfs.server.namenode.INodeReference.WithName;
@@ -827,16 +827,16 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
     /**
      * The list of blocks that need to be removed from blocksMap
      */
-    private final List<Block> toDeleteList;
+    private final List<BlockInfo> toDeleteList;
     
     public BlocksMapUpdateInfo() {
-      toDeleteList = new ChunkedArrayList<Block>();
+      toDeleteList = new ChunkedArrayList<>();
     }
     
     /**
      * @return The list of blocks that need to be removed from blocksMap
      */
-    public List<Block> getToDeleteList() {
+    public List<BlockInfo> getToDeleteList() {
       return toDeleteList;
     }
     
@@ -845,12 +845,12 @@ public abstract class INode implements INodeAttributes, Diff.Element<byte[]> {
      * {@link BlocksMapUpdateInfo#toDeleteList}
      * @param toDelete the to-be-deleted block
      */
-    public void addDeleteBlock(Block toDelete) {
+    public void addDeleteBlock(BlockInfo toDelete) {
       assert toDelete != null : "toDelete is null";
       toDeleteList.add(toDelete);
     }
 
-    public void removeDeleteBlock(Block block) {
+    public void removeDeleteBlock(BlockInfo block) {
       assert block != null : "block is null";
       toDeleteList.remove(block);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 9cd7ddd..d0b4c32 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -1071,8 +1071,8 @@ public class INodeFile extends INodeWithAdditionalFields
         getDiffs().findEarlierSnapshotBlocks(snapshotId);
     if(snapshotBlocks == null)
       return;
-    List<Block> toDelete = collectedBlocks.getToDeleteList();
-    for(Block blk : snapshotBlocks) {
+    List<BlockInfo> toDelete = collectedBlocks.getToDeleteList();
+    for(BlockInfo blk : snapshotBlocks) {
       if(toDelete.contains(blk))
         collectedBlocks.removeDeleteBlock(blk);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index 5f5a1e0..49aa8a4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -244,7 +244,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       }
       BlockCollection bc = bm.getBlockCollection(blockInfo);
       INode iNode = (INode) bc;
-      NumberReplicas numberReplicas= bm.countNodes(block);
+      NumberReplicas numberReplicas= bm.countNodes(blockInfo);
       out.println("Block Id: " + blockId);
       out.println("Block belongs to: "+iNode.getFullPathName());
       out.println("No. of Expected Replica: " + bc.getBlockReplication());
@@ -461,7 +461,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
     // Get block locations without updating the file access time 
     // and without block access tokens
     LocatedBlocks blocks = null;
-    FSNamesystem fsn = namenode.getNamesystem();
+    final FSNamesystem fsn = namenode.getNamesystem();
     fsn.readLock();
     try {
       blocks = fsn.getBlockLocations(path, 0, fileLen, false, false).blocks;
@@ -509,8 +509,10 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       boolean isCorrupt = lBlk.isCorrupt();
       String blkName = block.toString();
       DatanodeInfo[] locs = lBlk.getLocations();
-      NumberReplicas numberReplicas =
-          namenode.getNamesystem().getBlockManager().countNodes(block.getLocalBlock());
+      final BlockManager blockManager = fsn.getBlockManager();
+      final BlockInfo storedBlock = blockManager.getStoredBlock(
+          block.getLocalBlock());
+      NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
       int liveReplicas = numberReplicas.liveReplicas();
       int decommissionedReplicas = numberReplicas.decommissioned();;
       int decommissioningReplicas = numberReplicas.decommissioning();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
index 95fc06b..0debb1f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/SafeMode.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 
 /** SafeMode related operations. */
 @InterfaceAudience.Private
@@ -49,5 +50,5 @@ public interface SafeMode {
   public void incrementSafeBlockCount(int replication);
 
   /** Decrement number of blocks that reached minimal replication. */
-  public void decrementSafeBlockCount(Block b);
+  public void decrementSafeBlockCount(BlockInfo b);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 23e610f..148135b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -69,9 +69,10 @@ public class BlockManagerTestUtil {
     final BlockManager bm = namesystem.getBlockManager();
     namesystem.readLock();
     try {
+      final BlockInfo storedBlock = bm.getStoredBlock(b);
       return new int[]{getNumberOfRacks(bm, b),
-          bm.countNodes(b).liveReplicas(),
-          bm.neededReplications.contains(b) ? 1 : 0};
+          bm.countNodes(storedBlock).liveReplicas(),
+          bm.neededReplications.contains(storedBlock) ? 1 : 0};
     } finally {
       namesystem.readUnlock();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 1bc6198..2fca5be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -435,14 +435,14 @@ public class TestBlockManager {
     return blockInfo;
   }
 
-  private DatanodeStorageInfo[] scheduleSingleReplication(Block block) {
+  private DatanodeStorageInfo[] scheduleSingleReplication(BlockInfo block) {
     // list for priority 1
-    List<Block> list_p1 = new ArrayList<Block>();
+    List<BlockInfo> list_p1 = new ArrayList<>();
     list_p1.add(block);
 
     // list of lists for each priority
-    List<List<Block>> list_all = new ArrayList<List<Block>>();
-    list_all.add(new ArrayList<Block>()); // for priority 0
+    List<List<BlockInfo>> list_all = new ArrayList<>();
+    list_all.add(new ArrayList<BlockInfo>()); // for priority 0
     list_all.add(list_p1); // for priority 1
 
     assertEquals("Block not initially pending replication", 0,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index c3726f2..1c3f075 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -166,10 +166,11 @@ public class TestNodeCount {
 
   /* threadsafe read of the replication counts for this block */
   NumberReplicas countNodes(Block block, FSNamesystem namesystem) {
+    BlockManager blockManager = namesystem.getBlockManager();
     namesystem.readLock();
     try {
       lastBlock = block;
-      lastNum = namesystem.getBlockManager().countNodes(block);
+      lastNum = blockManager.countNodes(blockManager.getStoredBlock(block));
       return lastNum;
     }
     finally {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
index a86b573..2d7bb44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
@@ -117,7 +117,8 @@ public class TestOverReplicatedBlocks {
 
           // corrupt one won't be chosen to be excess one
           // without 4910 the number of live replicas would be 0: block gets lost
-          assertEquals(1, bm.countNodes(block.getLocalBlock()).liveReplicas());
+          assertEquals(1, bm.countNodes(
+              bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
         }
       } finally {
         namesystem.writeUnlock();
@@ -219,7 +220,7 @@ public class TestOverReplicatedBlocks {
       out.close();
       ExtendedBlock block = DFSTestUtil.getFirstBlock(fs, p);
       assertEquals("Expected only one live replica for the block", 1, bm
-          .countNodes(block.getLocalBlock()).liveReplicas());
+          .countNodes(bm.getStoredBlock(block.getLocalBlock())).liveReplicas());
     } finally {
       cluster.shutdown();
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
index c63badc..b9032f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestPendingReplication.java
@@ -53,6 +53,11 @@ public class TestPendingReplication {
   // Number of datanodes in the cluster
   private static final int DATANODE_COUNT = 5;
 
+  private BlockInfo genBlockInfo(long id, long length, long gs) {
+    return new BlockInfoContiguous(new Block(id, length, gs),
+        (short) DATANODE_COUNT);
+  }
+
   @Test
   public void testPendingReplication() {
     PendingReplicationBlocks pendingReplications;
@@ -64,7 +69,7 @@ public class TestPendingReplication {
     //
     DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos(10);
     for (int i = 0; i < storages.length; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       DatanodeStorageInfo[] targets = new DatanodeStorageInfo[i];
       System.arraycopy(storages, 0, targets, 0, i);
       pendingReplications.increment(block,
@@ -77,7 +82,7 @@ public class TestPendingReplication {
     //
     // remove one item and reinsert it
     //
-    Block blk = new Block(8, 8, 0);
+    BlockInfo blk = genBlockInfo(8, 8, 0);
     pendingReplications.decrement(blk, storages[7].getDatanodeDescriptor()); // removes one replica
     assertEquals("pendingReplications.getNumReplicas ",
                  7, pendingReplications.getNumReplicas(blk));
@@ -97,7 +102,7 @@ public class TestPendingReplication {
     // are sane.
     //
     for (int i = 0; i < 10; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       int numReplicas = pendingReplications.getNumReplicas(block);
       assertTrue(numReplicas == i);
     }
@@ -116,7 +121,7 @@ public class TestPendingReplication {
     }
 
     for (int i = 10; i < 15; i++) {
-      Block block = new Block(i, i, 0);
+      BlockInfo block = genBlockInfo(i, i, 0);
       pendingReplications.increment(block,
           DatanodeStorageInfo.toDatanodeDescriptors(
               DFSTestUtil.createDatanodeStorageInfos(i)));
@@ -185,7 +190,7 @@ public class TestPendingReplication {
 
       assertEquals(1, blkManager.pendingReplications.size());
       INodeFile fileNode = fsn.getFSDirectory().getINode4Write(file).asFile();
-      Block[] blocks = fileNode.getBlocks();
+      BlockInfo[] blocks = fileNode.getBlocks();
       assertEquals(DATANODE_COUNT - 1,
           blkManager.pendingReplications.getNumReplicas(blocks[0]));
 
@@ -291,8 +296,8 @@ public class TestPendingReplication {
       BlockManagerTestUtil.computeAllPendingWork(bm);
       BlockManagerTestUtil.updateState(bm);
       assertEquals(bm.getPendingReplicationBlocksCount(), 1L);
-      assertEquals(bm.pendingReplications.getNumReplicas(block.getBlock()
-          .getLocalBlock()), 2);
+      BlockInfo storedBlock = bm.getStoredBlock(block.getBlock().getLocalBlock());
+      assertEquals(bm.pendingReplications.getNumReplicas(storedBlock), 2);
       
       // 4. delete the file
       fs.delete(filePath, true);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
index 728934d..1a32892 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRBWBlockInvalidation.java
@@ -58,7 +58,9 @@ public class TestRBWBlockInvalidation {
   
   private static NumberReplicas countReplicas(final FSNamesystem namesystem,
       ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index 6cd0d1c..7eeca01 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -833,6 +833,10 @@ public class TestReplicationPolicy {
     assertEquals(targets.length, 2);
     assertTrue(isOnSameRack(targets[0], dataNodes[2]));
   }
+
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
   
   /**
    * Test for the high priority blocks are processed before the low priority
@@ -852,14 +856,14 @@ public class TestReplicationPolicy {
           .getNamesystem().getBlockManager().neededReplications;
       for (int i = 0; i < 100; i++) {
         // Adding the blocks directly to normal priority
-        neededReplications.add(new Block(random.nextLong()), 2, 0, 3);
+        neededReplications.add(genBlockInfo(random.nextLong()), 2, 0, 3);
       }
       // Lets wait for the replication interval, to start process normal
       // priority blocks
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
       
       // Adding the block directly to high priority list
-      neededReplications.add(new Block(random.nextLong()), 1, 0, 3);
+      neededReplications.add(genBlockInfo(random.nextLong()), 1, 0, 3);
       
       // Lets wait for the replication interval
       Thread.sleep(DFS_NAMENODE_REPLICATION_INTERVAL);
@@ -882,25 +886,26 @@ public class TestReplicationPolicy {
 
     for (int i = 0; i < 5; i++) {
       // Adding QUEUE_HIGHEST_PRIORITY block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 1, 0, 3);
 
       // Adding QUEUE_VERY_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 2, 0, 7);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 2, 0, 7);
 
       // Adding QUEUE_REPLICAS_BADLY_DISTRIBUTED block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 6, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 6, 0, 6);
 
       // Adding QUEUE_UNDER_REPLICATED block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 5, 0, 6);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 5, 0, 6);
 
       // Adding QUEUE_WITH_CORRUPT_BLOCKS block
-      underReplicatedBlocks.add(new Block(random.nextLong()), 0, 0, 3);
+      underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 0, 0, 3);
     }
 
     // Choose 6 blocks from UnderReplicatedBlocks. Then it should pick 5 blocks
     // from
     // QUEUE_HIGHEST_PRIORITY and 1 block from QUEUE_VERY_UNDER_REPLICATED.
-    List<List<Block>> chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
+    List<List<BlockInfo>> chosenBlocks =
+        underReplicatedBlocks.chooseUnderReplicatedBlocks(6);
     assertTheChosenBlocks(chosenBlocks, 5, 1, 0, 0, 0);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 4 blocks from
@@ -910,7 +915,7 @@ public class TestReplicationPolicy {
     assertTheChosenBlocks(chosenBlocks, 0, 4, 5, 1, 0);
 
     // Adding QUEUE_HIGHEST_PRIORITY
-    underReplicatedBlocks.add(new Block(random.nextLong()), 1, 0, 3);
+    underReplicatedBlocks.add(genBlockInfo(random.nextLong()), 1, 0, 3);
 
     // Choose 10 blocks from UnderReplicatedBlocks. Then it should pick 1 block from
     // QUEUE_HIGHEST_PRIORITY, 4 blocks from QUEUE_REPLICAS_BADLY_DISTRIBUTED
@@ -928,7 +933,7 @@ public class TestReplicationPolicy {
   
   /** asserts the chosen blocks with expected priority blocks */
   private void assertTheChosenBlocks(
-      List<List<Block>> chosenBlocks, int firstPrioritySize,
+      List<List<BlockInfo>> chosenBlocks, int firstPrioritySize,
       int secondPrioritySize, int thirdPrioritySize, int fourthPrioritySize,
       int fifthPrioritySize) {
     assertEquals(
@@ -1102,9 +1107,9 @@ public class TestReplicationPolicy {
   public void testUpdateDoesNotCauseSkippedReplication() {
     UnderReplicatedBlocks underReplicatedBlocks = new UnderReplicatedBlocks();
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
-    Block block3 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
+    BlockInfo block3 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_VERY_UNDER_REPLICATED block
     final int block1CurReplicas = 2;
@@ -1118,7 +1123,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block3, 2, 0, 6);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1151,8 +1156,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1160,7 +1165,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1197,8 +1202,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1206,7 +1211,7 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
@@ -1260,8 +1265,8 @@ public class TestReplicationPolicy {
     BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration());
     UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications;
 
-    Block block1 = new Block(random.nextLong());
-    Block block2 = new Block(random.nextLong());
+    BlockInfo block1 = genBlockInfo(random.nextLong());
+    BlockInfo block2 = genBlockInfo(random.nextLong());
 
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block1, 0, 1, 1);
@@ -1269,14 +1274,14 @@ public class TestReplicationPolicy {
     // Adding QUEUE_UNDER_REPLICATED block
     underReplicatedBlocks.add(block2, 0, 1, 1);
 
-    List<List<Block>> chosenBlocks;
+    List<List<BlockInfo>> chosenBlocks;
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.
     chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1);
     assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0);
 
-    bm.setReplication((short)0, (short)1, "", block1);
+    bm.setReplication((short)0, (short)1, "", (BlockInfoContiguous) block1);
 
     // Choose 1 block from UnderReplicatedBlocks. Then it should pick 1 block
     // from QUEUE_VERY_UNDER_REPLICATED.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
index e87a043..de36e07 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestUnderReplicatedBlockQueues.java
@@ -28,6 +28,10 @@ import static org.junit.Assert.fail;
 
 public class TestUnderReplicatedBlockQueues {
 
+  private BlockInfo genBlockInfo(long id) {
+    return new BlockInfoContiguous(new Block(id), (short) 3);
+  }
+
   /**
    * Test that adding blocks with different replication counts puts them
    * into different queues
@@ -36,11 +40,11 @@ public class TestUnderReplicatedBlockQueues {
   @Test
   public void testBlockPriorities() throws Throwable {
     UnderReplicatedBlocks queues = new UnderReplicatedBlocks();
-    Block block1 = new Block(1);
-    Block block2 = new Block(2);
-    Block block_very_under_replicated = new Block(3);
-    Block block_corrupt = new Block(4);
-    Block block_corrupt_repl_one = new Block(5);
+    BlockInfo block1 = genBlockInfo(1);
+    BlockInfo block2 = genBlockInfo(2);
+    BlockInfo block_very_under_replicated = genBlockInfo(3);
+    BlockInfo block_corrupt = genBlockInfo(4);
+    BlockInfo block_corrupt_repl_one = genBlockInfo(5);
 
     //add a block with a single entry
     assertAdded(queues, block1, 1, 0, 3);
@@ -82,7 +86,7 @@ public class TestUnderReplicatedBlockQueues {
   }
 
   private void assertAdded(UnderReplicatedBlocks queues,
-                           Block block,
+                           BlockInfo block,
                            int curReplicas,
                            int decomissionedReplicas,
                            int expectedReplicas) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
index 8f99afb..5604246 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestReadOnlySharedStorage.java
@@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -81,6 +82,7 @@ public class TestReadOnlySharedStorage {
   private DatanodeInfo readOnlyDataNode;
   
   private Block block;
+  private BlockInfo storedBlock;
 
   private ExtendedBlock extendedBlock;
 
@@ -132,6 +134,7 @@ public class TestReadOnlySharedStorage {
     LocatedBlock locatedBlock = getLocatedBlock();
     extendedBlock = locatedBlock.getBlock();
     block = extendedBlock.getLocalBlock();
+    storedBlock = blockManager.getStoredBlock(block);
     
     assertThat(locatedBlock.getLocations().length, is(1));
     normalDataNode = locatedBlock.getLocations()[0];
@@ -188,7 +191,7 @@ public class TestReadOnlySharedStorage {
   }
   
   private void validateNumberReplicas(int expectedReplicas) throws IOException {
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(expectedReplicas));
     assertThat(numberReplicas.excessReplicas(), is(0));
     assertThat(numberReplicas.corruptReplicas(), is(0));
@@ -230,7 +233,7 @@ public class TestReadOnlySharedStorage {
         cluster.getNameNode(), normalDataNode.getXferAddr());
     
     // The live replica count should now be zero (since the NORMAL replica is offline)
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.liveReplicas(), is(0));
     
     // The block should be reported as under-replicated
@@ -263,7 +266,7 @@ public class TestReadOnlySharedStorage {
     waitForLocations(1);
     
     // However, the corrupt READ_ONLY_SHARED replica should *not* affect the overall corrupt replicas count
-    NumberReplicas numberReplicas = blockManager.countNodes(block);
+    NumberReplicas numberReplicas = blockManager.countNodes(storedBlock);
     assertThat(numberReplicas.corruptReplicas(), is(0));
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/c3b03388/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
index 37abc5b..228a667 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestProcessCorruptBlocks.java
@@ -32,6 +32,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.NumberReplicas;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.junit.Test;
@@ -260,7 +261,9 @@ public class TestProcessCorruptBlocks {
   }
 
   private static NumberReplicas countReplicas(final FSNamesystem namesystem, ExtendedBlock block) {
-    return namesystem.getBlockManager().countNodes(block.getLocalBlock());
+    final BlockManager blockManager = namesystem.getBlockManager();
+    return blockManager.countNodes(blockManager.getStoredBlock(
+        block.getLocalBlock()));
   }
 
   private void corruptBlock(MiniDFSCluster cluster, FileSystem fs, final Path fileName,


[03/50] [abbrv] hadoop git commit: HDFS-7749. Erasure Coding: Add striped block support in INodeFile. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7749. Erasure Coding: Add striped block support in INodeFile. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/5608b43b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/5608b43b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/5608b43b

Branch: refs/heads/HDFS-7285
Commit: 5608b43b215856bac53990f5e9c4311ae170b723
Parents: b70d584
Author: Jing Zhao <ji...@apache.org>
Authored: Wed Feb 25 22:10:26 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:34:38 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  17 ++
 .../server/blockmanagement/BlockCollection.java |  13 +-
 .../hdfs/server/blockmanagement/BlockInfo.java  |  88 ++++++-
 .../BlockInfoContiguousUnderConstruction.java   |   6 +-
 .../blockmanagement/BlockInfoStriped.java       |  31 +++
 .../BlockInfoStripedUnderConstruction.java      | 240 ++++++++++++++++++
 .../server/blockmanagement/BlockManager.java    | 150 +++++------
 .../CacheReplicationMonitor.java                |  16 +-
 .../hdfs/server/namenode/FSDirConcatOp.java     |   8 +-
 .../hdfs/server/namenode/FSDirectory.java       |   5 +-
 .../hadoop/hdfs/server/namenode/FSEditLog.java  |   8 +-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  16 +-
 .../hdfs/server/namenode/FSImageFormat.java     |   7 +-
 .../server/namenode/FSImageFormatPBINode.java   |  46 +++-
 .../hdfs/server/namenode/FSNamesystem.java      | 108 ++++----
 .../namenode/FileUnderConstructionFeature.java  |  15 +-
 .../namenode/FileWithStripedBlocksFeature.java  | 112 ++++++++
 .../hadoop/hdfs/server/namenode/INodeFile.java  | 254 +++++++++++++------
 .../hdfs/server/namenode/LeaseManager.java      |   6 +-
 .../hdfs/server/namenode/NamenodeFsck.java      |   4 +-
 .../hadoop/hdfs/server/namenode/Namesystem.java |   3 +-
 .../snapshot/FSImageFormatPBSnapshot.java       |   7 +-
 .../server/namenode/snapshot/FileDiffList.java  |   9 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |   5 +
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |  10 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |   3 +-
 .../blockmanagement/TestReplicationPolicy.java  |   4 +-
 .../hdfs/server/namenode/TestAddBlock.java      |  12 +-
 .../hdfs/server/namenode/TestAddBlockgroup.java |   3 +-
 .../namenode/TestBlockUnderConstruction.java    |   6 +-
 .../hdfs/server/namenode/TestFSImage.java       |   4 +-
 .../hdfs/server/namenode/TestFileTruncate.java  |   4 +-
 .../hadoop/hdfs/server/namenode/TestFsck.java   |   4 +-
 .../snapshot/TestSnapshotBlocksMap.java         |  24 +-
 .../namenode/snapshot/TestSnapshotDeletion.java |  16 +-
 35 files changed, 951 insertions(+), 313 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 1942ea9..14410e6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -172,6 +172,7 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageReportProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypeProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageTypesProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StorageUuidsProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.InotifyProtos;
 import org.apache.hadoop.hdfs.protocol.proto.JournalProtocolProtos.JournalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsResponseProto;
@@ -184,6 +185,7 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType;
@@ -430,6 +432,21 @@ public class PBHelper {
     return new Block(b.getBlockId(), b.getNumBytes(), b.getGenStamp());
   }
 
+  public static BlockInfoStriped convert(StripedBlockProto p) {
+    return new BlockInfoStriped(convert(p.getBlock()),
+        (short) p.getDataBlockNum(), (short) p.getParityBlockNum());
+  }
+
+  public static StripedBlockProto convert(BlockInfoStriped blk) {
+    BlockProto bp = BlockProto.newBuilder().setBlockId(blk.getBlockId())
+        .setGenStamp(blk.getGenerationStamp()).setNumBytes(blk.getNumBytes())
+        .build();
+    return StripedBlockProto.newBuilder()
+        .setDataBlockNum(blk.getDataBlockNum())
+        .setParityBlockNum(blk.getParityBlockNum())
+        .setBlock(bp).build();
+  }
+
   public static BlockWithLocationsProto convert(BlockWithLocations blk) {
     return BlockWithLocationsProto.newBuilder()
         .setBlock(convert(blk.getBlock()))

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index b14efb4..440a081 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -31,7 +31,7 @@ public interface BlockCollection {
   /**
    * Get the last block of the collection.
    */
-  public BlockInfoContiguous getLastBlock();
+  public BlockInfo getLastBlock();
 
   /** 
    * Get content summary.
@@ -44,9 +44,9 @@ public interface BlockCollection {
   public int numBlocks();
 
   /**
-   * Get the blocks or block groups.
+   * Get the blocks (striped or contiguous).
    */
-  public BlockInfoContiguous[] getBlocks();
+  public BlockInfo[] getBlocks();
 
   /**
    * Get preferred block size for the collection 
@@ -71,16 +71,15 @@ public interface BlockCollection {
   public String getName();
 
   /**
-   * Set the block/block-group at the given index.
+   * Set the block (contiguous or striped) at the given index.
    */
-  public void setBlock(int index, BlockInfoContiguous blk);
+  public void setBlock(int index, BlockInfo blk);
 
   /**
    * Convert the last block of the collection to an under-construction block
    * and set the locations.
    */
-  public BlockInfoContiguousUnderConstruction setLastBlock(
-      BlockInfoContiguous lastBlock,
+  public void convertLastBlockToUC(BlockInfo lastBlock,
       DatanodeStorageInfo[] targets) throws IOException;
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index f19ad32..d15cbec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -21,6 +21,7 @@ import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.LightWeightGSet;
 
+import java.io.IOException;
 import java.util.LinkedList;
 
 /**
@@ -289,8 +290,9 @@ public abstract class BlockInfo extends Block
 
   /**
    * BlockInfo represents a block that is not being constructed.
-   * In order to start modifying the block, the BlockInfo should be converted
-   * to {@link BlockInfoContiguousUnderConstruction}.
+   * In order to start modifying the block, the BlockInfo should be converted to
+   * {@link BlockInfoContiguousUnderConstruction} or
+   * {@link BlockInfoStripedUnderConstruction}.
    * @return {@link HdfsServerConstants.BlockUCState#COMPLETE}
    */
   public HdfsServerConstants.BlockUCState getBlockUCState() {
@@ -336,4 +338,86 @@ public abstract class BlockInfo extends Block
       return new BlockInfoStriped((BlockInfoStriped) b);
     }
   }
+
+  static BlockInfo convertToCompleteBlock(BlockInfo blk) throws IOException {
+    if (blk instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) blk)
+          .convertToCompleteBlock();
+    } else if (blk instanceof BlockInfoStripedUnderConstruction) {
+      return ((BlockInfoStripedUnderConstruction) blk).convertToCompleteBlock();
+    } else {
+      return blk;
+    }
+  }
+
+  static void commitBlock(BlockInfo blockInfo, Block reported)
+      throws IOException {
+    if (blockInfo instanceof BlockInfoContiguousUnderConstruction) {
+      ((BlockInfoContiguousUnderConstruction) blockInfo).commitBlock(reported);
+    } else if (blockInfo instanceof BlockInfoStripedUnderConstruction) {
+      ((BlockInfoStripedUnderConstruction) blockInfo).commitBlock(reported);
+    }
+  }
+
+  static void addReplica(BlockInfo ucBlock, DatanodeStorageInfo storageInfo,
+      Block reportedBlock, HdfsServerConstants.ReplicaState reportedState) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      ((BlockInfoContiguousUnderConstruction) ucBlock).addReplicaIfNotPresent(
+          storageInfo, reportedBlock, reportedState);
+    } else { // StripedUC
+      ((BlockInfoStripedUnderConstruction) ucBlock).addReplicaIfNotPresent(
+          storageInfo, reportedBlock, reportedState);
+    }
+  }
+
+  static int getNumExpectedLocations(BlockInfo ucBlock) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .getNumExpectedLocations();
+    } else { // StripedUC
+      return ((BlockInfoStripedUnderConstruction) ucBlock)
+          .getNumExpectedLocations();
+    }
+  }
+
+  public static DatanodeStorageInfo[] getExpectedStorageLocations(
+      BlockInfo ucBlock) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .getExpectedStorageLocations();
+    } else { // StripedUC
+      return ((BlockInfoStripedUnderConstruction) ucBlock)
+          .getExpectedStorageLocations();
+    }
+  }
+
+  public static void setExpectedLocations(BlockInfo ucBlock,
+      DatanodeStorageInfo[] targets) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .setExpectedLocations(targets);
+    } else { // StripedUC
+      ((BlockInfoStripedUnderConstruction) ucBlock)
+          .setExpectedLocations(targets);
+    }
+  }
+
+  public static long getBlockRecoveryId(BlockInfo ucBlock) {
+    assert ucBlock instanceof BlockInfoContiguousUnderConstruction ||
+        ucBlock instanceof BlockInfoStripedUnderConstruction;
+    if (ucBlock instanceof BlockInfoContiguousUnderConstruction) {
+      return ((BlockInfoContiguousUnderConstruction) ucBlock)
+          .getBlockRecoveryId();
+    } else { // StripedUC
+      return ((BlockInfoStripedUnderConstruction) ucBlock).getBlockRecoveryId();
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
index c78c9e2..7a052fd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
@@ -74,7 +74,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
       BlockUCState state, DatanodeStorageInfo[] targets) {
     super(blk, replication);
     assert getBlockUCState() != BlockUCState.COMPLETE :
-      "BlockInfoUnderConstruction cannot be in COMPLETE state";
+      "BlockInfoContiguousUnderConstruction cannot be in COMPLETE state";
     this.blockUCState = state;
     setExpectedLocations(targets);
   }
@@ -82,7 +82,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
   /**
    * Convert an under construction block to a complete block.
    * 
-   * @return BlockInfo - a complete block.
+   * @return BlockInfoContiguous - a complete block.
    * @throws IOException if the state of the block 
    * (the generation stamp and the length) has not been committed by 
    * the client or it does not have at least a minimal number of replicas 
@@ -197,7 +197,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     blockRecoveryId = recoveryId;
     if (replicas.size() == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*"
-        + " BlockInfoUnderConstruction.initLeaseRecovery:"
+        + " BlockInfoContiguousUnderConstruction.initLeaseRecovery:"
         + " No blocks found, lease removed.");
     }
     boolean allLiveReplicasTriedAsPrimary = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 5fff41e..57de772 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -18,6 +18,8 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
@@ -59,6 +61,14 @@ public class BlockInfoStriped extends BlockInfo {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
+  public short getDataBlockNum() {
+    return dataBlockNum;
+  }
+
+  public short getParityBlockNum() {
+    return parityBlockNum;
+  }
+
   private void initIndices() {
     for (int i = 0; i < indices.length; i++) {
       indices[i] = -1;
@@ -176,4 +186,25 @@ public class BlockInfoStriped extends BlockInfo {
     }
     return num;
   }
+
+  /**
+   * Convert a complete block to an under construction block.
+   * @return BlockInfoUnderConstruction -  an under construction block.
+   */
+  public BlockInfoStripedUnderConstruction convertToBlockUnderConstruction(
+      BlockUCState s, DatanodeStorageInfo[] targets) {
+    final BlockInfoStripedUnderConstruction ucBlock;
+    if(isComplete()) {
+      ucBlock = new BlockInfoStripedUnderConstruction(this, getDataBlockNum(),
+              getParityBlockNum(),  s, targets);
+      ucBlock.setBlockCollection(getBlockCollection());
+    } else {
+      // the block is already under construction
+      ucBlock = (BlockInfoStripedUnderConstruction) this;
+      ucBlock.setBlockUCState(s);
+      ucBlock.setExpectedLocations(targets);
+      ucBlock.setBlockCollection(getBlockCollection());
+    }
+    return ucBlock;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
new file mode 100644
index 0000000..151241b2
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
@@ -0,0 +1,240 @@
+/**
+ * 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.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
+import org.apache.hadoop.hdfs.server.namenode.NameNode;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.Iterator;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
+
+/**
+ * Represents a striped block that is currently being constructed.
+ * This is usually the last block of a file opened for write or append.
+ */
+public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
+  private BlockUCState blockUCState;
+
+  /**
+   * Block replicas as assigned when the block was allocated.
+   *
+   * TODO: we need to update this attribute, along with the return type of
+   * getExpectedStorageLocations and LocatedBlock. For striped blocks, clients
+   * need to understand the index of each striped block in the block group.
+   */
+  private List<ReplicaUnderConstruction> replicas;
+
+  /**
+   * The new generation stamp, which this block will have
+   * after the recovery succeeds. Also used as a recovery id to identify
+   * the right recovery if any of the abandoned recoveries re-appear.
+   */
+  private long blockRecoveryId = 0;
+
+  /**
+   * Constructor with null storage targets.
+   */
+  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
+      short parityBlockNum) {
+    this(blk, dataBlockNum, parityBlockNum, UNDER_CONSTRUCTION, null);
+  }
+
+  /**
+   * Create a striped block that is currently being constructed.
+   */
+  public BlockInfoStripedUnderConstruction(Block blk, short dataBlockNum,
+      short parityBlockNum, BlockUCState state, DatanodeStorageInfo[] targets) {
+    super(blk, dataBlockNum, parityBlockNum);
+    assert getBlockUCState() != COMPLETE :
+      "BlockInfoStripedUnderConstruction cannot be in COMPLETE state";
+    this.blockUCState = state;
+    setExpectedLocations(targets);
+  }
+
+  /**
+   * Convert an under construction striped block to a complete striped block.
+   * 
+   * @return BlockInfoStriped - a complete block.
+   * @throws IOException if the state of the block 
+   * (the generation stamp and the length) has not been committed by 
+   * the client or it does not have at least a minimal number of replicas 
+   * reported from data-nodes. 
+   */
+  BlockInfoStriped convertToCompleteBlock() throws IOException {
+    assert getBlockUCState() != COMPLETE :
+      "Trying to convert a COMPLETE block";
+    return new BlockInfoStriped(this);
+  }
+
+  /** Set expected locations */
+  public void setExpectedLocations(DatanodeStorageInfo[] targets) {
+    int numLocations = targets == null ? 0 : targets.length;
+    this.replicas = new ArrayList<>(numLocations);
+    for(int i = 0; i < numLocations; i++) {
+      replicas.add(new ReplicaUnderConstruction(this, targets[i],
+          ReplicaState.RBW));
+    }
+  }
+
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  public DatanodeStorageInfo[] getExpectedStorageLocations() {
+    int numLocations = getNumExpectedLocations();
+    DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
+    for (int i = 0; i < numLocations; i++) {
+      storages[i] = replicas.get(i).getExpectedStorageLocation();
+    }
+    return storages;
+  }
+
+  /** Get the number of expected locations */
+  public int getNumExpectedLocations() {
+    return replicas == null ? 0 : replicas.size();
+  }
+
+  /**
+   * Return the state of the block under construction.
+   * @see BlockUCState
+   */
+  @Override // BlockInfo
+  public BlockUCState getBlockUCState() {
+    return blockUCState;
+  }
+
+  void setBlockUCState(BlockUCState s) {
+    blockUCState = s;
+  }
+
+  /** Get block recovery ID */
+  public long getBlockRecoveryId() {
+    return blockRecoveryId;
+  }
+
+  /**
+   * Process the recorded replicas. When about to commit or finish the
+   * pipeline recovery sort out bad replicas.
+   * @param genStamp  The final generation stamp for the block.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp) {
+    // Set the generation stamp for the block.
+    setGenerationStamp(genStamp);
+    if (replicas == null)
+      return;
+
+    // Remove the replicas with wrong gen stamp.
+    // The replica list is unchanged.
+    for (ReplicaUnderConstruction r : replicas) {
+      if (genStamp != r.getGenerationStamp()) {
+        r.getExpectedStorageLocation().removeBlock(this);
+        NameNode.blockStateChangeLog.info("BLOCK* Removing stale replica "
+            + "from location: {}", r.getExpectedStorageLocation());
+      }
+    }
+  }
+
+  /**
+   * Commit block's length and generation stamp as reported by the client.
+   * Set block state to {@link BlockUCState#COMMITTED}.
+   * @param block - contains client reported block length and generation
+   */
+  void commitBlock(Block block) throws IOException {
+    if (getBlockId() != block.getBlockId()) {
+      throw new IOException("Trying to commit inconsistent block: id = "
+          + block.getBlockId() + ", expected id = " + getBlockId());
+    }
+    blockUCState = BlockUCState.COMMITTED;
+    this.set(getBlockId(), block.getNumBytes(), block.getGenerationStamp());
+    // Sort out invalid replicas.
+    setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
+  }
+
+  /**
+   * Initialize lease recovery for this striped block.
+   */
+  public void initializeBlockRecovery(long recoveryId) {
+    setBlockUCState(BlockUCState.UNDER_RECOVERY);
+    blockRecoveryId = recoveryId;
+    if (replicas == null || replicas.size() == 0) {
+      NameNode.blockStateChangeLog.warn("BLOCK*" +
+          " BlockInfoUnderConstruction.initLeaseRecovery:" +
+          " No blocks found, lease removed.");
+    }
+    // TODO we need to implement different recovery logic here
+  }
+
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block,
+      ReplicaState rState) {
+    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
+    while (it.hasNext()) {
+      ReplicaUnderConstruction r = it.next();
+      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
+      if (expectedLocation == storage) {
+        // Record the gen stamp from the report
+        r.setGenerationStamp(block.getGenerationStamp());
+        return;
+      } else if (expectedLocation != null &&
+          expectedLocation.getDatanodeDescriptor() ==
+              storage.getDatanodeDescriptor()) {
+        // The Datanode reported that the block is on a different storage
+        // than the one chosen by BlockPlacementPolicy. This can occur as
+        // we allow Datanodes to choose the target storage. Update our
+        // state by removing the stale entry and adding a new one.
+        it.remove();
+        break;
+      }
+    }
+    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
+  }
+
+  @Override
+  public String toString() {
+    final StringBuilder b = new StringBuilder(100);
+    appendStringTo(b);
+    return b.toString();
+  }
+
+  @Override
+  public void appendStringTo(StringBuilder sb) {
+    super.appendStringTo(sb);
+    appendUCParts(sb);
+  }
+
+  private void appendUCParts(StringBuilder sb) {
+    sb.append("{UCState=").append(blockUCState).append(", replicas=[");
+    if (replicas != null) {
+      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
+      if (iter.hasNext()) {
+        iter.next().appendStringTo(sb);
+        while (iter.hasNext()) {
+          sb.append(", ");
+          iter.next().appendStringTo(sb);
+        }
+      }
+    }
+    sb.append("]}");
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 7a3f410..2e05854 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -541,8 +541,8 @@ public class BlockManager {
     int usableReplicas = numReplicas.liveReplicas() +
                          numReplicas.decommissionedAndDecommissioning();
     
-    if (block instanceof BlockInfoContiguous) {
-      BlockCollection bc = ((BlockInfoContiguous) block).getBlockCollection();
+    if (block instanceof BlockInfo) {
+      BlockCollection bc = ((BlockInfo) block).getBlockCollection();
       String fileName = (bc == null) ? "[orphaned]" : bc.getName();
       out.print(fileName + ": ");
     }
@@ -596,15 +596,14 @@ public class BlockManager {
    * @throws IOException if the block does not have at least a minimal number
    * of replicas reported from data-nodes.
    */
-  private static boolean commitBlock(
-      final BlockInfoContiguousUnderConstruction block,
+  private static boolean commitBlock(final BlockInfo block,
       final Block commitBlock) throws IOException {
     if (block.getBlockUCState() == BlockUCState.COMMITTED)
       return false;
     assert block.getNumBytes() <= commitBlock.getNumBytes() :
       "commitBlock length is less than the stored one "
       + commitBlock.getNumBytes() + " vs. " + block.getNumBytes();
-    block.commitBlock(commitBlock);
+    BlockInfo.commitBlock(block, commitBlock);
     return true;
   }
   
@@ -622,16 +621,16 @@ public class BlockManager {
       Block commitBlock) throws IOException {
     if(commitBlock == null)
       return false; // not committing, this is a block allocation retry
-    BlockInfoContiguous lastBlock = bc.getLastBlock();
+    BlockInfo lastBlock = bc.getLastBlock();
     if(lastBlock == null)
       return false; // no blocks in file yet
     if(lastBlock.isComplete())
       return false; // already completed (e.g. by syncBlock)
     
-    final boolean b = commitBlock(
-        (BlockInfoContiguousUnderConstruction)lastBlock, commitBlock);
-    if(countNodes(lastBlock).liveReplicas() >= minReplication)
-      completeBlock(bc, bc.numBlocks()-1, false);
+    final boolean b = commitBlock(lastBlock, commitBlock);
+    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
+      completeBlock(bc, bc.numBlocks() - 1, false);
+    }
     return b;
   }
 
@@ -644,22 +643,25 @@ public class BlockManager {
    */
   private BlockInfo completeBlock(final BlockCollection bc,
       final int blkIndex, boolean force) throws IOException {
-    if(blkIndex < 0)
+    if (blkIndex < 0) {
       return null;
-    BlockInfoContiguous curBlock = bc.getBlocks()[blkIndex];
-    if (curBlock.isComplete())
+    }
+    BlockInfo curBlock = bc.getBlocks()[blkIndex];
+    if (curBlock.isComplete()) {
       return curBlock;
-    // TODO: support BlockInfoStripedUC
-    BlockInfoContiguousUnderConstruction ucBlock =
-        (BlockInfoContiguousUnderConstruction)curBlock;
-    int numNodes = ucBlock.numNodes();
-    if (!force && numNodes < minReplication)
+    }
+
+    int numNodes = curBlock.numNodes();
+    if (!force && numNodes < minReplication) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
-    if(!force && ucBlock.getBlockUCState() != BlockUCState.COMMITTED)
+    }
+    if (!force && curBlock.getBlockUCState() != BlockUCState.COMMITTED) {
       throw new IOException(
           "Cannot complete block: block has not been COMMITTED by the client");
-    BlockInfoContiguous completeBlock = ucBlock.convertToCompleteBlock();
+    }
+
+    final BlockInfo completeBlock = BlockInfo.convertToCompleteBlock(curBlock);
     // replace penultimate block in file
     bc.setBlock(blkIndex, completeBlock);
     
@@ -677,10 +679,9 @@ public class BlockManager {
     return blocksMap.replaceBlock(completeBlock);
   }
 
-  // TODO: support BlockInfoStrippedUC
   private BlockInfo completeBlock(final BlockCollection bc,
       final BlockInfo block, boolean force) throws IOException {
-    BlockInfoContiguous[] fileBlocks = bc.getBlocks();
+    BlockInfo[] fileBlocks = bc.getBlocks();
     for (int idx = 0; idx < fileBlocks.length; idx++) {
       if (fileBlocks[idx] == block) {
         return completeBlock(bc, idx, force);
@@ -696,6 +697,7 @@ public class BlockManager {
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
       final BlockInfoContiguousUnderConstruction block) throws IOException {
+    // TODO: support BlockInfoStripedUC for editlog
     block.commitBlock(block);
     return completeBlock(bc, block, true);
   }
@@ -717,7 +719,7 @@ public class BlockManager {
    */
   public LocatedBlock convertLastBlockToUnderConstruction(
       BlockCollection bc, long bytesToRemove) throws IOException {
-    BlockInfoContiguous oldBlock = bc.getLastBlock();
+    BlockInfo oldBlock = bc.getLastBlock();
     if(oldBlock == null ||
        bc.getPreferredBlockSize() == oldBlock.getNumBytes() - bytesToRemove)
       return null;
@@ -726,8 +728,10 @@ public class BlockManager {
 
     DatanodeStorageInfo[] targets = getStorages(oldBlock);
 
-    BlockInfoContiguousUnderConstruction ucBlock = bc.setLastBlock(oldBlock,
-        targets);
+    // convert the last block to UC
+    bc.convertLastBlockToUC(oldBlock, targets);
+    // get the new created uc block
+    BlockInfo ucBlock = bc.getLastBlock();
     blocksMap.replaceBlock(ucBlock);
 
     // Remove block from replication queue.
@@ -769,11 +773,10 @@ public class BlockManager {
     return locations;
   }
   
-  private List<LocatedBlock> createLocatedBlockList(
-      final BlockInfoContiguous[] blocks,
+  private List<LocatedBlock> createLocatedBlockList(final BlockInfo[] blocks,
       final long offset, final long length, final int nrBlocksToReturn,
       final AccessMode mode) throws IOException {
-    int curBlk = 0;
+    int curBlk;
     long curPos = 0, blkSize = 0;
     int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
     for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
@@ -786,10 +789,10 @@ public class BlockManager {
     }
 
     if (nrBlocks > 0 && curBlk == nrBlocks)   // offset >= end of file
-      return Collections.<LocatedBlock>emptyList();
+      return Collections.emptyList();
 
     long endOff = offset + length;
-    List<LocatedBlock> results = new ArrayList<LocatedBlock>(blocks.length);
+    List<LocatedBlock> results = new ArrayList<>(blocks.length);
     do {
       results.add(createLocatedBlock(blocks[curBlk], curPos, mode));
       curPos += blocks[curBlk].getNumBytes();
@@ -800,9 +803,9 @@ public class BlockManager {
     return results;
   }
 
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous[] blocks,
+  private LocatedBlock createLocatedBlock(final BlockInfo[] blocks,
       final long endPos, final AccessMode mode) throws IOException {
-    int curBlk = 0;
+    int curBlk;
     long curPos = 0;
     int nrBlocks = (blocks[0].getNumBytes() == 0) ? 0 : blocks.length;
     for (curBlk = 0; curBlk < nrBlocks; curBlk++) {
@@ -815,8 +818,8 @@ public class BlockManager {
     
     return createLocatedBlock(blocks[curBlk], curPos, mode);
   }
-  
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos,
+
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos,
     final AccessMode mode) throws IOException {
     final LocatedBlock lb = createLocatedBlock(blk, pos);
     if (mode != null) {
@@ -826,8 +829,8 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfoContiguous blk, final long pos
-      ) throws IOException {
+  private LocatedBlock createLocatedBlock(final BlockInfo blk,
+      final long pos) throws IOException {
     if (blk instanceof BlockInfoContiguousUnderConstruction) {
       if (blk.isComplete()) {
         throw new IOException(
@@ -840,6 +843,7 @@ public class BlockManager {
       final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
       return new LocatedBlock(eb, storages, pos, false);
     }
+    // TODO support BlockInfoStripedUC
 
     // get block locations
     final int numCorruptNodes = countNodes(blk).corruptReplicas();
@@ -874,7 +878,7 @@ public class BlockManager {
   }
 
   /** Create a LocatedBlocks. */
-  public LocatedBlocks createLocatedBlocks(final BlockInfoContiguous[] blocks,
+  public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks,
       final long fileSizeExcludeBlocksUnderConstruction,
       final boolean isFileUnderConstruction, final long offset,
       final long length, final boolean needBlockToken,
@@ -897,7 +901,7 @@ public class BlockManager {
       final LocatedBlock lastlb;
       final boolean isComplete;
       if (!inSnapshot) {
-        final BlockInfoContiguous last = blocks[blocks.length - 1];
+        final BlockInfo last = blocks[blocks.length - 1];
         final long lastPos = last.isComplete()?
             fileSizeExcludeBlocksUnderConstruction - last.getNumBytes()
             : fileSizeExcludeBlocksUnderConstruction;
@@ -1714,12 +1718,15 @@ public class BlockManager {
    * reported by the datanode in the block report. 
    */
   static class StatefulBlockInfo {
-    final BlockInfoContiguousUnderConstruction storedBlock;
+    final BlockInfo storedBlock; // should be UC block
     final Block reportedBlock;
     final ReplicaState reportedState;
     
-    StatefulBlockInfo(BlockInfoContiguousUnderConstruction storedBlock,
+    StatefulBlockInfo(BlockInfo storedBlock,
         Block reportedBlock, ReplicaState reportedState) {
+      Preconditions.checkArgument(
+          storedBlock instanceof BlockInfoContiguousUnderConstruction ||
+          storedBlock instanceof BlockInfoStripedUnderConstruction);
       this.storedBlock = storedBlock;
       this.reportedBlock = reportedBlock;
       this.reportedState = reportedState;
@@ -2115,15 +2122,12 @@ public class BlockManager {
       
       // If block is under construction, add this replica to its list
       if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-        ((BlockInfoContiguousUnderConstruction) storedBlock)
-            .addReplicaIfNotPresent(storageInfo, iblk, reportedState);
+        BlockInfo.addReplica(storedBlock, storageInfo, iblk, reportedState);
         // OpenFileBlocks only inside snapshots also will be added to safemode
         // threshold. So we need to update such blocks to safemode
         // refer HDFS-5283
-        BlockInfoContiguousUnderConstruction blockUC =
-            (BlockInfoContiguousUnderConstruction) storedBlock;
-        if (namesystem.isInSnapshot(blockUC)) {
-          int numOfReplicas = blockUC.getNumExpectedLocations();
+        if (namesystem.isInSnapshot(storedBlock.getBlockCollection())) {
+          int numOfReplicas = BlockInfo.getNumExpectedLocations(storedBlock);
           namesystem.incrementSafeBlockCount(numOfReplicas);
         }
         //and fall through to next clause
@@ -2146,7 +2150,7 @@ public class BlockManager {
     // place a delimiter in the list which separates blocks 
     // that have been reported from those that have not
     Block delimiterBlock = new Block();
-    BlockInfoContiguous delimiter = new BlockInfoContiguous(delimiterBlock,
+    BlockInfo delimiter = new BlockInfoContiguous(delimiterBlock,
         (short) 1);
     AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock);
     assert result == AddBlockResult.ADDED 
@@ -2276,9 +2280,8 @@ public class BlockManager {
     }
 
     if (isBlockUnderConstruction(storedBlock, ucState, reportedState)) {
-      toUC.add(new StatefulBlockInfo(
-          (BlockInfoContiguousUnderConstruction) storedBlock,
-          new Block(block), reportedState));
+      toUC.add(new StatefulBlockInfo(storedBlock, new Block(block),
+          reportedState));
       return storedBlock;
     }
 
@@ -2462,9 +2465,8 @@ public class BlockManager {
 
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
-    BlockInfoContiguousUnderConstruction block = ucBlock.storedBlock;
-    block.addReplicaIfNotPresent(
-        storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
+    BlockInfo block = ucBlock.storedBlock;
+    BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         !block.findDatanode(storageInfo.getDatanodeDescriptor())) {
@@ -2524,7 +2526,8 @@ public class BlockManager {
     assert block != null && namesystem.hasWriteLock();
     BlockInfo storedBlock;
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
-    if (block instanceof BlockInfoContiguousUnderConstruction) {
+    if (block instanceof BlockInfoContiguousUnderConstruction ||
+        block instanceof BlockInfoStripedUnderConstruction) {
       //refresh our copy in case the block got completed in another thread
       storedBlock = getStoredBlock(block);
     } else {
@@ -2540,7 +2543,6 @@ public class BlockManager {
       return block;
     }
     BlockCollection bc = storedBlock.getBlockCollection();
-    assert bc != null : "Block must belong to a file";
 
     // add block to the datanode
     AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock);
@@ -3457,8 +3459,8 @@ public class BlockManager {
    * replicated.
    */
   public boolean checkBlocksProperlyReplicated(
-      String src, BlockInfoContiguous[] blocks) {
-    for (BlockInfoContiguous b: blocks) {
+      String src, BlockInfo[] blocks) {
+    for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
         final BlockInfoContiguousUnderConstruction uc =
             (BlockInfoContiguousUnderConstruction)b;
@@ -3527,7 +3529,7 @@ public class BlockManager {
     if (!this.shouldCheckForEnoughRacks) {
       return true;
     }
-    boolean enoughRacks = false;;
+    boolean enoughRacks = false;
     Collection<DatanodeDescriptor> corruptNodes = 
                                   corruptReplicas.getNodes(b);
     int numExpectedReplicas = getReplication(b);
@@ -3573,21 +3575,15 @@ public class BlockManager {
     return this.neededReplications.getCorruptReplOneBlockSize();
   }
 
-  public BlockInfoContiguous addBlockCollection(BlockInfoContiguous block,
+  public BlockInfo addBlockCollection(BlockInfo block,
       BlockCollection bc) {
-    // TODO
-    return (BlockInfoContiguous) blocksMap.addBlockCollection(block, bc);
+    return blocksMap.addBlockCollection(block, bc);
   }
 
   public BlockCollection getBlockCollection(Block b) {
     return blocksMap.getBlockCollection(b);
   }
 
-  /** @return an iterator of the datanodes. */
-  public Iterable<DatanodeStorageInfo> getStorages(final Block block) {
-    return blocksMap.getStorages(block);
-  }
-
   public int numCorruptReplicas(Block block) {
     return corruptReplicas.numCorruptReplicas(block);
   }
@@ -3620,26 +3616,6 @@ public class BlockManager {
   public int getCapacity() {
     return blocksMap.getCapacity();
   }
-  
-  /**
-   * Return a range of corrupt replica block ids. Up to numExpectedBlocks 
-   * blocks starting at the next block after startingBlockId are returned
-   * (fewer if numExpectedBlocks blocks are unavailable). If startingBlockId 
-   * is null, up to numExpectedBlocks blocks are returned from the beginning.
-   * If startingBlockId cannot be found, null is returned.
-   *
-   * @param numExpectedBlocks Number of block ids to return.
-   *  0 <= numExpectedBlocks <= 100
-   * @param startingBlockId Block id from which to start. If null, start at
-   *  beginning.
-   * @return Up to numExpectedBlocks blocks from startingBlockId if it exists
-   *
-   */
-  public long[] getCorruptReplicaBlockIds(int numExpectedBlocks,
-                                   Long startingBlockId) {
-    return corruptReplicas.getCorruptReplicaBlockIds(numExpectedBlocks,
-                                                     startingBlockId);
-  }
 
   /**
    * Return an iterator over the set of blocks for which there are no replicas.
@@ -3753,7 +3729,7 @@ public class BlockManager {
     datanodeManager.clearPendingQueues();
     postponedMisreplicatedBlocks.clear();
     postponedMisreplicatedBlocksCount.set(0);
-  };
+  }
   
 
   private static class ReplicationWork {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
index 79d7713..a1290a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/CacheReplicationMonitor.java
@@ -369,7 +369,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
    * @param file The file.
    */
   private void rescanFile(CacheDirective directive, INodeFile file) {
-    BlockInfoContiguous[] blockInfos = file.getBlocks();
+    BlockInfo[] blockInfos = file.getBlocks();
 
     // Increment the "needed" statistics
     directive.addFilesNeeded(1);
@@ -394,7 +394,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
     }
 
     long cachedTotal = 0;
-    for (BlockInfoContiguous blockInfo : blockInfos) {
+    for (BlockInfo blockInfo : blockInfos) {
       if (!blockInfo.getBlockUCState().equals(BlockUCState.COMPLETE)) {
         // We don't try to cache blocks that are under construction.
         LOG.trace("Directive {}: can't cache block {} because it is in state "
@@ -452,8 +452,8 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
         file.getFullPathName(), cachedTotal, neededTotal);
   }
 
-  private String findReasonForNotCaching(CachedBlock cblock, 
-          BlockInfoContiguous blockInfo) {
+  private String findReasonForNotCaching(CachedBlock cblock,
+      BlockInfo blockInfo) {
     if (blockInfo == null) {
       // Somehow, a cache report with the block arrived, but the block
       // reports from the DataNode haven't (yet?) described such a block.
@@ -513,7 +513,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
           iter.remove();
         }
       }
-      BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId()));
+      BlockInfo blockInfo = namesystem.getStoredBlock(new Block(cblock.getBlockId()));
       String reason = findReasonForNotCaching(cblock, blockInfo);
       int neededCached = 0;
       if (reason != null) {
@@ -627,7 +627,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       List<DatanodeDescriptor> pendingCached) {
     // To figure out which replicas can be cached, we consult the
     // blocksMap.  We don't want to try to cache a corrupt replica, though.
-    BlockInfoContiguous blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId()));
+    BlockInfo blockInfo = namesystem.getStoredBlock(new Block(cachedBlock.getBlockId()));
     if (blockInfo == null) {
       LOG.debug("Block {}: can't add new cached replicas," +
           " because there is no record of this block " +
@@ -665,7 +665,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       Iterator<CachedBlock> it = datanode.getPendingCached().iterator();
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
-        BlockInfoContiguous info =
+        BlockInfo info =
             namesystem.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes -= info.getNumBytes();
@@ -675,7 +675,7 @@ public class CacheReplicationMonitor extends Thread implements Closeable {
       // Add pending uncached blocks from effective capacity
       while (it.hasNext()) {
         CachedBlock cBlock = it.next();
-        BlockInfoContiguous info =
+        BlockInfo info =
             namesystem.getStoredBlock(new Block(cBlock.getBlockId()));
         if (info != null) {
           pendingBytes += info.getNumBytes();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
index 31a6af7..576c0b0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirConcatOp.java
@@ -143,6 +143,7 @@ class FSDirConcatOp {
         throw new HadoopIllegalArgumentException("concat: source file " + src
             + " is invalid or empty or underConstruction");
       }
+
       // source file's preferred block size cannot be greater than the target
       // file
       if (srcINodeFile.getPreferredBlockSize() >
@@ -152,6 +153,11 @@ class FSDirConcatOp {
             + " which is greater than the target file's preferred block size "
             + targetINode.getPreferredBlockSize());
       }
+      // TODO currently we do not support concatenating EC files
+      if (srcINodeFile.isStriped()) {
+        throw new HadoopIllegalArgumentException("concat: the src file " + src
+            + " is with striped blocks");
+      }
       si.add(srcINodeFile);
     }
 
@@ -228,7 +234,7 @@ class FSDirConcatOp {
     int count = 0;
     for (INodeFile nodeToRemove : srcList) {
       if(nodeToRemove != null) {
-        nodeToRemove.setBlocks(null);
+        nodeToRemove.setContiguousBlocks(null);
         nodeToRemove.getParent().removeChild(nodeToRemove);
         fsd.getINodeMap().remove(nodeToRemove);
         count++;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index f48d4ff..0aad56d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -1072,7 +1073,7 @@ public class FSDirectory implements Closeable {
         unprotectedTruncate(iip, newLength, collectedBlocks, mtime, null);
 
     if(! onBlockBoundary) {
-      BlockInfoContiguous oldBlock = file.getLastBlock();
+      BlockInfo oldBlock = file.getLastBlock();
       Block tBlk =
       getFSNamesystem().prepareFileForTruncate(iip,
           clientName, clientMachine, file.computeFileSize() - newLength,
@@ -1081,7 +1082,7 @@ public class FSDirectory implements Closeable {
           tBlk.getNumBytes() == truncateBlock.getNumBytes() :
           "Should be the same block.";
       if(oldBlock.getBlockId() != tBlk.getBlockId() &&
-         !file.isBlockInLatestSnapshot(oldBlock)) {
+         !file.isBlockInLatestSnapshot((BlockInfoContiguous) oldBlock)) {
         getBlockManager().removeBlockFromMap(oldBlock);
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
index df9c585..392a670 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLog.java
@@ -45,7 +45,7 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
 import org.apache.hadoop.hdfs.protocol.CachePoolInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole;
 import org.apache.hadoop.hdfs.server.common.Storage.FormatConfirmable;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
@@ -773,10 +773,10 @@ public class FSEditLog implements LogsPurgeable {
   
   public void logAddBlock(String path, INodeFile file) {
     Preconditions.checkArgument(file.isUnderConstruction());
-    BlockInfoContiguous[] blocks = file.getBlocks();
+    BlockInfo[] blocks = file.getBlocks();
     Preconditions.checkState(blocks != null && blocks.length > 0);
-    BlockInfoContiguous pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
-    BlockInfoContiguous lastBlock = blocks[blocks.length - 1];
+    BlockInfo pBlock = blocks.length > 1 ? blocks[blocks.length - 2] : null;
+    BlockInfo lastBlock = blocks[blocks.length - 1];
     AddBlockOp op = AddBlockOp.getInstance(cache.get()).setPath(path)
         .setPenultimateBlock(pBlock).setLastBlock(lastBlock);
     logEdit(op);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index f50dc4d..946a37c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -507,7 +508,7 @@ public class FSEditLogLoader {
       }
       INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
       // add the new block to the INodeFile
-      addNewBlock(fsDir, addBlockOp, oldFile);
+      addNewBlock(addBlockOp, oldFile);
       break;
     }
     case OP_SET_REPLICATION: {
@@ -936,15 +937,15 @@ public class FSEditLogLoader {
 
   /**
    * Add a new block into the given INodeFile
+   * TODO support adding striped block
    */
-  private void addNewBlock(FSDirectory fsDir, AddBlockOp op, INodeFile file)
-      throws IOException {
-    BlockInfoContiguous[] oldBlocks = file.getBlocks();
+  private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException {
+    BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
     
     if (pBlock != null) { // the penultimate block is not null
-      Preconditions.checkState(oldBlocks != null && oldBlocks.length > 0);
+      assert oldBlocks != null && oldBlocks.length > 0;
       // compare pBlock with the last block of oldBlocks
       Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
       if (oldLastBlock.getBlockId() != pBlock.getBlockId()
@@ -974,12 +975,13 @@ public class FSEditLogLoader {
   
   /**
    * Update in-memory data structures with new block information.
+   * TODO support adding striped block
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
       INodesInPath iip, INodeFile file) throws IOException {
     // Update its block list
-    BlockInfoContiguous[] oldBlocks = file.getBlocks();
+    BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
     String path = op.getPath();
     
@@ -988,7 +990,7 @@ public class FSEditLogLoader {
     
     // First, update blocks in common
     for (int i = 0; i < oldBlocks.length && i < newBlocks.length; i++) {
-      BlockInfoContiguous oldBlock = oldBlocks[i];
+      BlockInfo oldBlock = oldBlocks[i];
       Block newBlock = newBlocks[i];
       
       boolean isLastBlock = i == newBlocks.length - 1;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index cce991f..d62b804 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -51,6 +51,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutFlags;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion.Feature;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -684,7 +685,7 @@ public class FSImageFormat {
 
     public void updateBlocksMap(INodeFile file) {
       // Add file->block mapping
-      final BlockInfoContiguous[] blocks = file.getBlocks();
+      final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         final BlockManager bm = namesystem.getBlockManager();
         for (int i = 0; i < blocks.length; i++) {
@@ -952,9 +953,9 @@ public class FSImageFormat {
         FileUnderConstructionFeature uc = cons.getFileUnderConstructionFeature();
         oldnode.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         if (oldnode.numBlocks() > 0) {
-          BlockInfoContiguous ucBlock = cons.getLastBlock();
+          BlockInfo ucBlock = cons.getLastBlock();
           // we do not replace the inode, just replace the last block of oldnode
-          BlockInfoContiguous info = namesystem.getBlockManager().addBlockCollection(
+          BlockInfo info = namesystem.getBlockManager().addBlockCollection(
               ucBlock, oldnode);
           oldnode.setBlock(oldnode.numBlocks() - 1, info);
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index b758458..a025bb0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -41,9 +41,13 @@ import org.apache.hadoop.fs.XAttr;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.StripedBlockProto;
 import org.apache.hadoop.hdfs.protocolPB.PBHelper;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext;
 import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.SaverContext;
@@ -52,6 +56,7 @@ import org.apache.hadoop.hdfs.server.namenode.FsImageProto.FilesUnderConstructio
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeDirectorySection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.AclFeatureProto;
+import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.StripedBlocksFeature;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrCompactProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.XAttrFeatureProto;
 import org.apache.hadoop.hdfs.server.namenode.FsImageProto.INodeSection.QuotaByStorageTypeEntryProto;
@@ -210,7 +215,7 @@ public final class FSImageFormatPBINode {
 
     public static void updateBlocksMap(INodeFile file, BlockManager bm) {
       // Add file->block mapping
-      final BlockInfoContiguous[] blocks = file.getBlocks();
+      final BlockInfo[] blocks = file.getBlocks();
       if (blocks != null) {
         for (int i = 0; i < blocks.length; i++) {
           file.setBlock(i, bm.addBlockCollection(blocks[i], file));
@@ -345,16 +350,30 @@ public final class FSImageFormatPBINode {
             loadXAttrs(f.getXAttrs(), state.getStringTable())));
       }
 
+      FileWithStripedBlocksFeature stripeFeature = null;
+      if (f.hasStripedBlocks()) {
+        StripedBlocksFeature sb = f.getStripedBlocks();
+        stripeFeature = file.addStripedBlocksFeature();
+        for (StripedBlockProto sp : sb.getBlocksList()) {
+          stripeFeature.addBlock(PBHelper.convert(sp));
+        }
+      }
+
       // under-construction information
       if (f.hasFileUC()) {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
-        if (blocks.length > 0) {
-          BlockInfoContiguous lastBlk = file.getLastBlock();
-          // replace the last block of file
-          file.setBlock(file.numBlocks() - 1, new BlockInfoContiguousUnderConstruction(
-              lastBlk, replication));
+        BlockInfo lastBlk = file.getLastBlock();
+        // replace the last block of file
+        final BlockInfo ucBlk;
+        if (stripeFeature != null) {
+          BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
+          ucBlk = new BlockInfoStripedUnderConstruction(striped,
+              striped.getDataBlockNum(), striped.getParityBlockNum());
+        } else {
+          ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication);
         }
+        file.setBlock(file.numBlocks() - 1, ucBlk);
       }
       return file;
     }
@@ -617,6 +636,19 @@ public final class FSImageFormatPBINode {
         }
       }
 
+      FileWithStripedBlocksFeature sb = n.getStripedBlocksFeature();
+      if (sb != null) {
+        StripedBlocksFeature.Builder builder =
+            StripedBlocksFeature.newBuilder();
+        BlockInfoStriped[] sblocks = sb.getBlocks();
+        if (sblocks != null) {
+          for (BlockInfoStriped sblk : sblocks) {
+            builder.addBlocks(PBHelper.convert(sblk));
+          }
+        }
+        b.setStripedBlocks(builder.build());
+      }
+
       FileUnderConstructionFeature uc = n.getFileUnderConstructionFeature();
       if (uc != null) {
         INodeSection.FileUnderConstructionFeature f =
@@ -645,7 +677,7 @@ public final class FSImageFormatPBINode {
       r.writeDelimitedTo(out);
     }
 
-    private final INodeSection.INode.Builder buildINodeCommon(INode n) {
+    private INodeSection.INode.Builder buildINodeCommon(INode n) {
       return INodeSection.INode.newBuilder()
           .setId(n.getId())
           .setName(ByteString.copyFrom(n.getLocalNameBytes()));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 0b93474..e5a2749 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -203,8 +203,10 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretMan
 import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenSecretManager.SecretManagerState;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -1979,6 +1981,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final BlockStoragePolicy lpPolicy =
         blockManager.getStoragePolicy("LAZY_PERSIST");
 
+    // not support truncating file with striped blocks
+    if (file.isStriped()) {
+      throw new UnsupportedOperationException(
+          "Cannot truncate file with striped block " + src);
+    }
     if (lpPolicy != null &&
         lpPolicy.getId() == file.getStoragePolicyID()) {
       throw new UnsupportedOperationException(
@@ -2061,8 +2068,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     leaseManager.addLease(
         file.getFileUnderConstructionFeature().getClientName(), src);
     boolean shouldRecoverNow = (newBlock == null);
-    BlockInfoContiguous oldBlock = file.getLastBlock();
-    boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file, oldBlock);
+
+    BlockInfo oldBlock = file.getLastBlock();
+    assert oldBlock instanceof BlockInfoContiguous;
+
+    boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file,
+        (BlockInfoContiguous) oldBlock);
     if(newBlock == null) {
       newBlock = (shouldCopyOnTruncate) ? createNewBlock(file.isStriped()) :
           new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(),
@@ -2077,7 +2088,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           file.getBlockReplication());
       truncatedBlockUC.setNumBytes(oldBlock.getNumBytes() - lastBlockDelta);
       truncatedBlockUC.setTruncateBlock(oldBlock);
-      file.setLastBlock(truncatedBlockUC, blockManager.getStorages(oldBlock));
+      file.convertLastBlockToUC(truncatedBlockUC,
+          blockManager.getStorages(oldBlock));
       getBlockManager().addBlockCollection(truncatedBlockUC, file);
 
       NameNode.stateChangeLog.info("BLOCK* prepareFileForTruncate: "
@@ -2658,6 +2670,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           + src + " for client " + clientMachine);
       }
       INodeFile myFile = INodeFile.valueOf(inode, src, true);
+
+      // not support appending file with striped blocks
+      if (myFile.isStriped()) {
+        throw new UnsupportedOperationException(
+            "Cannot truncate file with striped block " + src);
+      }
+
       final BlockStoragePolicy lpPolicy =
           blockManager.getStoragePolicy("LAZY_PERSIST");
       if (lpPolicy != null &&
@@ -2669,7 +2688,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       recoverLeaseInternal(RecoverLeaseOp.APPEND_FILE,
           iip, src, holder, clientMachine, false);
       
-      final BlockInfoContiguous lastBlock = myFile.getLastBlock();
+      final BlockInfoContiguous lastBlock =
+          (BlockInfoContiguous) myFile.getLastBlock();
       // Check that the block has at least minimum replication.
       if(lastBlock != null && lastBlock.isComplete() &&
           !getBlockManager().isSufficientlyReplicated(lastBlock)) {
@@ -2725,7 +2745,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         }
       }
     } else {
-      BlockInfoContiguous lastBlock = file.getLastBlock();
+      BlockInfo lastBlock = file.getLastBlock();
       if (lastBlock != null) {
         ExtendedBlock blk = new ExtendedBlock(this.getBlockPoolId(), lastBlock);
         ret = new LocatedBlock(blk, new DatanodeInfo[0]);
@@ -2904,7 +2924,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 op.getExceptionMessage(src, holder, clientMachine,
                     "lease recovery is in progress. Try again later."));
         } else {
-          final BlockInfoContiguous lastBlock = file.getLastBlock();
+          final BlockInfo lastBlock = file.getLastBlock();
           if (lastBlock != null
               && lastBlock.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
             throw new RecoveryInProgressException(
@@ -3127,9 +3147,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           return onRetryBlock[0];
         } else {
           // add new chosen targets to already allocated block and return
-          BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
-          ((BlockInfoContiguousUnderConstruction) lastBlockInFile)
-              .setExpectedLocations(targets);
+          BlockInfo lastBlockInFile = pendingFile.getLastBlock();
+          BlockInfo.setExpectedLocations(lastBlockInFile, targets);
           offset = pendingFile.computeFileSize();
           return makeLocatedBlock(lastBlockInFile, targets, offset);
         }
@@ -3219,7 +3238,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       }
     }
     final INodeFile pendingFile = checkLease(src, clientName, inode, fileId);
-    BlockInfoContiguous lastBlockInFile = pendingFile.getLastBlock();
+    BlockInfo lastBlockInFile = pendingFile.getLastBlock();
     if (!Block.matchingIdAndGenStamp(previousBlock, lastBlockInFile)) {
       // The block that the client claims is the current last block
       // doesn't match up with what we think is the last block. There are
@@ -3247,7 +3266,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       //    changed the namesystem state yet.
       //    We run this analysis again in Part II where case 4 is impossible.
 
-      BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
+      BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
       if (previous == null &&
           lastBlockInFile != null &&
           lastBlockInFile.getNumBytes() >= pendingFile.getPreferredBlockSize() &&
@@ -3274,8 +3293,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
             src + ". Returning previously allocated block " + lastBlockInFile);
         long offset = pendingFile.computeFileSize();
         onRetryBlock[0] = makeLocatedBlock(lastBlockInFile,
-            ((BlockInfoContiguousUnderConstruction)lastBlockInFile)
-                .getExpectedStorageLocations(), offset);
+            BlockInfo.getExpectedStorageLocations(lastBlockInFile), offset);
         return new FileState(pendingFile, src, iip);
       } else {
         // Case 3
@@ -3595,13 +3613,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
           .getBlocks());
     } else {
       // check the penultimate block of this file
-      BlockInfoContiguous b = v.getPenultimateBlock();
+      BlockInfo b = v.getPenultimateBlock();
       return b == null ||
           blockManager.checkBlocksProperlyReplicated(
-              src, new BlockInfoContiguous[] { b });
+              src, new BlockInfo[] { b });
     }
   }
-  
+
   /**
    * Change the indicated filename. 
    * @deprecated Use {@link #renameTo(String, String, boolean,
@@ -3770,7 +3788,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     for (Block b : blocks.getToDeleteList()) {
       if (trackBlockCounts) {
-        BlockInfoContiguous bi = getStoredBlock(b);
+        BlockInfo bi = getStoredBlock(b);
         if (bi.isComplete()) {
           numRemovedComplete++;
           if (bi.numNodes() >= blockManager.minReplication) {
@@ -3994,10 +4012,10 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     final INodeFile pendingFile = iip.getLastINode().asFile();
     int nrBlocks = pendingFile.numBlocks();
-    BlockInfoContiguous[] blocks = pendingFile.getBlocks();
+    BlockInfo[] blocks = pendingFile.getBlocks();
 
     int nrCompleteBlocks;
-    BlockInfoContiguous curBlock = null;
+    BlockInfo curBlock = null;
     for(nrCompleteBlocks = 0; nrCompleteBlocks < nrBlocks; nrCompleteBlocks++) {
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
@@ -4032,12 +4050,12 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // The last block is not COMPLETE, and
     // that the penultimate block if exists is either COMPLETE or COMMITTED
-    final BlockInfoContiguous lastBlock = pendingFile.getLastBlock();
+    final BlockInfo lastBlock = pendingFile.getLastBlock();
     BlockUCState lastBlockState = lastBlock.getBlockUCState();
-    BlockInfoContiguous penultimateBlock = pendingFile.getPenultimateBlock();
+    BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
 
     // If penultimate block doesn't exist then its minReplication is met
-    boolean penultimateBlockMinReplication = penultimateBlock == null ? true :
+    boolean penultimateBlockMinReplication = penultimateBlock == null ||
         blockManager.checkMinReplication(penultimateBlock);
 
     switch(lastBlockState) {
@@ -4067,6 +4085,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
+      // TODO support Striped block's recovery
       final BlockInfoContiguousUnderConstruction uc =
           (BlockInfoContiguousUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
@@ -4178,14 +4197,13 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     blockManager.checkReplication(pendingFile);
   }
 
-  public BlockInfoContiguous getStoredBlock(Block block) {
-    return (BlockInfoContiguous) blockManager.getStoredBlock(block);
+  public BlockInfo getStoredBlock(Block block) {
+    return blockManager.getStoredBlock(block);
   }
   
   @Override
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC) {
+  public boolean isInSnapshot(BlockCollection bc) {
     assert hasReadLock();
-    final BlockCollection bc = blockUC.getBlockCollection();
     if (bc == null || !(bc instanceof INodeFile)
         || !bc.isUnderConstruction()) {
       return false;
@@ -4236,7 +4254,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   
       checkNameNodeSafeMode(
           "Cannot commitBlockSynchronization while in safe mode");
-      final BlockInfoContiguous storedBlock = getStoredBlock(
+      final BlockInfo storedBlock = getStoredBlock(
           ExtendedBlock.getLocalBlock(oldBlock));
       if (storedBlock == null) {
         if (deleteblock) {
@@ -4286,11 +4304,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return;
       }
 
-      BlockInfoContiguousUnderConstruction truncatedBlock =
-          (BlockInfoContiguousUnderConstruction) iFile.getLastBlock();
-      long recoveryId = truncatedBlock.getBlockRecoveryId();
-      boolean copyTruncate =
-          truncatedBlock.getBlockId() != storedBlock.getBlockId();
+      BlockInfo ucBlock = iFile.getLastBlock();
+      long recoveryId = BlockInfo.getBlockRecoveryId(ucBlock);
+      boolean copyTruncate = ucBlock.getBlockId() != storedBlock.getBlockId();
       if(recoveryId != newgenerationstamp) {
         throw new IOException("The recovery id " + newgenerationstamp
                               + " does not match current recovery id "
@@ -4303,8 +4319,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         if (remove) {
           blockManager.removeBlock(storedBlock);
         }
-      }
-      else {
+      } else {
         // update last block
         if(!copyTruncate) {
           storedBlock.setGenerationStamp(newgenerationstamp);
@@ -4338,7 +4353,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i));
             if (storageInfo != null) {
               if(copyTruncate) {
-                storageInfo.addBlock(truncatedBlock, truncatedBlock);
+                storageInfo.addBlock(ucBlock, ucBlock);
               } else {
                 storageInfo.addBlock(storedBlock, storedBlock);
               }
@@ -4352,9 +4367,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                 trimmedTargets.toArray(new DatanodeID[trimmedTargets.size()]),
                 trimmedStorages.toArray(new String[trimmedStorages.size()]));
         if(copyTruncate) {
-          iFile.setLastBlock(truncatedBlock, trimmedStorageInfos);
+          iFile.convertLastBlockToUC(ucBlock, trimmedStorageInfos);
         } else {
-          iFile.setLastBlock(storedBlock, trimmedStorageInfos);
+          iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos);
           if (closeFile) {
             blockManager.markBlockReplicasAsCorrupt(storedBlock,
                 oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
@@ -4364,8 +4379,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       if (closeFile) {
         if(copyTruncate) {
-          src = closeFileCommitBlocks(iFile, truncatedBlock);
-          if(!iFile.isBlockInLatestSnapshot(storedBlock)) {
+          src = closeFileCommitBlocks(iFile, ucBlock);
+          if(!iFile.isBlockInLatestSnapshot((BlockInfoContiguous) storedBlock)) {
             blockManager.removeBlock(storedBlock);
           }
         } else {
@@ -4398,7 +4413,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    * @throws IOException on error
    */
   @VisibleForTesting
-  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfoContiguous storedBlock)
+  String closeFileCommitBlocks(INodeFile pendingFile, BlockInfo storedBlock)
       throws IOException {
     final INodesInPath iip = INodesInPath.fromINode(pendingFile);
     final String src = iip.getPath();
@@ -4708,7 +4723,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
         while (it.hasNext()) {
           Block b = it.next();
-          BlockInfoContiguous blockInfo = getStoredBlock(b);
+          BlockInfo blockInfo = getStoredBlock(b);
           if (blockInfo.getBlockCollection().getStoragePolicyID() == lpPolicy.getId()) {
             filesToDelete.add(blockInfo.getBlockCollection());
           }
@@ -5661,7 +5676,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     SafeModeInfo safeMode = this.safeMode;
     if (safeMode == null) // mostly true
       return;
-    BlockInfoContiguous storedBlock = getStoredBlock(b);
+    BlockInfo storedBlock = getStoredBlock(b);
     if (storedBlock.isComplete()) {
       safeMode.decrementSafeBlockCount((short)blockManager.countNodes(b).liveReplicas());
     }
@@ -6206,7 +6221,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         + "access token for block " + block);
     
     // check stored block state
-    BlockInfoContiguous storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
+    BlockInfo storedBlock = getStoredBlock(ExtendedBlock.getLocalBlock(block));
     if (storedBlock == null || 
         storedBlock.getBlockUCState() != BlockUCState.UNDER_CONSTRUCTION) {
         throw new IOException(block + 
@@ -6335,8 +6350,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     // check the vadility of the block and lease holder name
     final INodeFile pendingFile = checkUCBlock(oldBlock, clientName);
-    final BlockInfoContiguousUnderConstruction blockinfo
-        = (BlockInfoContiguousUnderConstruction)pendingFile.getLastBlock();
+    final BlockInfo lastBlock = pendingFile.getLastBlock();
+    // when updating pipeline, the last block must be contiguous block
+    assert lastBlock instanceof BlockInfoContiguousUnderConstruction;
+    BlockInfoContiguousUnderConstruction blockinfo =
+        (BlockInfoContiguousUnderConstruction) lastBlock;
 
     // check new GS & length: this is not expected
     if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() ||

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
index 1ebdde6..a8e2e00 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileUnderConstructionFeature.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hdfs.server.namenode;
 import java.io.IOException;
 
 import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo;
 
 /**
@@ -58,12 +60,12 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void updateLengthOfLastBlock(INodeFile f, long lastBlockLength)
       throws IOException {
-    BlockInfoContiguous lastBlock = f.getLastBlock();
+    BlockInfo lastBlock = f.getLastBlock();
     assert (lastBlock != null) : "The last block for path "
         + f.getFullPathName() + " is null when updating its length";
-    assert (lastBlock instanceof BlockInfoContiguousUnderConstruction)
+    assert !lastBlock.isComplete()
         : "The last block for path " + f.getFullPathName()
-            + " is not a BlockInfoUnderConstruction when updating its length";
+        + " is not a BlockInfoUnderConstruction when updating its length";
     lastBlock.setNumBytes(lastBlockLength);
   }
 
@@ -74,11 +76,10 @@ public class FileUnderConstructionFeature implements INode.Feature {
    */
   void cleanZeroSizeBlock(final INodeFile f,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfoContiguous[] blocks = f.getBlocks();
+    final BlockInfo[] blocks = f.getBlocks();
     if (blocks != null && blocks.length > 0
-        && blocks[blocks.length - 1] instanceof BlockInfoContiguousUnderConstruction) {
-      BlockInfoContiguousUnderConstruction lastUC =
-          (BlockInfoContiguousUnderConstruction) blocks[blocks.length - 1];
+        && !blocks[blocks.length - 1].isComplete()) {
+      BlockInfo lastUC = blocks[blocks.length - 1];
       if (lastUC.getNumBytes() == 0) {
         // this is a 0-sized block. do not need check its UC state here
         collectedBlocks.addDeleteBlock(lastUC);


[13/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903 and HDFS-7435. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903 and HDFS-7435. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/8336831b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/8336831b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/8336831b

Branch: refs/heads/HDFS-7285
Commit: 8336831bba78528a90c2df456f7c6ae161f57585
Parents: a6285fa
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 16 14:27:21 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:41:59 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/DecommissionManager.java | 2 +-
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java    | 2 +-
 .../hadoop/hdfs/server/namenode/snapshot/FileDiffList.java      | 3 ++-
 .../src/test/java/org/apache/hadoop/hdfs/TestDecommission.java  | 5 ++---
 .../hadoop/hdfs/server/namenode/TestAddStripedBlocks.java       | 4 ++--
 5 files changed, 8 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/8336831b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 3ea5311..546cc30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -535,7 +535,7 @@ public class DecommissionManager {
      */
     private void processBlocksForDecomInternal(
         final DatanodeDescriptor datanode,
-        final Iterator<BlockInfoContiguous> it,
+        final Iterator<? extends BlockInfo> it,
         final List<BlockInfoContiguous> insufficientlyReplicated,
         boolean pruneSufficientlyReplicated) {
       boolean firstReplicationLog = true;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8336831b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 2b23cd3..14e97a1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -2000,7 +2000,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Check if the file is already being truncated with the same length
-    final BlockInfoContiguous last = file.getLastBlock();
+    final BlockInfo last = file.getLastBlock();
     if (last != null && last.getBlockUCState() == BlockUCState.UNDER_RECOVERY) {
       final Block truncateBlock
           = ((BlockInfoContiguousUnderConstruction)last).getTruncateBlock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8336831b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index a1263c5..d0248eb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -21,6 +21,7 @@ import java.util.Collections;
 import java.util.List;
 
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -132,7 +133,7 @@ public class FileDiffList extends
       break;
     }
     // Check if last block is part of truncate recovery
-    BlockInfoContiguous lastBlock = file.getLastBlock();
+    BlockInfo lastBlock = file.getLastBlock();
     Block dontRemoveBlock = null;
     if(lastBlock != null && lastBlock.getBlockUCState().equals(
         HdfsServerConstants.BlockUCState.UNDER_RECOVERY)) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8336831b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
index 1ab7427..b233057 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDecommission.java
@@ -48,7 +48,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
@@ -671,8 +671,7 @@ public class TestDecommission {
       GenericTestUtils.waitFor(new Supplier<Boolean>() {
         @Override
         public Boolean get() {
-          BlockInfoContiguous info =
-              blockManager.getStoredBlock(b.getLocalBlock());
+          BlockInfo info = blockManager.getStoredBlock(b.getLocalBlock());
           int count = 0;
           StringBuilder sb = new StringBuilder("Replica locations: ");
           for (int i = 0; i < info.numNodes(); i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/8336831b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 8b51309..05aec4b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -265,9 +265,9 @@ public class TestAddStripedBlocks {
       ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null,
           null);
       blocks.add(replica);
-      BlockListAsLongs bll = new BlockListAsLongs(null, blocks);
+      BlockListAsLongs bll = BlockListAsLongs.encode(blocks);
       StorageBlockReport[] reports = {new StorageBlockReport(storage,
-          bll.getBlockListAsLongs())};
+          bll)};
       cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
           bpId, reports);
     }


[39/50] [abbrv] hadoop git commit: HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang. Updated CHANGES-HDFS-EC-7285.txt

Posted by zh...@apache.org.
HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang.
Updated CHANGES-HDFS-EC-7285.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b2306c4b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b2306c4b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b2306c4b

Branch: refs/heads/HDFS-7285
Commit: b2306c4bf6b1f83e08cb5382a912132345b41caf
Parents: 1a6bfbc
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 7 15:35:18 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:15:35 2015 -0700

----------------------------------------------------------------------
 hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt | 2 ++
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt     | 5 +----
 2 files changed, 3 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2306c4b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 68d1d32..7716728 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -33,5 +33,7 @@
     HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
     ( Xinwei Qin via Kai Zheng )
 
+    HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang)
+
     HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng
     ( Kai Zheng )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b2306c4b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 3874cb4..9927ccf 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -49,7 +49,4 @@
     (Hui Zheng via Zhe Zhang)
 
     HDFS-7839. Erasure coding: implement facilities in NameNode to create and
-    manage EC zones (Zhe Zhang)
-
-    HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang)
-
+    manage EC zones (Zhe Zhang)
\ No newline at end of file


[23/50] [abbrv] hadoop git commit: HDFS-7716. Add a test for BlockGroup support in FSImage. Contributed by Takuya Fukudome

Posted by zh...@apache.org.
HDFS-7716. Add a test for BlockGroup support in FSImage.  Contributed by Takuya Fukudome


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d77a788c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d77a788c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d77a788c

Branch: refs/heads/HDFS-7285
Commit: d77a788cf2f91fc93b6f619bc0f08173848eb109
Parents: bd9bf78
Author: Tsz-Wo Nicholas Sze <sz...@hortonworks.com>
Authored: Wed Mar 25 19:01:03 2015 +0900
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:52 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  6 ++-
 .../hdfs/server/namenode/TestFSImage.java       | 53 ++++++++++++++++++++
 2 files changed, 58 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77a788c/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 2ef8527..21e4c03 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -1,4 +1,8 @@
   BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS
 
     HDFS-7347. Configurable erasure coding policy for individual files and
-    directories ( Zhe Zhang via vinayakumarb )
\ No newline at end of file
+    directories ( Zhe Zhang via vinayakumarb )
+
+    HDFS-7716. Add a test for BlockGroup support in FSImage.
+    (Takuya Fukudome via szetszwo)
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d77a788c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 71dc978..440f5cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertNotNull;
 import static org.junit.Assert.assertTrue;
 
 import java.io.File;
@@ -31,7 +32,12 @@ import java.io.ByteArrayInputStream;
 import java.io.IOException;
 import java.util.EnumSet;
 
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
@@ -46,6 +52,7 @@ import org.apache.hadoop.fs.FileUtil;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSOutputStream;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
@@ -378,4 +385,50 @@ public class TestFSImage {
       FileUtil.fullyDelete(dfsDir);
     }
   }
+
+  /**
+   * Ensure that FSImage supports BlockGroup.
+   */
+  @Test
+  public void testSupportBlockGroup() throws IOException {
+    final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
+        HdfsConstants.NUM_PARITY_BLOCKS;
+    final int BLOCK_SIZE = 8 * 1024 * 1024;
+    Configuration conf = new HdfsConfiguration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      fs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+      Path file = new Path("/striped");
+      FSDataOutputStream out = fs.create(file);
+      byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);
+      out.write(bytes);
+      out.close();
+
+      fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER);
+      fs.saveNamespace();
+      fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE);
+
+      cluster.restartNameNodes();
+      fs = cluster.getFileSystem();
+      assertTrue(fs.exists(file));
+
+      // check the information of striped blocks
+      FSNamesystem fsn = cluster.getNamesystem();
+      INodeFile inode = fsn.dir.getINode(file.toString()).asFile();
+      FileWithStripedBlocksFeature sb = inode.getStripedBlocksFeature();
+      assertNotNull(sb);
+      BlockInfoStriped[] blks = sb.getBlocks();
+      assertEquals(1, blks.length);
+      assertTrue(blks[0].isStriped());
+      assertEquals(HdfsConstants.NUM_DATA_BLOCKS, blks[0].getDataBlockNum());
+      assertEquals(HdfsConstants.NUM_PARITY_BLOCKS, blks[0].getParityBlockNum());
+    } finally {
+      cluster.shutdown();
+    }
+  }
 }


[38/50] [abbrv] hadoop git commit: HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin Updated CHANGES-HDFS-EC-7285.txt

Posted by zh...@apache.org.
HADOOP-11782 Correct two thrown messages in ECSchema class. Contributed by Xinwei Qin
Updated CHANGES-HDFS-EC-7285.txt


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/1a6bfbc9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/1a6bfbc9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/1a6bfbc9

Branch: refs/heads/HDFS-7285
Commit: 1a6bfbc9583b081dfa443c757840b85c0e921736
Parents: d20e16e
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 7 15:34:37 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:15:35 2015 -0700

----------------------------------------------------------------------
 hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt | 3 ---
 1 file changed, 3 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/1a6bfbc9/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 4e60a7c..3874cb4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -48,9 +48,6 @@
     HDFS-7617. Add unit tests for editlog transactions for EC 
     (Hui Zheng via Zhe Zhang)
 
-    HADOOP-11782. Correct two thrown messages in ECSchema class
-    (Xinwei Qin via Kai Zheng)
-
     HDFS-7839. Erasure coding: implement facilities in NameNode to create and
     manage EC zones (Zhe Zhang)
 


[33/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for conflicts from HDFS-6945). Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for conflicts from HDFS-6945). Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/38fb3e15
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/38fb3e15
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/38fb3e15

Branch: refs/heads/HDFS-7285
Commit: 38fb3e15319904dfd8be4e13fcf39d881098200f
Parents: 784f9d3
Author: Zhe Zhang <zh...@apache.org>
Authored: Thu Apr 2 11:25:58 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:48:57 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java    | 2 +-
 1 file changed, 1 insertion(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/38fb3e15/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 78ada82..2c3f525 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -3678,7 +3678,7 @@ public class BlockManager {
   private void removeFromExcessReplicateMap(Block block) {
     for (DatanodeStorageInfo info : blocksMap.getStorages(block)) {
       String uuid = info.getDatanodeDescriptor().getDatanodeUuid();
-      LightWeightLinkedSet<Block> excessReplicas = excessReplicateMap.get(uuid);
+      LightWeightLinkedSet<BlockInfo> excessReplicas = excessReplicateMap.get(uuid);
       if (excessReplicas != null) {
         if (excessReplicas.remove(block)) {
           excessBlocksCount.decrementAndGet();


[34/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit is for HDFS-8035). Contributed by Zhe Zhang

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit is for HDFS-8035). Contributed by Zhe Zhang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/abbcfef6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/abbcfef6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/abbcfef6

Branch: refs/heads/HDFS-7285
Commit: abbcfef604890bba1cb28ac5c0a4aa755930e13c
Parents: 6e0a319
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 6 10:37:23 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:15:23 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java | 11 +++++------
 .../apache/hadoop/hdfs/server/namenode/FSNamesystem.java |  8 ++++----
 2 files changed, 9 insertions(+), 10 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/abbcfef6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 2c3f525..fcf1421 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -3540,13 +3540,12 @@ public class BlockManager {
       String src, BlockInfo[] blocks) {
     for (BlockInfo b: blocks) {
       if (!b.isComplete()) {
-        final BlockInfoContiguousUnderConstruction uc =
-            (BlockInfoContiguousUnderConstruction)b;
         final int numNodes = b.numNodes();
-        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = "
-          + uc.getBlockUCState() + ", replication# = " + numNodes
-          + (numNodes < minReplication ? " < ": " >= ")
-          + " minimum = " + minReplication + ") in file " + src);
+        final int min = getMinStorageNum(b);
+        final BlockUCState state = b.getBlockUCState();
+        LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " + state
+            + ", replication# = " + numNodes + (numNodes < min ? " < " : " >= ")
+            + " minimum = " + min + ") in file " + src);
         return false;
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/abbcfef6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 9ffb6fc..7c087d0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3133,7 +3133,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   LocatedBlock storeAllocatedBlock(String src, long fileId, String clientName,
       ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException {
-    BlockInfo newBlockInfo = null;
+    Block newBlock = null;
     long offset;
     checkOperation(OperationCategory.WRITE);
     waitForLoadingFSImage();
@@ -3166,8 +3166,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                                 ExtendedBlock.getLocalBlock(previous));
 
       // allocate new block, record block locations in INode.
-      Block newBlock = createNewBlock(isStriped);
-      newBlockInfo = saveAllocatedBlock(src, fileState.iip, newBlock, targets,
+      newBlock = createNewBlock(isStriped);
+      saveAllocatedBlock(src, fileState.iip, newBlock, targets,
           isStriped);
 
       persistNewBlock(src, pendingFile);
@@ -3178,7 +3178,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     getEditLog().logSync();
 
     // Return located block
-    return makeLocatedBlock(newBlockInfo, targets, offset);
+    return makeLocatedBlock(getStoredBlock(newBlock), targets, offset);
   }
 
   /*


[20/50] [abbrv] hadoop git commit: HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9c6e1f10
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9c6e1f10
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9c6e1f10

Branch: refs/heads/HDFS-7285
Commit: 9c6e1f10f074cf3278161d6ffc730c8423b1eeff
Parents: 04011ac
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Mar 20 19:15:52 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:51 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  3 +
 .../hadoop/fs/CommonConfigurationKeys.java      | 15 ++++
 .../erasurecode/coder/AbstractErasureCoder.java | 65 ++++++++++++++
 .../coder/AbstractErasureDecoder.java           |  6 +-
 .../coder/AbstractErasureEncoder.java           |  6 +-
 .../io/erasurecode/coder/RSErasureDecoder.java  | 83 ++++++++++++++++++
 .../io/erasurecode/coder/RSErasureEncoder.java  | 47 ++++++++++
 .../io/erasurecode/coder/XorErasureDecoder.java |  2 +-
 .../io/erasurecode/coder/XorErasureEncoder.java |  2 +-
 .../erasurecode/coder/TestRSErasureCoder.java   | 92 ++++++++++++++++++++
 10 files changed, 315 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index f566f0e..b69e69a 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -26,3 +26,6 @@
 
     HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng
     ( Kai Zheng )
+
+    HADOOP-11647. Reed-Solomon ErasureCoder. Contributed by Kai Zheng
+    ( Kai Zheng )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 7575496..70fea01 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -135,6 +135,21 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
       false;
 
   /**
+   * Erasure Coding configuration family
+   */
+
+  /** Supported erasure codec classes */
+  public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
+
+  /** Use XOR raw coder when possible for the RS codec */
+  public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
+      "io.erasurecode.codec.rs.usexor";
+
+  /** Raw coder factory for the RS codec */
+  public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY =
+      "io.erasurecode.codec.rs.rawcoder";
+
+  /**
    * Service Authorization
    */
   public static final String 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
index 8d3bc34..0e4de89 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -17,7 +17,12 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 
 /**
  * A common class of basic facilities to be shared by encoder and decoder
@@ -31,6 +36,66 @@ public abstract class AbstractErasureCoder
   private int numParityUnits;
   private int chunkSize;
 
+  /**
+   * Create raw decoder using the factory specified by rawCoderFactoryKey
+   * @param rawCoderFactoryKey
+   * @return raw decoder
+   */
+  protected RawErasureDecoder createRawDecoder(String rawCoderFactoryKey) {
+    RawErasureCoder rawCoder = createRawCoder(getConf(),
+        rawCoderFactoryKey, false);
+    return (RawErasureDecoder) rawCoder;
+  }
+
+  /**
+   * Create raw encoder using the factory specified by rawCoderFactoryKey
+   * @param rawCoderFactoryKey
+   * @return raw encoder
+   */
+  protected RawErasureEncoder createRawEncoder(String rawCoderFactoryKey) {
+    RawErasureCoder rawCoder = createRawCoder(getConf(),
+        rawCoderFactoryKey, true);
+    return (RawErasureEncoder) rawCoder;
+  }
+
+  /**
+   * Create raw coder using specified conf and raw coder factory key.
+   * @param conf
+   * @param rawCoderFactoryKey
+   * @param isEncoder
+   * @return raw coder
+   */
+  protected static RawErasureCoder createRawCoder(Configuration conf,
+      String rawCoderFactoryKey, boolean isEncoder) {
+
+    if (conf == null) {
+      return null;
+    }
+
+    Class<? extends RawErasureCoderFactory> factClass = null;
+    factClass = conf.getClass(rawCoderFactoryKey,
+        factClass, RawErasureCoderFactory.class);
+
+    if (factClass == null) {
+      return null;
+    }
+
+    RawErasureCoderFactory fact;
+    try {
+      fact = factClass.newInstance();
+    } catch (InstantiationException e) {
+      throw new RuntimeException("Failed to create raw coder", e);
+    } catch (IllegalAccessException e) {
+      throw new RuntimeException("Failed to create raw coder", e);
+    }
+
+    if (fact != null) {
+      return isEncoder ? fact.createEncoder() : fact.createDecoder();
+    }
+
+    return null;
+  }
+
   @Override
   public void initialize(int numDataUnits, int numParityUnits,
                          int chunkSize) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
index df71528..54a6d1e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -30,7 +30,8 @@ public abstract class AbstractErasureDecoder extends AbstractErasureCoder
 
   @Override
   public ErasureCodingStep decode(ECBlockGroup blockGroup) {
-    return performDecoding(blockGroup);
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareDecodingStep(blockGroup);
   }
 
   /**
@@ -38,7 +39,8 @@ public abstract class AbstractErasureDecoder extends AbstractErasureCoder
    * @param blockGroup
    * @return decoding step for caller to do the real work
    */
-  protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup);
+  protected abstract ErasureCodingStep prepareDecodingStep(
+      ECBlockGroup blockGroup);
 
   /**
    * We have all the data blocks and parity blocks as input blocks for

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
index 80b9875..09b31e5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
@@ -30,7 +30,8 @@ public abstract class AbstractErasureEncoder extends AbstractErasureCoder
 
   @Override
   public ErasureCodingStep encode(ECBlockGroup blockGroup) {
-    return performEncoding(blockGroup);
+    // We may have more than this when considering complicate cases. HADOOP-11550
+    return prepareEncodingStep(blockGroup);
   }
 
   /**
@@ -38,7 +39,8 @@ public abstract class AbstractErasureEncoder extends AbstractErasureCoder
    * @param blockGroup
    * @return encoding step for caller to do the real work
    */
-  protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup);
+  protected abstract ErasureCodingStep prepareEncodingStep(
+      ECBlockGroup blockGroup);
 
   protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
     return blockGroup.getDataBlocks();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
new file mode 100644
index 0000000..ba32f04
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -0,0 +1,83 @@
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
+
+/**
+ * Reed-Solomon erasure decoder that decodes a block group.
+ *
+ * It implements {@link ErasureDecoder}.
+ */
+public class RSErasureDecoder extends AbstractErasureDecoder {
+  private RawErasureDecoder rsRawDecoder;
+  private RawErasureDecoder xorRawDecoder;
+  private boolean useXorWhenPossible = true;
+
+  @Override
+  public void setConf(Configuration conf) {
+    super.setConf(conf);
+
+    if (conf != null) {
+      this.useXorWhenPossible = conf.getBoolean(
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, true);
+    }
+  }
+
+    @Override
+  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
+
+    RawErasureDecoder rawDecoder;
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+    ECBlock[] outputBlocks = getOutputBlocks(blockGroup);
+
+    /**
+     * Optimization: according to some benchmark, when only one block is erased
+     * and to be recovering, the most simple XOR scheme can be much efficient.
+     * We will have benchmark tests to verify this opt is effect or not.
+     */
+    if (outputBlocks.length == 1 && useXorWhenPossible) {
+      rawDecoder = checkCreateXorRawDecoder();
+    } else {
+      rawDecoder = checkCreateRSRawDecoder();
+    }
+
+    return new ErasureDecodingStep(inputBlocks,
+        getErasedIndexes(inputBlocks), outputBlocks, rawDecoder);
+  }
+
+  private RawErasureDecoder checkCreateRSRawDecoder() {
+    if (rsRawDecoder == null) {
+      rsRawDecoder = createRawDecoder(
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
+      if (rsRawDecoder == null) {
+        rsRawDecoder = new JRSRawDecoder();
+      }
+      rsRawDecoder.initialize(getNumDataUnits(),
+          getNumParityUnits(), getChunkSize());
+    }
+    return rsRawDecoder;
+  }
+
+  private RawErasureDecoder checkCreateXorRawDecoder() {
+    if (xorRawDecoder == null) {
+      xorRawDecoder = new XorRawDecoder();
+      xorRawDecoder.initialize(getNumDataUnits(), 1, getChunkSize());
+    }
+    return xorRawDecoder;
+  }
+
+  @Override
+  public void release() {
+    if (xorRawDecoder != null) {
+      xorRawDecoder.release();
+    } else if (rsRawDecoder != null) {
+      rsRawDecoder.release();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
new file mode 100644
index 0000000..430749d
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -0,0 +1,47 @@
+package org.apache.hadoop.io.erasurecode.coder;
+
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Reed-Solomon erasure encoder that encodes a block group.
+ *
+ * It implements {@link ErasureEncoder}.
+ */
+public class RSErasureEncoder extends AbstractErasureEncoder {
+  private RawErasureEncoder rawEncoder;
+
+  @Override
+  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
+
+    RawErasureEncoder rawEncoder = checkCreateRSRawEncoder();
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+
+  private RawErasureEncoder checkCreateRSRawEncoder() {
+    if (rawEncoder == null) {
+      rawEncoder = createRawEncoder(
+          CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY);
+      if (rawEncoder == null) {
+        rawEncoder = new JRSRawEncoder();
+      }
+      rawEncoder.initialize(getNumDataUnits(),
+          getNumParityUnits(), getChunkSize());
+    }
+    return rawEncoder;
+  }
+
+  @Override
+  public void release() {
+    if (rawEncoder != null) {
+      rawEncoder.release();
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
index 9a6f6e2..33f5386 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
 public class XorErasureDecoder extends AbstractErasureDecoder {
 
   @Override
-  protected ErasureCodingStep performDecoding(final ECBlockGroup blockGroup) {
+  protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) {
     // May be configured
     RawErasureDecoder rawDecoder = new XorRawDecoder();
     rawDecoder.initialize(getNumDataUnits(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
index db6e3db..f8d67c3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
@@ -30,7 +30,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XorRawEncoder;
 public class XorErasureEncoder extends AbstractErasureEncoder {
 
   @Override
-  protected ErasureCodingStep performEncoding(final ECBlockGroup blockGroup) {
+  protected ErasureCodingStep prepareEncodingStep(final ECBlockGroup blockGroup) {
     // May be configured
     RawErasureEncoder rawEncoder = new XorRawEncoder();
     rawEncoder.initialize(getNumDataUnits(),

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9c6e1f10/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java
new file mode 100644
index 0000000..8a7561c
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.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.io.erasurecode.coder;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.apache.hadoop.io.erasurecode.rawcoder.JRSRawErasureCoderFactory;
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test Reed-Solomon encoding and decoding.
+ */
+public class TestRSErasureCoder extends TestErasureCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = RSErasureEncoder.class;
+    this.decoderClass = RSErasureDecoder.class;
+
+    this.numDataUnits = 10;
+    this.numParityUnits = 1;
+
+    this.numChunksInBlock = 10;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4() {
+    prepare(null, 10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBufferWithConf_10x4() {
+    /**
+     * This tests if the two configuration items work or not.
+     */
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY,
+        JRSRawErasureCoderFactory.class.getCanonicalName());
+    conf.setBoolean(
+        CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_USEXOR_KEY, false);
+    prepare(conf, 10, 4, null);
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
+    prepare(null, 10, 4, new int[] {2, 4});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_10x4_erasing_all() {
+    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
+    testCoding(true);
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer_3x3() {
+    prepare(null, 3, 3, null);
+    testCoding(true);
+  }
+
+}


[09/50] [abbrv] hadoop git commit: Fixed a compiling issue introduced by HADOOP-11705.

Posted by zh...@apache.org.
Fixed a compiling issue introduced by HADOOP-11705.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/a6285fa0
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/a6285fa0
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/a6285fa0

Branch: refs/heads/HDFS-7285
Commit: a6285fa076e1506fcea78434d3b3961ff2b9e7a1
Parents: d984ffb
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Mar 13 00:13:06 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:41:58 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/a6285fa0/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
index 36e061a..d911db9 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -162,7 +162,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    encoder.setConf(conf);
+    ((AbstractErasureCoder)encoder).setConf(conf);
     return encoder;
   }
 
@@ -179,7 +179,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    decoder.setConf(conf);
+    ((AbstractErasureCoder)decoder).setConf(conf);
     return decoder;
   }
 


[45/50] [abbrv] hadoop git commit: HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. Contributed by Jing Zhao and Zhe Zhang.

Posted by zh...@apache.org.
HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. Contributed by Jing Zhao and Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/81d1e700
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/81d1e700
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/81d1e700

Branch: refs/heads/HDFS-7285
Commit: 81d1e700676022763cd51bc676771fa2dda974e9
Parents: ac5a46d
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Apr 9 17:53:22 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:57 2015 -0700

----------------------------------------------------------------------
 .../BlockInfoStripedUnderConstruction.java         |  2 +-
 .../hdfs/server/blockmanagement/BlockManager.java  | 12 ++++++------
 .../server/namenode/ErasureCodingZoneManager.java  |  7 +++++++
 .../hadoop/hdfs/server/namenode/FSDirectory.java   |  4 ++--
 .../hdfs/server/namenode/FSEditLogLoader.java      | 11 ++++++-----
 .../hdfs/server/namenode/FSImageSerialization.java |  4 ++--
 .../hadoop/hdfs/server/namenode/INodeFile.java     | 17 ++++-------------
 .../hdfs/server/namenode/TestFSEditLogLoader.java  |  4 ++--
 .../hadoop/hdfs/server/namenode/TestFSImage.java   |  2 +-
 .../server/namenode/TestRecoverStripedBlocks.java  |  2 +-
 10 files changed, 32 insertions(+), 33 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
index cfaf3a0..0373314 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
@@ -96,7 +96,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
     for(int i = 0; i < numLocations; i++) {
       // when creating a new block we simply sequentially assign block index to
       // each storage
-      Block blk = new Block(this.getBlockId() + i, this.getGenerationStamp(), 0);
+      Block blk = new Block(this.getBlockId() + i, 0, this.getGenerationStamp());
       replicas[i] = new ReplicaUnderConstruction(blk, targets[i],
           ReplicaState.RBW);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index fcf1421..94aafc7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2450,12 +2450,12 @@ public class BlockManager {
       case COMMITTED:
         if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
               "block is " + ucState + " and reported genstamp " + reportedGS
               + " does not match genstamp in block map "
               + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
         } else if (storedBlock.getNumBytes() != reported.getNumBytes()) {
-          return new BlockToMarkCorrupt(reported, storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "block is " + ucState + " and reported length " +
               reported.getNumBytes() + " does not match " +
               "length in block map " + storedBlock.getNumBytes(),
@@ -2466,7 +2466,7 @@ public class BlockManager {
       case UNDER_CONSTRUCTION:
         if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) {
           final long reportedGS = reported.getGenerationStamp();
-          return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
               "block is " + ucState + " and reported state " + reportedState
               + ", But reported genstamp " + reportedGS
               + " does not match genstamp in block map "
@@ -2482,7 +2482,7 @@ public class BlockManager {
         return null; // not corrupt
       } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) {
         final long reportedGS = reported.getGenerationStamp();
-        return new BlockToMarkCorrupt(reported, storedBlock, reportedGS,
+        return new BlockToMarkCorrupt(new Block(reported), storedBlock, reportedGS,
             "reported " + reportedState + " replica with genstamp " + reportedGS
             + " does not match COMPLETE block's genstamp in block map "
             + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
@@ -2497,7 +2497,7 @@ public class BlockManager {
               "complete with the same genstamp");
           return null;
         } else {
-          return new BlockToMarkCorrupt(reported, storedBlock,
+          return new BlockToMarkCorrupt(new Block(reported), storedBlock,
               "reported replica has invalid state " + reportedState,
               Reason.INVALID_STATE);
         }
@@ -2510,7 +2510,7 @@ public class BlockManager {
       " on " + dn + " size " + storedBlock.getNumBytes();
       // log here at WARN level since this is really a broken HDFS invariant
       LOG.warn(msg);
-      return new BlockToMarkCorrupt(reported, storedBlock, msg,
+      return new BlockToMarkCorrupt(new Block(reported), storedBlock, msg,
           Reason.INVALID_STATE);
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
index d4ff7c5..606e804 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -59,6 +59,13 @@ public class ErasureCodingZoneManager {
       if (inode == null) {
         continue;
       }
+      // We don't allow symlinks in an EC zone, or pointing to a file/dir in
+      // an EC. Therefore if a symlink is encountered, the dir shouldn't have
+      // EC
+      // TODO: properly support symlinks in EC zones
+      if (inode.isSymlink()) {
+        return false;
+      }
       final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
           new ArrayList<XAttr>(0)
           : inode.getXAttrFeature().getXAttrs();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 53d2040..d68128a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -468,8 +468,8 @@ public class FSDirectory implements Closeable {
     try {
       INodesInPath iip = addINode(existing, newNode);
       if (iip != null) {
-        // TODO: we will no longer use storage policy for "Erasure Coding Zone"
-        if (newNode.isStriped()) {
+        // check if the file is in an EC zone
+        if (getECPolicy(iip)) {
           newNode.addStripedBlocksFeature();
         }
         if (aclEntries != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 89cfe05..f530772 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -417,7 +417,7 @@ public class FSEditLogLoader {
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped());
+      updateBlocks(fsDir, addCloseOp, iip, newFile, fsDir.getECPolicy(iip));
       break;
     }
     case OP_CLOSE: {
@@ -438,7 +438,7 @@ public class FSEditLogLoader {
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped());
+      updateBlocks(fsDir, addCloseOp, iip, file, fsDir.getECPolicy(iip));
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -497,7 +497,7 @@ public class FSEditLogLoader {
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
       // TODO whether the file is striped should later be retrieved from iip
-      updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped());
+      updateBlocks(fsDir, updateOp, iip, oldFile, fsDir.getECPolicy(iip));
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -511,10 +511,11 @@ public class FSEditLogLoader {
         FSNamesystem.LOG.debug(op.opCode + ": " + path +
             " new block id : " + addBlockOp.getLastBlock().getBlockId());
       }
-      INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
+      INodesInPath iip = fsDir.getINodesInPath(path, true);
+      INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // add the new block to the INodeFile
       // TODO whether the file is striped should later be retrieved from iip
-      addNewBlock(addBlockOp, oldFile, oldFile.isStriped());
+      addNewBlock(addBlockOp, oldFile, fsDir.getECPolicy(iip));
       break;
     }
     case OP_SET_REPLICATION: {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
index 1e58858..58244e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageSerialization.java
@@ -207,7 +207,7 @@ public class FSImageSerialization {
     out.writeLong(cons.getModificationTime());
     out.writeLong(cons.getPreferredBlockSize());
     // whether the file has striped blocks
-    out.writeBoolean(cons.isWithStripedBlocks());
+    out.writeBoolean(cons.isStriped());
     writeBlocks(cons.getBlocks(), out);
     cons.getPermissionStatus().write(out);
 
@@ -233,7 +233,7 @@ public class FSImageSerialization {
     out.writeLong(file.getAccessTime());
     out.writeLong(file.getPreferredBlockSize());
     // whether the file has striped blocks
-    out.writeBoolean(file.isWithStripedBlocks());
+    out.writeBoolean(file.isStriped());
     writeBlocks(file.getBlocks(), out);
     SnapshotFSImageFormat.saveFileDiffList(file, out);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index f95e54e..b5c510e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -185,17 +185,13 @@ public class INodeFile extends INodeWithAdditionalFields
   public FileWithStripedBlocksFeature addStripedBlocksFeature() {
     assert blocks == null || blocks.length == 0:
         "The file contains contiguous blocks";
-    assert !isWithStripedBlocks();
+    assert !isStriped();
     this.setFileReplication((short) 0);
     FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature();
     addFeature(sb);
     return sb;
   }
 
-  public boolean isWithStripedBlocks() {
-    return getStripedBlocksFeature() != null;
-  }
-
   /** Used to make sure there is no contiguous block related info */
   private boolean hasNoContiguousBlock() {
     return (blocks == null || blocks.length == 0) && getFileReplication() == 0;
@@ -431,7 +427,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication,
       int latestSnapshotId) throws QuotaExceededException {
-    Preconditions.checkState(!isWithStripedBlocks(),
+    Preconditions.checkState(!isStriped(),
         "Cannot set replication to a file with striped blocks");
     recordModification(latestSnapshotId);
     setFileReplication(replication);
@@ -653,7 +649,7 @@ public class INodeFile extends INodeWithAdditionalFields
     long nsDelta = 1;
     final long ssDeltaNoReplication;
     short replication;
-    if (isWithStripedBlocks()) {
+    if (isStriped()) {
       return computeQuotaUsageWithStriped(bsps, counts);
     }
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
@@ -695,11 +691,6 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /**
    * Compute quota of striped file
-   * @param bsps
-   * @param counts
-   * @param useCache
-   * @param lastSnapshotId
-   * @return quota counts
    */
   public final QuotaCounts computeQuotaUsageWithStriped(
       BlockStoragePolicySuite bsps, QuotaCounts counts) {
@@ -828,7 +819,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * Use preferred block size for the last block if it is under construction.
    */
   public final long storagespaceConsumed() {
-    if (isWithStripedBlocks()) {
+    if (isStriped()) {
       return storagespaceConsumedWithStriped();
     } else {
       return storagespaceConsumedNoReplication() * getBlockReplication();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 407d07e..0eeb7f8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -472,7 +472,7 @@ public class TestFSEditLogLoader {
       INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
           .getINode(testFilePath);
 
-      assertTrue(inodeLoaded.isWithStripedBlocks());
+      assertTrue(inodeLoaded.isStriped());
 
       BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
       assertEquals(1, blks.length);
@@ -551,7 +551,7 @@ public class TestFSEditLogLoader {
       INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
           .getINode(testFilePath);
 
-      assertTrue(inodeLoaded.isWithStripedBlocks());
+      assertTrue(inodeLoaded.isStriped());
 
       BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
       assertEquals(1, blks.length);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 83f01c6..a456cad 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -199,7 +199,7 @@ public class TestFSImage {
     assertEquals(mtime, fileByLoaded.getModificationTime());
     assertEquals(isUC ? mtime : atime, fileByLoaded.getAccessTime());
     assertEquals(0, fileByLoaded.getContiguousBlocks().length);
-    assertEquals(0, fileByLoaded.getBlockReplication());
+    assertEquals(0, fileByLoaded.getFileReplication());
     assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize());
 
     //check the BlockInfoStriped

http://git-wip-us.apache.org/repos/asf/hadoop/blob/81d1e700/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index b2ff6c8..4292f9a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -84,7 +84,7 @@ public class TestRecoverStripedBlocks {
     final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
         .getINode4Write(filePath.toString()).asFile();
     assertFalse(fileNode.isUnderConstruction());
-    assertTrue(fileNode.isWithStripedBlocks());
+    assertTrue(fileNode.isStriped());
     BlockInfo[] blocks = fileNode.getBlocks();
     assertEquals(numBlocks, blocks.length);
     for (BlockInfo blk : blocks) {


[47/50] [abbrv] hadoop git commit: HADOOP-11818 Minor improvements for erasurecode classes. Contributed by Rakesh R

Posted by zh...@apache.org.
HADOOP-11818 Minor improvements for erasurecode classes. Contributed by Rakesh R


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ac5a46d7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ac5a46d7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ac5a46d7

Branch: refs/heads/HDFS-7285
Commit: ac5a46d733c43202013729dc6a1f357b109d315a
Parents: cb237e4
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Apr 10 04:31:48 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:57 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt       |  2 ++
 .../hadoop/io/erasurecode/SchemaLoader.java      | 12 ++++++------
 .../io/erasurecode/coder/RSErasureDecoder.java   | 19 ++++++++++++++++++-
 .../io/erasurecode/coder/RSErasureEncoder.java   | 19 ++++++++++++++++++-
 .../io/erasurecode/coder/XORErasureDecoder.java  |  2 +-
 .../io/erasurecode/rawcoder/util/RSUtil.java     | 17 +++++++++++++++++
 6 files changed, 62 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac5a46d7/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index c72394e..b850e11 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -40,3 +40,5 @@
 
     HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code
     ( Kai Zheng via vinayakumarb )
+  
+    HADOOP-11818. Minor improvements for erasurecode classes. (Rakesh R via Kai Zheng)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac5a46d7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
index c51ed37..75dd03a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
@@ -17,8 +17,8 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.CommonConfigurationKeys;
 import org.w3c.dom.*;
@@ -36,7 +36,7 @@ import java.util.*;
  * A EC schema loading utility that loads predefined EC schemas from XML file
  */
 public class SchemaLoader {
-  private static final Log LOG = LogFactory.getLog(SchemaLoader.class.getName());
+  private static final Logger LOG = LoggerFactory.getLogger(SchemaLoader.class.getName());
 
   /**
    * Load predefined ec schemas from configuration file. This file is
@@ -63,7 +63,7 @@ public class SchemaLoader {
   private List<ECSchema> loadSchema(File schemaFile)
       throws ParserConfigurationException, IOException, SAXException {
 
-    LOG.info("Loading predefined EC schema file " + schemaFile);
+    LOG.info("Loading predefined EC schema file {}", schemaFile);
 
     // Read and parse the schema file.
     DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
@@ -87,7 +87,7 @@ public class SchemaLoader {
           ECSchema schema = loadSchema(element);
             schemas.add(schema);
         } else {
-          LOG.warn("Bad element in EC schema configuration file: " +
+          LOG.warn("Bad element in EC schema configuration file: {}",
               element.getTagName());
         }
       }
@@ -109,7 +109,7 @@ public class SchemaLoader {
       URL url = Thread.currentThread().getContextClassLoader()
           .getResource(schemaFilePath);
       if (url == null) {
-        LOG.warn(schemaFilePath + " not found on the classpath.");
+        LOG.warn("{} not found on the classpath.", schemaFilePath);
         schemaFile = null;
       } else if (! url.getProtocol().equalsIgnoreCase("file")) {
         throw new RuntimeException(

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac5a46d7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
index e2c5051..fc664a5 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java
@@ -1,3 +1,20 @@
+/**
+ * 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.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
@@ -11,7 +28,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 /**
  * Reed-Solomon erasure decoder that decodes a block group.
  *
- * It implements {@link ErasureDecoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class RSErasureDecoder extends AbstractErasureDecoder {
   private RawErasureDecoder rsRawDecoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac5a46d7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
index a7d02b5..18ca5ac 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java
@@ -1,3 +1,20 @@
+/**
+ * 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.io.erasurecode.coder;
 
 import org.apache.hadoop.fs.CommonConfigurationKeys;
@@ -9,7 +26,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
 /**
  * Reed-Solomon erasure encoder that encodes a block group.
  *
- * It implements {@link ErasureEncoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class RSErasureEncoder extends AbstractErasureEncoder {
   private RawErasureEncoder rawEncoder;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac5a46d7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
index 6f4b423..0672549 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java
@@ -25,7 +25,7 @@ import org.apache.hadoop.io.erasurecode.rawcoder.XORRawDecoder;
 /**
  * Xor erasure decoder that decodes a block group.
  *
- * It implements {@link ErasureDecoder}.
+ * It implements {@link ErasureCoder}.
  */
 public class XORErasureDecoder extends AbstractErasureDecoder {
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ac5a46d7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
index 33ba561..8badf02 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.java
@@ -1,3 +1,20 @@
+/**
+ * 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.io.erasurecode.rawcoder.util;
 
 /**


[31/50] [abbrv] hadoop git commit: HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones. Contributed by Zhe Zhang

Posted by zh...@apache.org.
HDFS-7839. Erasure coding: implement facilities in NameNode to create and manage EC zones. Contributed by Zhe Zhang


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ed06d9e9
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ed06d9e9
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ed06d9e9

Branch: refs/heads/HDFS-7285
Commit: ed06d9e92cc56e079af7ecbb436aa77f7997a7f8
Parents: 38fb3e1
Author: Zhe Zhang <zh...@apache.org>
Authored: Thu Apr 2 22:38:29 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:48:57 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  15 ++
 .../hadoop/hdfs/protocol/ClientProtocol.java    |   8 +
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   2 -
 ...tNamenodeProtocolServerSideTranslatorPB.java |  14 ++
 .../ClientNamenodeProtocolTranslatorPB.java     |  16 ++
 .../BlockStoragePolicySuite.java                |   5 -
 .../hdfs/server/common/HdfsServerConstants.java |   2 +
 .../namenode/ErasureCodingZoneManager.java      | 112 ++++++++++++++
 .../hdfs/server/namenode/FSDirRenameOp.java     |   2 +
 .../hdfs/server/namenode/FSDirectory.java       |  26 +++-
 .../hdfs/server/namenode/FSNamesystem.java      |  40 +++++
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  10 +-
 .../hdfs/server/namenode/NameNodeRpcServer.java |  16 ++
 .../src/main/proto/ClientNamenodeProtocol.proto |   9 ++
 .../hadoop/hdfs/TestBlockStoragePolicy.java     |  12 +-
 .../hadoop/hdfs/TestErasureCodingZones.java     | 151 +++++++++++++++++++
 .../TestBlockInitialEncoding.java               |  75 ---------
 .../server/namenode/TestAddStripedBlocks.java   |   2 +-
 .../server/namenode/TestFSEditLogLoader.java    |   6 +-
 .../hdfs/server/namenode/TestFSImage.java       |  23 ++-
 .../namenode/TestRecoverStripedBlocks.java      |   7 +-
 21 files changed, 431 insertions(+), 122 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index f79d160..cd06e09 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -2950,6 +2950,21 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return new EncryptionZoneIterator(namenode, traceSampler);
   }
 
+  public void createErasureCodingZone(String src)
+      throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("createErasureCodingZone", src);
+    try {
+      namenode.createErasureCodingZone(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          SafeModeException.class,
+          UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
   public void setXAttr(String src, String name, byte[] value, 
       EnumSet<XAttrSetFlag> flag) throws IOException {
     checkOpen();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index bafb02b..8efe344 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1363,6 +1363,14 @@ public interface ClientProtocol {
       long prevId) throws IOException;
 
   /**
+   * Create an erasure coding zone (currently with hardcoded schema)
+   * TODO: Configurable and pluggable schemas (HDFS-7337)
+   */
+  @Idempotent
+  public void createErasureCodingZone(String src)
+      throws IOException;
+
+  /**
    * Set xattr of a file or directory.
    * The name must be prefixed with the namespace followed by ".". For example,
    * "user.attr".

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index 07b72e6..b2882af 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -171,7 +171,6 @@ public class HdfsConstants {
   public static final String ONESSD_STORAGE_POLICY_NAME = "ONE_SSD";
   public static final String HOT_STORAGE_POLICY_NAME = "HOT";
   public static final String WARM_STORAGE_POLICY_NAME = "WARM";
-  public static final String EC_STORAGE_POLICY_NAME = "EC";
   public static final String COLD_STORAGE_POLICY_NAME = "COLD";
 
   public static final byte MEMORY_STORAGE_POLICY_ID = 15;
@@ -179,7 +178,6 @@ public class HdfsConstants {
   public static final byte ONESSD_STORAGE_POLICY_ID = 10;
   public static final byte HOT_STORAGE_POLICY_ID = 7;
   public static final byte WARM_STORAGE_POLICY_ID = 5;
-  public static final byte EC_STORAGE_POLICY_ID = 4;
   public static final byte COLD_STORAGE_POLICY_ID = 2;
 
   public static final byte NUM_DATA_BLOCKS = 3;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index cf51e3a..a209539 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -191,6 +191,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathResponseProto;
@@ -1391,6 +1393,18 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
   }
 
   @Override
+  public CreateErasureCodingZoneResponseProto createErasureCodingZone(
+      RpcController controller, CreateErasureCodingZoneRequestProto req)
+      throws ServiceException {
+    try {
+      server.createErasureCodingZone(req.getSrc());
+      return CreateErasureCodingZoneResponseProto.newBuilder().build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
+
+  @Override
   public SetXAttrResponseProto setXAttr(RpcController controller,
       SetXAttrRequestProto req) throws ServiceException {
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 22da083..43a0322 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -160,6 +160,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Trunca
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdateBlockForPipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.UpdatePipelineRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
@@ -1407,6 +1409,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
   }
 
   @Override
+  public void createErasureCodingZone(String src)
+      throws IOException {
+    final CreateErasureCodingZoneRequestProto.Builder builder =
+        CreateErasureCodingZoneRequestProto.newBuilder();
+    builder.setSrc(src);
+    CreateErasureCodingZoneRequestProto req = builder.build();
+    try {
+      rpcProxy.createErasureCodingZone(null, req);
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
+
+  @Override
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
     SetXAttrRequestProto req = SetXAttrRequestProto.newBuilder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
index 3d121cc..020cb5f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockStoragePolicySuite.java
@@ -78,11 +78,6 @@ public class BlockStoragePolicySuite {
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE},
         new StorageType[]{StorageType.DISK, StorageType.ARCHIVE});
-    final byte ecId = HdfsConstants.EC_STORAGE_POLICY_ID;
-    policies[ecId] = new BlockStoragePolicy(ecId,
-        HdfsConstants.EC_STORAGE_POLICY_NAME,
-        new StorageType[]{StorageType.DISK}, StorageType.EMPTY_ARRAY,
-        new StorageType[]{StorageType.ARCHIVE});
     final byte coldId = HdfsConstants.COLD_STORAGE_POLICY_ID;
     policies[coldId] = new BlockStoragePolicy(coldId,
         HdfsConstants.COLD_STORAGE_POLICY_NAME,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
index 2d267ce..8fd0197 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/common/HdfsServerConstants.java
@@ -327,4 +327,6 @@ public final class HdfsServerConstants {
       "raw.hdfs.crypto.file.encryption.info";
   public static final String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER =
       "security.hdfs.unreadable.by.superuser";
+  public static final String XATTR_ERASURECODING_ZONE =
+      "raw.hdfs.erasurecoding.zone";
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
new file mode 100644
index 0000000..d4ff7c5
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingZoneManager.java
@@ -0,0 +1,112 @@
+/**
+ * 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.server.namenode;
+
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
+import org.apache.hadoop.fs.XAttr;
+import org.apache.hadoop.fs.XAttrSetFlag;
+import org.apache.hadoop.hdfs.XAttrHelper;
+
+import java.io.IOException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_ZONE;
+
+/**
+ * Manages the list of erasure coding zones in the filesystem.
+ * <p/>
+ * The ErasureCodingZoneManager has its own lock, but relies on the FSDirectory
+ * lock being held for many operations. The FSDirectory lock should not be
+ * taken if the manager lock is already held.
+ * TODO: consolidate zone logic w/ encrypt. zones {@link EncryptionZoneManager}
+ */
+public class ErasureCodingZoneManager {
+  private final FSDirectory dir;
+
+  /**
+   * Construct a new ErasureCodingZoneManager.
+   *
+   * @param dir Enclosing FSDirectory
+   */
+  public ErasureCodingZoneManager(FSDirectory dir) {
+    this.dir = dir;
+  }
+
+  boolean getECPolicy(INodesInPath iip) {
+    assert dir.hasReadLock();
+    Preconditions.checkNotNull(iip);
+    List<INode> inodes = iip.getReadOnlyINodes();
+    for (int i = inodes.size() - 1; i >= 0; i--) {
+      final INode inode = inodes.get(i);
+      if (inode == null) {
+        continue;
+      }
+      final List<XAttr> xAttrs = inode.getXAttrFeature() == null ?
+          new ArrayList<XAttr>(0)
+          : inode.getXAttrFeature().getXAttrs();
+      for (XAttr xAttr : xAttrs) {
+        if (XATTR_ERASURECODING_ZONE.equals(XAttrHelper.getPrefixName(xAttr))) {
+          return true;
+        }
+      }
+    }
+    return false;
+  }
+
+  XAttr createErasureCodingZone(String src)
+      throws IOException {
+    assert dir.hasWriteLock();
+    final INodesInPath srcIIP = dir.getINodesInPath4Write(src, false);
+    if (dir.isNonEmptyDirectory(srcIIP)) {
+      throw new IOException(
+          "Attempt to create an erasure coding zone for a " +
+              "non-empty directory.");
+    }
+    if (srcIIP != null &&
+        srcIIP.getLastINode() != null &&
+        !srcIIP.getLastINode().isDirectory()) {
+      throw new IOException("Attempt to create an erasure coding zone " +
+          "for a file.");
+    }
+    if (getECPolicy(srcIIP)) {
+      throw new IOException("Directory " + src + " is already in an " +
+          "erasure coding zone.");
+    }
+    final XAttr ecXAttr = XAttrHelper
+        .buildXAttr(XATTR_ERASURECODING_ZONE, null);
+    final List<XAttr> xattrs = Lists.newArrayListWithCapacity(1);
+    xattrs.add(ecXAttr);
+    FSDirXAttrOp.unprotectedSetXAttrs(dir, src, xattrs,
+        EnumSet.of(XAttrSetFlag.CREATE));
+    return ecXAttr;
+  }
+
+  void checkMoveValidity(INodesInPath srcIIP, INodesInPath dstIIP, String src)
+      throws IOException {
+    assert dir.hasReadLock();
+    if (getECPolicy(srcIIP)
+        != getECPolicy(dstIIP)) {
+      throw new IOException(
+          src + " can't be moved because the source and destination have " +
+              "different erasure coding policies.");
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
index c57cae2..c38c5c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirRenameOp.java
@@ -185,6 +185,7 @@ class FSDirRenameOp {
     }
 
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
+    fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src);
     // Ensure dst has quota to accommodate rename
     verifyFsLimitsForRename(fsd, srcIIP, dstIIP);
     verifyQuotaForRename(fsd, srcIIP, dstIIP);
@@ -358,6 +359,7 @@ class FSDirRenameOp {
 
     BlockStoragePolicySuite bsps = fsd.getBlockStoragePolicySuite();
     fsd.ezManager.checkMoveValidity(srcIIP, dstIIP, src);
+    fsd.ecZoneManager.checkMoveValidity(srcIIP, dstIIP, src);
     final INode dstInode = dstIIP.getLastINode();
     List<INodeDirectory> snapshottableDirs = new ArrayList<>();
     if (dstInode != null) { // Destination exists

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index fa3f6a6..53d2040 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -212,6 +212,9 @@ public class FSDirectory implements Closeable {
   @VisibleForTesting
   public final EncryptionZoneManager ezManager;
 
+  @VisibleForTesting
+  public final ErasureCodingZoneManager ecZoneManager;
+
   /**
    * Caches frequently used file names used in {@link INode} to reuse 
    * byte[] objects and reduce heap usage.
@@ -303,6 +306,7 @@ public class FSDirectory implements Closeable {
     namesystem = ns;
     this.editLog = ns.getEditLog();
     ezManager = new EncryptionZoneManager(this, conf);
+    ecZoneManager = new ErasureCodingZoneManager(this);
   }
     
   FSNamesystem getFSNamesystem() {
@@ -426,8 +430,7 @@ public class FSDirectory implements Closeable {
     writeLock();
     try {
       newiip = addINode(existing, newNode);
-      // TODO: we will no longer use storage policy for "Erasure Coding Zone"
-      if (newiip != null && newNode.isStriped()) {
+      if (newiip != null && getECPolicy(newiip)) {
         newNode.addStripedBlocksFeature();
       }
     } finally {
@@ -1397,6 +1400,25 @@ public class FSDirectory implements Closeable {
     }
   }
 
+  XAttr createErasureCodingZone(String src)
+      throws IOException {
+    writeLock();
+    try {
+      return ecZoneManager.createErasureCodingZone(src);
+    } finally {
+      writeUnlock();
+    }
+  }
+
+  public boolean getECPolicy(INodesInPath iip) {
+    readLock();
+    try {
+      return ecZoneManager.getECPolicy(iip);
+    } finally {
+      readUnlock();
+    }
+  }
+
   static INode resolveLastINode(INodesInPath iip) throws FileNotFoundException {
     INode inode = iip.getLastINode();
     if (inode == null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 5fd956e..9ffb6fc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -8094,6 +8094,46 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
   }
 
+  /**
+   * Create an erasure coding zone on directory src.
+   *
+   * @param src     the path of a directory which will be the root of the
+   *                erasure coding zone. The directory must be empty.
+   * @throws AccessControlException  if the caller is not the superuser.
+   * @throws UnresolvedLinkException if the path can't be resolved.
+   * @throws SafeModeException       if the Namenode is in safe mode.
+   */
+  void createErasureCodingZone(final String srcArg,
+      final boolean logRetryCache)
+      throws IOException, UnresolvedLinkException,
+      SafeModeException, AccessControlException {
+    String src = srcArg;
+    HdfsFileStatus resultingStat = null;
+    checkSuperuserPrivilege();
+    checkOperation(OperationCategory.WRITE);
+    final byte[][] pathComponents =
+        FSDirectory.getPathComponentsForReservedPath(src);
+    FSPermissionChecker pc = getPermissionChecker();
+    writeLock();
+    try {
+      checkSuperuserPrivilege();
+      checkOperation(OperationCategory.WRITE);
+      checkNameNodeSafeMode("Cannot create erasure coding zone on " + src);
+      src = dir.resolvePath(pc, src, pathComponents);
+
+      final XAttr ecXAttr = dir.createErasureCodingZone(src);
+      List<XAttr> xAttrs = Lists.newArrayListWithCapacity(1);
+      xAttrs.add(ecXAttr);
+      getEditLog().logSetXAttrs(src, xAttrs, logRetryCache);
+      final INodesInPath iip = dir.getINodesInPath4Write(src, false);
+      resultingStat = dir.getAuditFileInfo(iip);
+    } finally {
+      writeUnlock();
+    }
+    getEditLog().logSync();
+    logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat);
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 2aa44cd..f95e54e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -419,7 +419,7 @@ public class INodeFile extends INodeWithAdditionalFields
       }
       max = maxInSnapshot > max ? maxInSnapshot : max;
     }
-    return isStriped()?
+    return isStriped() ?
         HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max;
   }
 
@@ -841,7 +841,10 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public final long storagespaceConsumedWithStriped() {
     BlockInfo[] blockInfos = getBlocks();
-    long size = 0;
+    if (blockInfos == null || blockInfos.length == 0) {
+      return 0;
+    }
+    long size;
     final int last = blockInfos.length - 1;
     if (blockInfos[last] instanceof BlockInfoStripedUnderConstruction) {
       BlockInfoStripedUnderConstruction blockInfoStripedUC
@@ -1100,8 +1103,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   @VisibleForTesting
   @Override
-  // TODO: move erasure coding policy to file XAttr
   public boolean isStriped() {
-    return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID;
+    return getStripedBlocksFeature() != null;
   }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index 7ab8b86..ce24662 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -1834,6 +1834,22 @@ class NameNodeRpcServer implements NamenodeProtocols {
   }
 
   @Override // ClientProtocol
+  public void createErasureCodingZone(String src)
+    throws IOException {
+    checkNNStartup();
+    final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache);
+    if (cacheEntry != null && cacheEntry.isSuccess()) {
+      return;
+    }
+    boolean success = false;
+    try {
+      namesystem.createErasureCodingZone(src, cacheEntry != null);
+    } finally {
+      RetryCache.setState(cacheEntry, success);
+    }
+  }
+
+  @Override // ClientProtocol
   public void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag)
       throws IOException {
     checkNNStartup();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index b44c556..183aff8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -714,6 +714,13 @@ message GetEditsFromTxidResponseProto {
   required EventsListProto eventsList = 1;
 }
 
+message CreateErasureCodingZoneRequestProto {
+  required string src = 1;
+}
+
+message CreateErasureCodingZoneResponseProto {
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -856,6 +863,8 @@ service ClientNamenodeProtocol {
       returns(ListEncryptionZonesResponseProto);
   rpc getEZForPath(GetEZForPathRequestProto)
       returns(GetEZForPathResponseProto);
+  rpc createErasureCodingZone(CreateErasureCodingZoneRequestProto)
+      returns(CreateErasureCodingZoneResponseProto);
   rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto)
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
index 606a332..8f99a85 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockStoragePolicy.java
@@ -68,7 +68,6 @@ public class TestBlockStoragePolicy {
   static final short REPLICATION = 3;
 
   static final byte COLD = HdfsConstants.COLD_STORAGE_POLICY_ID;
-  static final byte EC = HdfsConstants.EC_STORAGE_POLICY_ID;
   static final byte WARM = HdfsConstants.WARM_STORAGE_POLICY_ID;
   static final byte HOT  = HdfsConstants.HOT_STORAGE_POLICY_ID;
   static final byte ONESSD  = HdfsConstants.ONESSD_STORAGE_POLICY_ID;
@@ -116,9 +115,6 @@ public class TestBlockStoragePolicy {
     expectedPolicyStrings.put(COLD,
         "BlockStoragePolicy{COLD:" + COLD + ", storageTypes=[ARCHIVE], " +
             "creationFallbacks=[], replicationFallbacks=[]}");
-    expectedPolicyStrings.put(EC,
-        "BlockStoragePolicy{EC:" + EC + ", storageTypes=[DISK], " +
-            "creationFallbacks=[], replicationFallbacks=[ARCHIVE]}");
     expectedPolicyStrings.put(WARM,
         "BlockStoragePolicy{WARM:" + WARM + ", storageTypes=[DISK, ARCHIVE], " +
             "creationFallbacks=[DISK, ARCHIVE], " +
@@ -1161,15 +1157,13 @@ public class TestBlockStoragePolicy {
     final DistributedFileSystem fs = cluster.getFileSystem();
     try {
       BlockStoragePolicy[] policies = fs.getStoragePolicies();
-      Assert.assertEquals(7, policies.length);
+      Assert.assertEquals(6, policies.length);
       Assert.assertEquals(POLICY_SUITE.getPolicy(COLD).toString(),
           policies[0].toString());
-      Assert.assertEquals(POLICY_SUITE.getPolicy(EC).toString(),
-          policies[1].toString());
       Assert.assertEquals(POLICY_SUITE.getPolicy(WARM).toString(),
-          policies[2].toString());
+          policies[1].toString());
       Assert.assertEquals(POLICY_SUITE.getPolicy(HOT).toString(),
-          policies[3].toString());
+          policies[2].toString());
     } finally {
       IOUtils.cleanup(null, fs);
       cluster.shutdown();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
new file mode 100644
index 0000000..49f08eef
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
@@ -0,0 +1,151 @@
+/**
+ * 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.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+public class TestErasureCodingZones {
+  private final int NUM_OF_DATANODES = 3;
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private static final int BLOCK_SIZE = 1024;
+  private FSNamesystem namesystem;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(NUM_OF_DATANODES).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    namesystem = cluster.getNamesystem();
+  }
+
+  @After
+  public void shutdownCluster() throws IOException {
+    cluster.shutdown();
+  }
+
+  @Test
+  public void testCreateECZone()
+      throws IOException, InterruptedException {
+    final Path testDir = new Path("/ec");
+    fs.mkdir(testDir, FsPermission.getDirDefault());
+
+    /* Normal creation of an erasure coding zone */
+    fs.getClient().createErasureCodingZone(testDir.toString());
+
+    /* Verify files under the zone are striped */
+    final Path ECFilePath = new Path(testDir, "foo");
+    fs.create(ECFilePath);
+    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
+    assertTrue(inode.asFile().isStriped());
+
+    /* Verify that EC zone cannot be created on non-empty dir */
+    final Path notEmpty = new Path("/nonEmpty");
+    fs.mkdir(notEmpty, FsPermission.getDirDefault());
+    fs.create(new Path(notEmpty, "foo"));
+    try {
+      fs.getClient().createErasureCodingZone(notEmpty.toString());
+      fail("Erasure coding zone on non-empty dir");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding zone for a non-empty directory", e);
+    }
+
+    /* Verify that nested EC zones cannot be created */
+    final Path zone1 = new Path("/zone1");
+    final Path zone2 = new Path(zone1, "zone2");
+    fs.mkdir(zone1, FsPermission.getDirDefault());
+    fs.getClient().createErasureCodingZone(zone1.toString());
+    fs.mkdir(zone2, FsPermission.getDirDefault());
+    try {
+      fs.getClient().createErasureCodingZone(zone2.toString());
+      fail("Nested erasure coding zones");
+    } catch (IOException e) {
+      assertExceptionContains("already in an erasure coding zone", e);
+    }
+
+    /* Verify that EC zone cannot be created on a file */
+    final Path fPath = new Path("/file");
+    fs.create(fPath);
+    try {
+      fs.getClient().createErasureCodingZone(fPath.toString());
+      fail("Erasure coding zone on file");
+    } catch (IOException e) {
+      assertExceptionContains("erasure coding zone for a file", e);
+    }
+  }
+
+  @Test
+  public void testMoveValidity() throws IOException, InterruptedException {
+    final Path srcECDir = new Path("/srcEC");
+    final Path dstECDir = new Path("/dstEC");
+    fs.mkdir(srcECDir, FsPermission.getDirDefault());
+    fs.mkdir(dstECDir, FsPermission.getDirDefault());
+    fs.getClient().createErasureCodingZone(srcECDir.toString());
+    fs.getClient().createErasureCodingZone(dstECDir.toString());
+    final Path srcFile = new Path(srcECDir, "foo");
+    fs.create(srcFile);
+
+    /* Verify that a file can be moved between 2 EC zones */
+    try {
+      fs.rename(srcFile, dstECDir);
+    } catch (IOException e) {
+      fail("A file should be able to move between 2 EC zones " + e);
+    }
+
+    // Move the file back
+    fs.rename(new Path(dstECDir, "foo"), srcECDir);
+
+    /* Verify that a file cannot be moved from a non-EC dir to an EC zone */
+    final Path nonECDir = new Path("/nonEC");
+    fs.mkdir(nonECDir, FsPermission.getDirDefault());
+    try {
+      fs.rename(srcFile, nonECDir);
+      fail("A file shouldn't be able to move from a non-EC dir to an EC zone");
+    } catch (IOException e) {
+      assertExceptionContains("can't be moved because the source and " +
+          "destination have different erasure coding policies", e);
+    }
+
+    /* Verify that a file cannot be moved from an EC zone to a non-EC dir */
+    final Path nonECFile = new Path(nonECDir, "nonECFile");
+    fs.create(nonECFile);
+    try {
+      fs.rename(nonECFile, dstECDir);
+    } catch (IOException e) {
+      assertExceptionContains("can't be moved because the source and " +
+          "destination have different erasure coding policies", e);
+    }
+  }
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
deleted file mode 100644
index a84f67b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInitialEncoding.java
+++ /dev/null
@@ -1,75 +0,0 @@
-/**
- * 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.server.blockmanagement;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.*;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.apache.hadoop.hdfs.server.namenode.INode;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_ID;
-import static org.junit.Assert.assertEquals;
-
-public class TestBlockInitialEncoding {
-  private final int NUM_OF_DATANODES = 3;
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private static final int BLOCK_SIZE = 1024;
-  private HdfsAdmin dfsAdmin;
-  private FSNamesystem namesystem;
-
-  @Before
-  public void setupCluster() throws IOException {
-    conf = new HdfsConfiguration();
-    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(NUM_OF_DATANODES).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
-    namesystem = cluster.getNamesystem();
-  }
-
-  @After
-  public void shutdownCluster() throws IOException {
-    cluster.shutdown();
-  }
-
-  @Test
-  public void testBlockInitialEncoding()
-      throws IOException, InterruptedException {
-    final Path testDir = new Path("/test");
-    fs.mkdir(testDir, FsPermission.getDirDefault());
-    dfsAdmin.setStoragePolicy(testDir, EC_STORAGE_POLICY_NAME);
-    final Path ECFilePath = new Path("/test/foo.ec");
-    DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, (short) 3, 0);
-    INode inode = namesystem.getFSDirectory().getINode(ECFilePath.toString());
-    assertEquals(EC_STORAGE_POLICY_ID, inode.getStoragePolicyID());
-  }
-
-}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 215a4e4..c3c8239 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -68,7 +68,7 @@ public class TestAddStripedBlocks {
         .numDataNodes(GROUP_SIZE).build();
     cluster.waitActive();
     dfs = cluster.getFileSystem();
-    dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+    dfs.getClient().createErasureCodingZone("/");
   }
 
   @After

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index d3cb749..407d07e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -444,8 +444,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.setStoragePolicy(new Path(testDir),
-          HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir);
 
       // Create a file with striped block
       Path p = new Path(testFilePath);
@@ -517,8 +516,7 @@ public class TestFSEditLogLoader {
 
       //set the storage policy of the directory
       fs.mkdir(new Path(testDir), new FsPermission("755"));
-      fs.setStoragePolicy(new Path(testDir),
-          HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone(testDir);
 
       //create a file with striped blocks
       Path p = new Path(testFilePath);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 440f5cd..83f01c6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -33,18 +33,14 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
 import org.apache.hadoop.fs.permission.PermissionStatus;
 import org.apache.hadoop.fs.permission.FsPermission;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileSystem;
@@ -59,7 +55,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
 import org.apache.hadoop.hdfs.util.MD5FileUtils;
@@ -137,9 +132,10 @@ public class TestFSImage {
     }
   }
 
-  private void testSaveAndLoadINodeFile(FSNamesystem fsn, Configuration conf,
-      boolean isUC) throws IOException{
+  private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf,
+                                               boolean isUC) throws IOException{
     // contruct a INode with StripedBlock for saving and loading
+    fsn.createErasureCodingZone("/", false);
     long id = 123456789;
     byte[] name = "testSaveAndLoadInodeFile_testfile".getBytes();
     PermissionStatus permissionStatus = new PermissionStatus("testuser_a",
@@ -149,9 +145,8 @@ public class TestFSImage {
     BlockInfoContiguous[] blks = new BlockInfoContiguous[0];
     short replication = 3;
     long preferredBlockSize = 128*1024*1024;
-    byte storagePolicyID = HdfsConstants.EC_STORAGE_POLICY_ID;
     INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime,
-        blks, replication, preferredBlockSize, storagePolicyID);
+        blks, replication, preferredBlockSize);
     ByteArrayOutputStream bs = new ByteArrayOutputStream();
     file.addStripedBlocksFeature();
 
@@ -237,13 +232,13 @@ public class TestFSImage {
    * FSImageSerialization and loaded by FSImageFormat#Loader.
    */
   @Test
-  public void testSaveAndLoadInodeFile() throws IOException{
+  public void testSaveAndLoadStripedINodeFile() throws IOException{
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
-      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, false);
+      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, false);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -256,14 +251,14 @@ public class TestFSImage {
    * saved and loaded by FSImageSerialization
    */
   @Test
-  public void testSaveAndLoadInodeFileUC() throws IOException{
+  public void testSaveAndLoadStripedINodeFileUC() throws IOException{
     // construct a INode with StripedBlock for saving and loading
     Configuration conf = new Configuration();
     MiniDFSCluster cluster = null;
     try {
       cluster = new MiniDFSCluster.Builder(conf).build();
       cluster.waitActive();
-      testSaveAndLoadINodeFile(cluster.getNamesystem(), conf, true);
+      testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true);
     } finally {
       if (cluster != null) {
         cluster.shutdown();
@@ -402,7 +397,7 @@ public class TestFSImage {
           .build();
       cluster.waitActive();
       DistributedFileSystem fs = cluster.getFileSystem();
-      fs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+      fs.getClient().getNamenode().createErasureCodingZone("/");
       Path file = new Path("/striped");
       FSDataOutputStream out = fs.create(file);
       byte[] bytes = DFSTestUtil.generateSequentialBytes(0, BLOCK_SIZE);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/ed06d9e9/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index b9fd4fe..d965ae7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -27,7 +27,6 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -37,23 +36,19 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
-import org.junit.Assert;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
-import java.util.Iterator;
 import java.util.List;
 import java.util.UUID;
 
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -93,7 +88,7 @@ public class TestRecoverStripedBlocks {
       int numBlocks) throws Exception {
     DistributedFileSystem dfs = cluster.getFileSystem();
     dfs.mkdirs(dir);
-    dfs.setStoragePolicy(dir, EC_STORAGE_POLICY_NAME);
+    dfs.getClient().getNamenode().createErasureCodingZone(dir.toString());
 
     FSDataOutputStream out = null;
     try {


[32/50] [abbrv] hadoop git commit: HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HADOOP-11740. Combine erasure encoder and decoder interfaces. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/6e0a3190
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/6e0a3190
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/6e0a3190

Branch: refs/heads/HDFS-7285
Commit: 6e0a31909e7baff9692221b7a320e50357cb58b0
Parents: ed06d9e
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Apr 3 15:22:50 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:48:57 2015 -0700

----------------------------------------------------------------------
 .../coder/AbstractErasureDecoder.java           |  7 ++--
 .../coder/AbstractErasureEncoder.java           |  7 ++--
 .../io/erasurecode/coder/ErasureCoder.java      | 12 ++++++
 .../io/erasurecode/coder/ErasureDecoder.java    | 41 --------------------
 .../io/erasurecode/coder/ErasureEncoder.java    | 39 -------------------
 .../erasurecode/coder/TestErasureCoderBase.java | 20 +++++-----
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        | 14 ++++++-
 7 files changed, 41 insertions(+), 99 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a3190/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
index 54a6d1e..cd31294 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -23,13 +23,12 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 /**
  * An abstract erasure decoder that's to be inherited by new decoders.
  *
- * It implements the {@link ErasureDecoder} interface.
+ * It implements the {@link ErasureCoder} interface.
  */
-public abstract class AbstractErasureDecoder extends AbstractErasureCoder
-    implements ErasureDecoder {
+public abstract class AbstractErasureDecoder extends AbstractErasureCoder {
 
   @Override
-  public ErasureCodingStep decode(ECBlockGroup blockGroup) {
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
     // We may have more than this when considering complicate cases. HADOOP-11550
     return prepareDecodingStep(blockGroup);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a3190/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
index 09b31e5..a836b75 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
@@ -23,13 +23,12 @@ import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 /**
  * An abstract erasure encoder that's to be inherited by new encoders.
  *
- * It implements the {@link ErasureEncoder} interface.
+ * It implements the {@link ErasureCoder} interface.
  */
-public abstract class AbstractErasureEncoder extends AbstractErasureCoder
-    implements ErasureEncoder {
+public abstract class AbstractErasureEncoder extends AbstractErasureCoder {
 
   @Override
-  public ErasureCodingStep encode(ECBlockGroup blockGroup) {
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) {
     // We may have more than this when considering complicate cases. HADOOP-11550
     return prepareEncodingStep(blockGroup);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a3190/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
index c5922f3..fb90156 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
 
 /**
  * An erasure coder to perform encoding or decoding given a group. Generally it
@@ -64,6 +65,17 @@ public interface ErasureCoder extends Configurable {
   public int getChunkSize();
 
   /**
+   * Calculate the encoding or decoding steps given a block blockGroup.
+   *
+   * Note, currently only one coding step is supported. Will support complex
+   * cases of multiple coding steps.
+   *
+   * @param blockGroup the erasure coding block group containing all necessary
+   *                   information for codec calculation
+   */
+  public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup);
+
+  /**
    * Tell if native or off-heap buffer is preferred or not. It's for callers to
    * decide how to allocate coding chunk buffers, either on heap or off heap.
    * It will return false by default.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a3190/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
deleted file mode 100644
index dfd9e54..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
+++ /dev/null
@@ -1,41 +0,0 @@
-/**
- * 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.io.erasurecode.coder;
-
-
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-
-/**
- * Erasure decoder interface to perform decoding given a block group.
- *
- * It extends {@link ErasureCoder}.
- */
-public interface ErasureDecoder extends ErasureCoder {
-
-  /**
-   * Perform the decoding given a blockGroup. By default it will try the best to
-   * attempt to recover all the missing blocks according to the codec logic.
-   *
-   * Note, currently only one coding step is supported. Will support complex
-   * cases of multiple coding steps.
-   *
-   * @param blockGroup
-   */
-  public ErasureCodingStep decode(ECBlockGroup blockGroup);
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a3190/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
deleted file mode 100644
index e837d22..0000000
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
+++ /dev/null
@@ -1,39 +0,0 @@
-/**
- * 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.io.erasurecode.coder;
-
-import org.apache.hadoop.io.erasurecode.ECBlockGroup;
-
-/**
- * Erasure encoder interface to perform encoding given a block group.
- *
- * It extends {@link ErasureCoder}.
- */
-public interface ErasureEncoder extends ErasureCoder {
-
-  /**
-   * Calculate the encoding steps given a block blockGroup.
-   *
-   * Note, currently only one coding step is supported. Will support complex
-   * cases of multiple coding steps.
-   *
-   * @param blockGroup
-   */
-  public ErasureCodingStep encode(ECBlockGroup blockGroup);
-
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a3190/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
index b963a59..05a62a7 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -26,8 +26,8 @@ import org.apache.hadoop.io.erasurecode.TestCoderBase;
  * Erasure coder test base with utilities.
  */
 public abstract class TestErasureCoderBase extends TestCoderBase {
-  protected Class<? extends ErasureEncoder> encoderClass;
-  protected Class<? extends ErasureDecoder> decoderClass;
+  protected Class<? extends ErasureCoder> encoderClass;
+  protected Class<? extends ErasureCoder> decoderClass;
 
   protected int numChunksInBlock = 16;
 
@@ -55,7 +55,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected void testCoding(boolean usingDirectBuffer) {
     this.usingDirectBuffer = usingDirectBuffer;
 
-    ErasureEncoder encoder = createEncoder();
+    ErasureCoder encoder = createEncoder();
 
     // Generate data and encode
     ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
@@ -68,7 +68,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
 
     ErasureCodingStep codingStep;
     try {
-      codingStep = encoder.encode(blockGroup);
+      codingStep = encoder.calculateCoding(blockGroup);
       performCodingStep(codingStep);
     } finally {
       encoder.release();
@@ -78,9 +78,9 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
 
     //Decode
     blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
-    ErasureDecoder decoder = createDecoder();
+    ErasureCoder decoder = createDecoder();
     try {
-      codingStep = decoder.decode(blockGroup);
+      codingStep = decoder.calculateCoding(blockGroup);
       performCodingStep(codingStep);
     } finally {
       decoder.release();
@@ -138,8 +138,8 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    * Create erasure encoder for test.
    * @return
    */
-  private ErasureEncoder createEncoder() {
-    ErasureEncoder encoder;
+  private ErasureCoder createEncoder() {
+    ErasureCoder encoder;
     try {
       encoder = encoderClass.newInstance();
     } catch (Exception e) {
@@ -155,8 +155,8 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
    * Create the erasure decoder for the test.
    * @return
    */
-  private ErasureDecoder createDecoder() {
-    ErasureDecoder decoder;
+  private ErasureCoder createDecoder() {
+    ErasureCoder decoder;
     try {
       decoder = decoderClass.newInstance();
     } catch (Exception e) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/6e0a3190/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index a686315..4e60a7c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -43,4 +43,16 @@
     blocks in NameNode (Jing Zhao)
 
     HDFS-8005. Erasure Coding: simplify striped block recovery work computation
-    and add tests (Jing Zhao)
\ No newline at end of file
+    and add tests (Jing Zhao)
+
+    HDFS-7617. Add unit tests for editlog transactions for EC 
+    (Hui Zheng via Zhe Zhang)
+
+    HADOOP-11782. Correct two thrown messages in ECSchema class
+    (Xinwei Qin via Kai Zheng)
+
+    HDFS-7839. Erasure coding: implement facilities in NameNode to create and
+    manage EC zones (Zhe Zhang)
+
+    HADOOP-11740. Combine erasure encoder and decoder interfaces (Zhe Zhang)
+


[49/50] [abbrv] hadoop git commit: HDFS-7889 Subclass DFSOutputStream to support writing striping layout files. Contributed by Li Bo

Posted by zh...@apache.org.
HDFS-7889 Subclass DFSOutputStream to support writing striping layout files. Contributed by Li Bo


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e006f68c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e006f68c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e006f68c

Branch: refs/heads/HDFS-7285
Commit: e006f68c8bc03cac6fc230de555d9e01aa9e7384
Parents: 81d1e70
Author: Kai Zheng <ka...@intel.com>
Authored: Sat Apr 11 01:03:37 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   4 +-
 .../org/apache/hadoop/hdfs/DFSOutputStream.java |  13 +-
 .../java/org/apache/hadoop/hdfs/DFSPacket.java  |  26 +-
 .../hadoop/hdfs/DFSStripedOutputStream.java     | 439 +++++++++++++++++++
 .../org/apache/hadoop/hdfs/DataStreamer.java    |  12 +-
 .../apache/hadoop/hdfs/StripedDataStreamer.java | 241 ++++++++++
 .../hadoop/hdfs/TestDFSStripedOutputStream.java | 311 +++++++++++++
 7 files changed, 1031 insertions(+), 15 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e006f68c/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 1e695c4..753795a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -56,4 +56,6 @@
     
     HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
 
-    HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
\ No newline at end of file
+    HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
+
+    HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e006f68c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
index 8cde274..8270331 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java
@@ -268,8 +268,14 @@ public class DFSOutputStream extends FSOutputSummer
         }
       }
       Preconditions.checkNotNull(stat, "HdfsFileStatus should not be null!");
-      final DFSOutputStream out = new DFSOutputStream(dfsClient, src, stat,
-          flag, progress, checksum, favoredNodes);
+      final DFSOutputStream out;
+      if(stat.getReplication() == 0) {
+        out = new DFSStripedOutputStream(dfsClient, src, stat,
+            flag, progress, checksum, favoredNodes);
+      } else {
+        out = new DFSOutputStream(dfsClient, src, stat,
+            flag, progress, checksum, favoredNodes);
+      }
       out.start();
       return out;
     } finally {
@@ -347,6 +353,9 @@ public class DFSOutputStream extends FSOutputSummer
       String[] favoredNodes) throws IOException {
     TraceScope scope =
         dfsClient.getPathTraceScope("newStreamForAppend", src);
+	if(stat.getReplication() == 0) {
+      throw new IOException("Not support appending to a striping layout file yet.");
+    }
     try {
       final DFSOutputStream out = new DFSOutputStream(dfsClient, src, flags,
           progress, lastBlock, stat, checksum);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e006f68c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
index 22055c3..9cd1ec1 100755
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java
@@ -20,6 +20,7 @@ package org.apache.hadoop.hdfs;
 import java.io.DataOutputStream;
 import java.io.IOException;
 import java.nio.BufferOverflowException;
+import java.nio.ByteBuffer;
 import java.nio.channels.ClosedChannelException;
 import java.util.Arrays;
 
@@ -113,6 +114,19 @@ class DFSPacket {
     dataPos += len;
   }
 
+  synchronized void writeData(ByteBuffer inBuffer, int len)
+      throws ClosedChannelException {
+    checkBuffer();
+    len =  len > inBuffer.remaining() ? inBuffer.remaining() : len;
+    if (dataPos + len > buf.length) {
+      throw new BufferOverflowException();
+    }
+    for (int i = 0; i < len; i++) {
+      buf[dataPos + i] = inBuffer.get();
+    }
+    dataPos += len;
+  }
+
   /**
    * Write checksums to this packet
    *
@@ -222,7 +236,7 @@ class DFSPacket {
    *
    * @return true if the packet is the last packet
    */
-  boolean isLastPacketInBlock(){
+  boolean isLastPacketInBlock() {
     return lastPacketInBlock;
   }
 
@@ -231,7 +245,7 @@ class DFSPacket {
    *
    * @return the sequence number of this packet
    */
-  long getSeqno(){
+  long getSeqno() {
     return seqno;
   }
 
@@ -240,14 +254,14 @@ class DFSPacket {
    *
    * @return the number of chunks in this packet
    */
-  synchronized int getNumChunks(){
+  synchronized int getNumChunks() {
     return numChunks;
   }
 
   /**
    * increase the number of chunks by one
    */
-  synchronized void incNumChunks(){
+  synchronized void incNumChunks() {
     numChunks++;
   }
 
@@ -256,7 +270,7 @@ class DFSPacket {
    *
    * @return the maximum number of packets
    */
-  int getMaxChunks(){
+  int getMaxChunks() {
     return maxChunks;
   }
 
@@ -265,7 +279,7 @@ class DFSPacket {
    *
    * @param syncBlock if to sync block
    */
-  synchronized void setSyncBlock(boolean syncBlock){
+  synchronized void setSyncBlock(boolean syncBlock) {
     this.syncBlock = syncBlock;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e006f68c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
new file mode 100644
index 0000000..aded4fe
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -0,0 +1,439 @@
+/**
+ * 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 java.io.IOException;
+import java.io.InterruptedIOException;
+import java.nio.ByteBuffer;
+import java.nio.channels.ClosedChannelException;
+import java.util.ArrayList;
+import java.util.EnumSet;
+import java.util.List;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.LinkedBlockingQueue;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.fs.CreateFlag;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.io.erasurecode.rawcoder.RSRawEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+import org.apache.htrace.Sampler;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+
+/****************************************************************
+ * The DFSStripedOutputStream class supports writing files in striped
+ * layout. Each stripe contains a sequence of cells and multiple
+ * {@link StripedDataStreamer}s in DFSStripedOutputStream are responsible
+ * for writing the cells to different datanodes.
+ *
+ ****************************************************************/
+
+@InterfaceAudience.Private
+public class DFSStripedOutputStream extends DFSOutputStream {
+
+  private final List<StripedDataStreamer> streamers;
+  /**
+   * Size of each striping cell, must be a multiple of bytesPerChecksum
+   */
+  private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private ByteBuffer[] cellBuffers;
+  private final short blockGroupBlocks = HdfsConstants.NUM_DATA_BLOCKS
+      + HdfsConstants.NUM_PARITY_BLOCKS;
+  private final short blockGroupDataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+  private int curIdx = 0;
+  /* bytes written in current block group */
+  private long currentBlockGroupBytes = 0;
+
+  //TODO: Use ErasureCoder interface (HDFS-7781)
+  private RawErasureEncoder encoder;
+
+  private StripedDataStreamer getLeadingStreamer() {
+    return streamers.get(0);
+  }
+
+  private long getBlockGroupSize() {
+    return blockSize * HdfsConstants.NUM_DATA_BLOCKS;
+  }
+
+  /** Construct a new output stream for creating a file. */
+  DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat,
+                         EnumSet<CreateFlag> flag, Progressable progress,
+                         DataChecksum checksum, String[] favoredNodes)
+                         throws IOException {
+    super(dfsClient, src, stat, flag, progress, checksum, favoredNodes);
+    DFSClient.LOG.info("Creating striped output stream");
+    if (blockGroupBlocks <= 1) {
+      throw new IOException("The block group must contain more than one block.");
+    }
+
+    cellBuffers = new ByteBuffer[blockGroupBlocks];
+    List<BlockingQueue<LocatedBlock>> stripeBlocks = new ArrayList<>();
+
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      stripeBlocks.add(new LinkedBlockingQueue<LocatedBlock>(blockGroupBlocks));
+      try {
+        cellBuffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize));
+      } catch (InterruptedException ie) {
+        final InterruptedIOException iioe = new InterruptedIOException(
+            "create cell buffers");
+        iioe.initCause(ie);
+        throw iioe;
+      }
+    }
+    encoder = new RSRawEncoder();
+    encoder.initialize(blockGroupDataBlocks,
+        blockGroupBlocks - blockGroupDataBlocks, cellSize);
+
+    streamers = new ArrayList<>(blockGroupBlocks);
+    for (short i = 0; i < blockGroupBlocks; i++) {
+      StripedDataStreamer streamer = new StripedDataStreamer(stat, null,
+          dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager,
+          i, stripeBlocks);
+      if (favoredNodes != null && favoredNodes.length != 0) {
+        streamer.setFavoredNodes(favoredNodes);
+      }
+      streamers.add(streamer);
+    }
+
+    refreshStreamer();
+  }
+
+  private void refreshStreamer() {
+    streamer = streamers.get(curIdx);
+  }
+
+  private void moveToNextStreamer() {
+    curIdx = (curIdx + 1) % blockGroupBlocks;
+    refreshStreamer();
+  }
+
+  /**
+   * encode the buffers.
+   * After encoding, flip each buffer.
+   *
+   * @param buffers data buffers + parity buffers
+   */
+  private void encode(ByteBuffer[] buffers) {
+    ByteBuffer[] dataBuffers = new ByteBuffer[blockGroupDataBlocks];
+    ByteBuffer[] parityBuffers = new ByteBuffer[blockGroupBlocks - blockGroupDataBlocks];
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      if (i < blockGroupDataBlocks) {
+        dataBuffers[i] = buffers[i];
+      } else {
+        parityBuffers[i - blockGroupDataBlocks] = buffers[i];
+      }
+    }
+    encoder.encode(dataBuffers, parityBuffers);
+  }
+
+  /**
+   * Generate packets from a given buffer
+   *
+   * @param byteBuffer the given buffer to generate packets
+   * @return packets generated
+   * @throws IOException
+   */
+  private List<DFSPacket> generatePackets(ByteBuffer byteBuffer)
+      throws IOException{
+    List<DFSPacket> packets = new ArrayList<>();
+    while (byteBuffer.remaining() > 0) {
+      DFSPacket p = createPacket(packetSize, chunksPerPacket,
+          streamer.getBytesCurBlock(),
+          streamer.getAndIncCurrentSeqno(), false);
+      int maxBytesToPacket = p.getMaxChunks() * bytesPerChecksum;
+      int toWrite = byteBuffer.remaining() > maxBytesToPacket ?
+          maxBytesToPacket: byteBuffer.remaining();
+      p.writeData(byteBuffer, toWrite);
+      streamer.incBytesCurBlock(toWrite);
+      packets.add(p);
+    }
+    return packets;
+  }
+
+  @Override
+  protected synchronized void writeChunk(byte[] b, int offset, int len,
+      byte[] checksum, int ckoff, int cklen) throws IOException {
+    super.writeChunk(b, offset, len, checksum, ckoff, cklen);
+
+    if (getSizeOfCellnBuffer(curIdx) <= cellSize) {
+      addToCellBuffer(b, offset, len);
+    } else {
+      String msg = "Writing a chunk should not overflow the cell buffer.";
+      DFSClient.LOG.info(msg);
+      throw new IOException(msg);
+    }
+
+
+    // If current packet has not been enqueued for transmission,
+    // but the cell buffer is full, we need to enqueue the packet
+    if (currentPacket != null && getSizeOfCellnBuffer(curIdx) == cellSize) {
+      if (DFSClient.LOG.isDebugEnabled()) {
+        DFSClient.LOG.debug("DFSClient writeChunk cell buffer full seqno=" +
+            currentPacket.getSeqno() +
+            ", curIdx=" + curIdx +
+            ", src=" + src +
+            ", bytesCurBlock=" + streamer.getBytesCurBlock() +
+            ", blockSize=" + blockSize +
+            ", appendChunk=" + streamer.getAppendChunk());
+      }
+      streamer.waitAndQueuePacket(currentPacket);
+      currentPacket = null;
+      adjustChunkBoundary();
+      endBlock();
+    }
+
+    // Two extra steps are needed when a striping cell is full:
+    // 1. Forward the current index pointer
+    // 2. Generate parity packets if a full stripe of data cells are present
+    if (getSizeOfCellnBuffer(curIdx) == cellSize) {
+      //move curIdx to next cell
+      moveToNextStreamer();
+      //When all data cells in a stripe are ready, we need to encode
+      //them and generate some parity cells. These cells will be
+      //converted to packets and put to their DataStreamer's queue.
+      if (curIdx == blockGroupDataBlocks) {
+        //encode the data cells
+        for (int k = 0; k < blockGroupDataBlocks; k++) {
+          cellBuffers[k].flip();
+        }
+        encode(cellBuffers);
+        for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) {
+          ByteBuffer parityBuffer = cellBuffers[i];
+          List<DFSPacket> packets = generatePackets(parityBuffer);
+          for (DFSPacket p : packets) {
+            currentPacket = p;
+            streamer.waitAndQueuePacket(currentPacket);
+            currentPacket = null;
+          }
+          endBlock();
+          moveToNextStreamer();
+        }
+        //read next stripe to cellBuffers
+        clearCellBuffers();
+      }
+    }
+  }
+
+  private void addToCellBuffer(byte[] b, int off, int len) {
+    cellBuffers[curIdx].put(b, off, len);
+  }
+
+  private int getSizeOfCellnBuffer(int cellIndex) {
+    return cellBuffers[cellIndex].position();
+  }
+
+  private void clearCellBuffers() {
+    for (int i = 0; i< blockGroupBlocks; i++) {
+      cellBuffers[i].clear();
+    }
+  }
+
+  private int stripeDataSize() {
+    return blockGroupDataBlocks * cellSize;
+  }
+
+  private void notSupported(String headMsg)
+      throws IOException{
+      throw new IOException(
+          headMsg + " is now not supported for striping layout.");
+  }
+
+  @Override
+  public void hflush() throws IOException {
+    notSupported("hflush");
+  }
+
+  @Override
+  public void hsync() throws IOException {
+    notSupported("hsync");
+  }
+
+
+  @Override
+  protected synchronized void start() {
+    for (StripedDataStreamer streamer : streamers) {
+      streamer.start();
+    }
+  }
+
+  @Override
+  synchronized void abort() throws IOException {
+    if (isClosed()) {
+      return;
+    }
+    for (StripedDataStreamer streamer : streamers) {
+      streamer.setLastException(new IOException("Lease timeout of "
+          + (dfsClient.getHdfsTimeout()/1000) + " seconds expired."));
+    }
+    closeThreads(true);
+    dfsClient.endFileLease(fileId);
+  }
+
+  //TODO: Handle slow writers (HDFS-7786)
+  //Cuurently only check if the leading streamer is terminated
+  boolean isClosed() {
+    return closed || getLeadingStreamer().streamerClosed();
+  }
+
+  // shutdown datastreamer and responseprocessor threads.
+  // interrupt datastreamer if force is true
+  @Override
+  protected void closeThreads(boolean force) throws IOException {
+    StripedDataStreamer leadingStreamer = null;
+    for (StripedDataStreamer streamer : streamers) {
+      try {
+        streamer.close(force);
+        streamer.join();
+        streamer.closeSocket();
+        if (streamer.isLeadingStreamer()) {
+          leadingStreamer = streamer;
+        } else {
+          streamer.countTailingBlockGroupBytes();
+        }
+
+      } catch (InterruptedException e) {
+        throw new IOException("Failed to shutdown streamer");
+      } finally {
+        streamer.setSocketToNull();
+        setClosed();
+      }
+    }
+    leadingStreamer.countTailingBlockGroupBytes();
+  }
+
+  @Override
+  public synchronized void write(int b) throws IOException {
+    super.write(b);
+    currentBlockGroupBytes = (currentBlockGroupBytes + 1) % getBlockGroupSize();
+  }
+
+  @Override
+  public synchronized void write(byte b[], int off, int len)
+      throws IOException {
+    super.write(b, off, len);
+    currentBlockGroupBytes = (currentBlockGroupBytes + len) % getBlockGroupSize();
+  }
+
+  private void writeParityCellsForLastStripe() throws IOException{
+    if(currentBlockGroupBytes == 0 ||
+        currentBlockGroupBytes % stripeDataSize() == 0)
+      return;
+    int lastStripeLen =(int)(currentBlockGroupBytes % stripeDataSize());
+    // Size of parity cells should equal the size of the first cell, if it
+    // is not full.
+    int parityCellSize = cellSize;
+    int index = lastStripeLen / cellSize;
+    if (lastStripeLen < cellSize) {
+      parityCellSize = lastStripeLen;
+      index++;
+    }
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      if (i >= index) {
+        int position = cellBuffers[i].position();
+        for (int j = 0; j < parityCellSize - position; j++) {
+          cellBuffers[i].put((byte)0);
+        }
+      }
+      cellBuffers[i].flip();
+    }
+    encode(cellBuffers);
+
+    //write parity cells
+    curIdx = blockGroupDataBlocks;
+    refreshStreamer();
+    for (int i = blockGroupDataBlocks; i < blockGroupBlocks; i++) {
+      ByteBuffer parityBuffer = cellBuffers[i];
+      List<DFSPacket> packets = generatePackets(parityBuffer);
+      for (DFSPacket p : packets) {
+        currentPacket = p;
+        streamer.waitAndQueuePacket(currentPacket);
+        currentPacket = null;
+      }
+      endBlock();
+      moveToNextStreamer();
+    }
+
+    clearCellBuffers();
+  }
+
+  @Override
+  void setClosed() {
+    super.setClosed();
+    for (int i = 0; i < blockGroupBlocks; i++) {
+      byteArrayManager.release(cellBuffers[i].array());
+      streamers.get(i).release();
+    }
+  }
+
+  @Override
+  protected synchronized void closeImpl() throws IOException {
+    if (isClosed()) {
+      IOException e = getLeadingStreamer().getLastException().getAndSet(null);
+      if (e == null)
+        return;
+      else
+        throw e;
+    }
+
+    try {
+      // flush from all upper layers
+      flushBuffer();
+      if (currentPacket != null) {
+        streamer.waitAndQueuePacket(currentPacket);
+        currentPacket = null;
+      }
+      //if the last stripe is incomplete, generate and write parity cells
+      writeParityCellsForLastStripe();
+
+      for (int i = 0; i < blockGroupBlocks; i++) {
+        curIdx = i;
+        refreshStreamer();
+        if (streamer.getBytesCurBlock()!= 0 ||
+            currentBlockGroupBytes < getBlockGroupSize()) {
+          // send an empty packet to mark the end of the block
+          currentPacket = createPacket(0, 0, streamer.getBytesCurBlock(),
+              streamer.getAndIncCurrentSeqno(), true);
+          currentPacket.setSyncBlock(shouldSyncBlock);
+        }
+        // flush all data to Datanode
+        flushInternal();
+      }
+
+      // get last block before destroying the streamer
+      ExtendedBlock lastBlock = streamers.get(0).getBlock();
+      closeThreads(false);
+      TraceScope scope = Trace.startSpan("completeFile", Sampler.NEVER);
+      try {
+        completeFile(lastBlock);
+      } finally {
+        scope.close();
+      }
+      dfsClient.endFileLease(fileId);
+    } catch (ClosedChannelException e) {
+    } finally {
+      setClosed();
+    }
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e006f68c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
index 405f775..3e5acf3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java
@@ -168,7 +168,7 @@ class DataStreamer extends Daemon {
   }
 
   private volatile boolean streamerClosed = false;
-  private ExtendedBlock block; // its length is number of bytes acked
+  protected ExtendedBlock block; // its length is number of bytes acked
   private Token<BlockTokenIdentifier> accessToken;
   private DataOutputStream blockStream;
   private DataInputStream blockReplyStream;
@@ -176,7 +176,7 @@ class DataStreamer extends Daemon {
   private volatile DatanodeInfo[] nodes = null; // list of targets for current block
   private volatile StorageType[] storageTypes = null;
   private volatile String[] storageIDs = null;
-  private String[] favoredNodes;
+  protected String[] favoredNodes;
   volatile boolean hasError = false;
   volatile int errorIndex = -1;
   // Restarting node index
@@ -203,12 +203,12 @@ class DataStreamer extends Daemon {
   private final AtomicReference<IOException> lastException = new AtomicReference<>();
   private Socket s;
 
-  private final DFSClient dfsClient;
-  private final String src;
+  protected final DFSClient dfsClient;
+  protected final String src;
   /** Only for DataTransferProtocol.writeBlock(..) */
   private final DataChecksum checksum4WriteBlock;
   private final Progressable progress;
-  private final HdfsFileStatus stat;
+  protected final HdfsFileStatus stat;
   // appending to existing partial block
   private volatile boolean appendChunk = false;
   // both dataQueue and ackQueue are protected by dataQueue lock
@@ -331,7 +331,7 @@ class DataStreamer extends Daemon {
     stage = BlockConstructionStage.DATA_STREAMING;
   }
 
-  private void endBlock() {
+  protected void endBlock() {
     if(DFSClient.LOG.isDebugEnabled()) {
       DFSClient.LOG.debug("Closing old block " + block);
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e006f68c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
new file mode 100644
index 0000000..710d92d
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java
@@ -0,0 +1,241 @@
+/**
+ * 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 java.util.List;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.hdfs.util.ByteArrayManager;
+import org.apache.hadoop.security.token.Token;
+import org.apache.hadoop.util.DataChecksum;
+import org.apache.hadoop.util.Progressable;
+
+import java.io.IOException;
+import java.util.concurrent.BlockingQueue;
+import java.util.concurrent.TimeUnit;
+import java.util.concurrent.atomic.AtomicReference;
+
+/****************************************************************************
+ * The StripedDataStreamer class is used by {@link DFSStripedOutputStream}.
+ * There are two kinds of StripedDataStreamer, leading streamer and ordinary
+ * stream. Leading streamer requests a block group from NameNode, unwraps
+ * it to located blocks and transfers each located block to its corresponding
+ * ordinary streamer via a blocking queue.
+ *
+ ****************************************************************************/
+public class StripedDataStreamer extends DataStreamer {
+  private final short index;
+  private final  List<BlockingQueue<LocatedBlock>> stripedBlocks;
+  private static short blockGroupSize = HdfsConstants.NUM_DATA_BLOCKS
+      + HdfsConstants.NUM_PARITY_BLOCKS;
+  private boolean hasCommittedBlock = false;
+
+  StripedDataStreamer(HdfsFileStatus stat, ExtendedBlock block,
+                      DFSClient dfsClient, String src,
+                      Progressable progress, DataChecksum checksum,
+                      AtomicReference<CachingStrategy> cachingStrategy,
+                      ByteArrayManager byteArrayManage, short index,
+                      List<BlockingQueue<LocatedBlock>> stripedBlocks) {
+    super(stat,block, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage);
+    this.index = index;
+    this.stripedBlocks = stripedBlocks;
+  }
+
+  /**
+   * Construct a data streamer for appending to the last partial block
+   * @param lastBlock last block of the file to be appended
+   * @param stat status of the file to be appended
+   * @throws IOException if error occurs
+   */
+  StripedDataStreamer(LocatedBlock lastBlock, HdfsFileStatus stat,
+                      DFSClient dfsClient, String src,
+                      Progressable progress, DataChecksum checksum,
+                      AtomicReference<CachingStrategy> cachingStrategy,
+                      ByteArrayManager byteArrayManage, short index,
+                      List<BlockingQueue<LocatedBlock>> stripedBlocks)
+      throws IOException {
+    super(lastBlock, stat, dfsClient, src, progress, checksum, cachingStrategy,
+        byteArrayManage);
+    this.index = index;
+    this.stripedBlocks = stripedBlocks;
+  }
+
+  public boolean isLeadingStreamer () {
+    return index == 0;
+  }
+
+  private boolean isParityStreamer() {
+    return index >= HdfsConstants.NUM_DATA_BLOCKS;
+  }
+
+  @Override
+  protected void endBlock() {
+    if (!isLeadingStreamer() && !isParityStreamer()) {
+      //before retrieving a new block, transfer the finished block to
+      //leading streamer
+      LocatedBlock finishedBlock = new LocatedBlock(
+          new ExtendedBlock(block.getBlockPoolId(), block.getBlockId(),
+                       block.getNumBytes(),block.getGenerationStamp()), null);
+      try{
+        boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30,
+            TimeUnit.SECONDS);
+      }catch (InterruptedException ie) {
+      //TODO: Handle InterruptedException (HDFS-7786)
+      }
+    }
+    super.endBlock();
+  }
+
+  /**
+   * This function is called after the streamer is closed.
+   */
+  void countTailingBlockGroupBytes () throws IOException {
+    if (isLeadingStreamer()) {
+      //when committing a block group, leading streamer has to adjust
+      // {@link block} including the size of block group
+      for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) {
+        try {
+          LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
+              TimeUnit.SECONDS);
+          if (finishedLocatedBlock == null) {
+            throw new IOException("Fail to get finished LocatedBlock " +
+                "from streamer, i=" + i);
+          }
+          ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock();
+          long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes();
+          if (block != null) {
+            block.setNumBytes(block.getNumBytes() + bytes);
+          }
+        } catch (InterruptedException ie) {
+          DFSClient.LOG.info("InterruptedException received when " +
+              "putting a block to stripeBlocks, ie = " + ie);
+        }
+      }
+    } else if (!isParityStreamer()) {
+      if (block == null || block.getNumBytes() == 0) {
+        LocatedBlock finishedBlock = new LocatedBlock(null, null);
+        try {
+          boolean offSuccess = stripedBlocks.get(0).offer(finishedBlock, 30,
+              TimeUnit.SECONDS);
+        } catch (InterruptedException ie) {
+          //TODO: Handle InterruptedException (HDFS-7786)
+          ie.printStackTrace();
+        }
+      }
+    }
+
+  }
+
+  @Override
+  protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes)
+      throws IOException {
+    LocatedBlock lb = null;
+    if (isLeadingStreamer()) {
+      if(hasCommittedBlock) {
+        //when committing a block group, leading streamer has to adjust
+        // {@link block} including the size of block group
+        for (int i = 1; i < HdfsConstants.NUM_DATA_BLOCKS; i++) {
+          try {
+            LocatedBlock finishedLocatedBlock = stripedBlocks.get(0).poll(30,
+                TimeUnit.SECONDS);
+            if (finishedLocatedBlock == null) {
+              throw new IOException("Fail to get finished LocatedBlock " +
+                  "from streamer, i=" + i);
+            }
+            ExtendedBlock finishedBlock = finishedLocatedBlock.getBlock();
+            long bytes = finishedBlock == null ? 0 : finishedBlock.getNumBytes();
+            if(block != null) {
+              block.setNumBytes(block.getNumBytes() + bytes);
+            }
+          } catch (InterruptedException ie) {
+            DFSClient.LOG.info("InterruptedException received when putting" +
+                " a block to stripeBlocks, ie = " + ie);
+          }
+        }
+      }
+
+      lb = super.locateFollowingBlock(excludedNodes);
+      hasCommittedBlock = true;
+      LocatedBlock[] blocks = unwrapBlockGroup(lb);
+      assert blocks.length == blockGroupSize :
+          "Fail to get block group from namenode: blockGroupSize: " +
+              blockGroupSize + ", blocks.length: " + blocks.length;
+      lb = blocks[0];
+      for (int i = 1; i < blocks.length; i++) {
+        try {
+          boolean offSuccess = stripedBlocks.get(i).offer(blocks[i],
+              90, TimeUnit.SECONDS);
+          if(!offSuccess){
+            String msg = "Fail to put block to stripeBlocks. i = " + i;
+            DFSClient.LOG.info(msg);
+            throw new IOException(msg);
+          } else {
+            DFSClient.LOG.info("Allocate a new block to a streamer. i = " + i
+                + ", block: " + blocks[i]);
+          }
+        } catch (InterruptedException ie) {
+          DFSClient.LOG.info("InterruptedException received when putting" +
+              " a block to stripeBlocks, ie = " + ie);
+        }
+      }
+    } else {
+      try {
+        //wait 90 seconds to get a block from the queue
+        lb = stripedBlocks.get(index).poll(90, TimeUnit.SECONDS);
+      } catch (InterruptedException ie) {
+        DFSClient.LOG.info("InterruptedException received when retrieving " +
+            "a block from stripeBlocks, ie = " + ie);
+      }
+    }
+    return lb;
+  }
+
+  /**
+   * Generate other blocks in a block group according to the first one.
+   *
+   * @param firstBlockInGroup the first block in a block group
+   * @return  other blocks in this group
+   */
+  public static LocatedBlock[] unwrapBlockGroup(
+      final LocatedBlock firstBlockInGroup) {
+    ExtendedBlock eb = firstBlockInGroup.getBlock();
+    DatanodeInfo[] locs = firstBlockInGroup.getLocations();
+    String[] storageIDs = firstBlockInGroup.getStorageIDs();
+    StorageType[] storageTypes = firstBlockInGroup.getStorageTypes();
+    Token<BlockTokenIdentifier> blockToken = firstBlockInGroup.getBlockToken();
+    LocatedBlock[] blocksInGroup = new LocatedBlock[locs.length];
+    for (int i = 0; i < blocksInGroup.length; i++) {
+      //each block in a group has the same number of bytes and timestamp
+      ExtendedBlock extendedBlock = new ExtendedBlock(eb.getBlockPoolId(),
+          eb.getBlockId() + i, eb.getNumBytes(), eb.getGenerationStamp());
+      blocksInGroup[i] = new LocatedBlock(extendedBlock,
+          new DatanodeInfo[] {locs[i]}, new String[]{storageIDs[i]},
+          new StorageType[] {storageTypes[i]});
+      blocksInGroup[i].setBlockToken(blockToken);
+    }
+    return blocksInGroup;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e006f68c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
new file mode 100644
index 0000000..f5a37f3
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -0,0 +1,311 @@
+package org.apache.hadoop.hdfs;
+
+import java.util.Arrays;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.net.Peer;
+import org.apache.hadoop.hdfs.net.TcpPeerServer;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+
+import org.apache.hadoop.hdfs.server.datanode.CachingStrategy;
+import org.apache.hadoop.io.IOUtils;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.hadoop.security.token.Token;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.net.InetSocketAddress;
+import java.net.Socket;
+import java.util.ArrayList;
+import java.util.List;
+
+public class TestDFSStripedOutputStream {
+  private int dataBlocks = HdfsConstants.NUM_DATA_BLOCKS;
+  private int parityBlocks = HdfsConstants.NUM_PARITY_BLOCKS;
+
+  private MiniDFSCluster cluster;
+  private Configuration conf = new Configuration();
+  private DistributedFileSystem fs;
+  int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  int blockSize = 8 * 1024 * 1024;
+  int cellsInBlock = blockSize / cellSize;
+  private int mod = 29;
+
+  @Before
+  public void setup() throws IOException {
+    int numDNs = dataBlocks + parityBlocks + 2;
+    Configuration conf = new Configuration();
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, cellsInBlock * cellSize);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build();
+    cluster.getFileSystem().getClient().createErasureCodingZone("/");
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void TestFileEmpty() throws IOException {
+    testOneFile("/EmptyFile", 0);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneCell1() throws IOException {
+    testOneFile("/SmallerThanOneCell", 1);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneCell2() throws IOException {
+    testOneFile("/SmallerThanOneCell", cellSize - 1);
+  }
+
+  @Test
+  public void TestFileEqualsWithOneCell() throws IOException {
+    testOneFile("/EqualsWithOneCell", cellSize);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneStripe1() throws IOException {
+    testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1);
+  }
+
+  @Test
+  public void TestFileSmallerThanOneStripe2() throws IOException {
+    testOneFile("/SmallerThanOneStripe", cellSize + 123);
+  }
+
+  @Test
+  public void TestFileEqualsWithOneStripe() throws IOException {
+    testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks);
+  }
+
+  @Test
+  public void TestFileMoreThanOneStripe1() throws IOException {
+    testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void TestFileMoreThanOneStripe2() throws IOException {
+    testOneFile("/MoreThanOneStripe2",
+        cellSize * dataBlocks * (cellsInBlock >= 2 ? cellsInBlock / 2 : 1)
+            + cellSize * dataBlocks + 123);
+  }
+
+  @Test
+  public void TestFileFullBlockGroup() throws IOException {
+    testOneFile("/FullBlockGroup", blockSize * dataBlocks);
+  }
+
+  //TODO: The following tests will pass after HDFS-8121 fixed
+//  @Test
+  public void TestFileMoreThanABlockGroup1() throws IOException {
+    testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123);
+  }
+
+  //  @Test
+  public void TestFileMoreThanABlockGroup2() throws IOException {
+    testOneFile("/MoreThanABlockGroup2",
+        blockSize * dataBlocks * 3
+            + (cellsInBlock >= 2 ? cellsInBlock / 2 : 1) * cellSize * dataBlocks
+            + 123);
+  }
+
+  private int stripeDataSize() {
+    return cellSize * dataBlocks;
+  }
+
+  private byte[] generateBytes(int cnt) {
+    byte[] bytes = new byte[cnt];
+    for (int i = 0; i < cnt; i++) {
+      bytes[i] = getByte(i);
+    }
+    return bytes;
+  }
+
+  private byte getByte(long pos) {
+    return (byte) (pos % mod + 1);
+  }
+
+  private void testOneFileUsingDFSStripedInputStream(String src, int writeBytes)
+      throws IOException {
+    Path TestPath = new Path(src);
+    byte[] bytes = generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+
+    //check file length
+    FileStatus status = fs.getFileStatus(TestPath);
+    long fileLength = status.getLen();
+    if (fileLength != writeBytes) {
+      Assert.fail("File Length error: expect=" + writeBytes
+          + ", actual=" + fileLength);
+    }
+
+    DFSStripedInputStream dis = new DFSStripedInputStream(
+        fs.getClient(), src, true);
+    byte[] buf = new byte[writeBytes + 100];
+    int readLen = dis.read(0, buf, 0, buf.length);
+    readLen = readLen >= 0 ? readLen : 0;
+    if (readLen != writeBytes) {
+      Assert.fail("The length of file is not correct.");
+    }
+
+    for (int i = 0; i < writeBytes; i++) {
+      if (getByte(i) != buf[i]) {
+        Assert.fail("Byte at i = " + i + " is wrongly written.");
+      }
+    }
+  }
+
+  private void testOneFile(String src, int writeBytes)
+      throws IOException {
+    Path TestPath = new Path(src);
+
+    int allBlocks = dataBlocks + parityBlocks;
+    byte[] bytes = generateBytes(writeBytes);
+    DFSTestUtil.writeFile(fs, TestPath, new String(bytes));
+
+    //check file length
+    FileStatus status = fs.getFileStatus(TestPath);
+    long fileLength = status.getLen();
+    if (fileLength != writeBytes) {
+      Assert.fail("File Length error: expect=" + writeBytes
+          + ", actual=" + fileLength);
+    }
+
+    List<List<LocatedBlock>> blockGroupList = new ArrayList<>();
+    LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0L);
+
+    for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) {
+      LocatedBlock[] blocks = StripedDataStreamer.unwrapBlockGroup(firstBlock);
+      List<LocatedBlock> oneGroup = Arrays.asList(blocks);
+      blockGroupList.add(oneGroup);
+    }
+
+    //test each block group
+    for (int group = 0; group < blockGroupList.size(); group++) {
+      //get the data of this block
+      List<LocatedBlock> blockList = blockGroupList.get(group);
+      byte[][] dataBlockBytes = new byte[dataBlocks][];
+      byte[][] parityBlockBytes = new byte[allBlocks - dataBlocks][];
+
+      //calculate the size of this block group
+      int lenOfBlockGroup = group < blockGroupList.size() - 1 ?
+          blockSize * dataBlocks :
+          writeBytes - blockSize * (blockGroupList.size() - 1) * dataBlocks;
+      int intactStripes = lenOfBlockGroup / stripeDataSize();
+      int lastStripeLen = lenOfBlockGroup % stripeDataSize();
+
+      //for each block, use BlockReader to read data
+      for (int i = 0; i < blockList.size(); i++) {
+        LocatedBlock lblock = blockList.get(i);
+        if (lblock == null) {
+          continue;
+        }
+        DatanodeInfo[] nodes = lblock.getLocations();
+        ExtendedBlock block = lblock.getBlock();
+        InetSocketAddress targetAddr = NetUtils.createSocketAddr(
+            nodes[0].getXferAddr());
+
+        int lenOfCell = cellSize;
+        if (i == lastStripeLen / cellSize) {
+          lenOfCell = lastStripeLen % cellSize;
+        } else if (i > lastStripeLen / cellSize) {
+          lenOfCell = 0;
+        }
+        int lenOfBlock = cellSize * intactStripes + lenOfCell;
+        byte[] blockBytes = new byte[lenOfBlock];
+        if (i < dataBlocks) {
+          dataBlockBytes[i] = blockBytes;
+        } else {
+          parityBlockBytes[i - dataBlocks] = blockBytes;
+        }
+
+        if (lenOfBlock == 0) {
+          continue;
+        }
+
+        block.setNumBytes(lenOfBlock);
+        BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
+            setFileName(src).
+            setBlock(block).
+            setBlockToken(lblock.getBlockToken()).
+            setInetSocketAddress(targetAddr).
+            setStartOffset(0).
+            setLength(block.getNumBytes()).
+            setVerifyChecksum(true).
+            setClientName("TestStripeLayoutWrite").
+            setDatanodeInfo(nodes[0]).
+            setCachingStrategy(CachingStrategy.newDefaultStrategy()).
+            setClientCacheContext(ClientContext.getFromConf(conf)).
+            setConfiguration(conf).
+            setRemotePeerFactory(new RemotePeerFactory() {
+              @Override
+              public Peer newConnectedPeer(InetSocketAddress addr,
+                                           Token<BlockTokenIdentifier> blockToken,
+                                           DatanodeID datanodeId)
+                  throws IOException {
+                Peer peer = null;
+                Socket sock = NetUtils.getDefaultSocketFactory(conf).createSocket();
+                try {
+                  sock.connect(addr, HdfsServerConstants.READ_TIMEOUT);
+                  sock.setSoTimeout(HdfsServerConstants.READ_TIMEOUT);
+                  peer = TcpPeerServer.peerFromSocket(sock);
+                } finally {
+                  if (peer == null) {
+                    IOUtils.closeSocket(sock);
+                  }
+                }
+                return peer;
+              }
+            }).build();
+
+        blockReader.readAll(blockBytes, 0, lenOfBlock);
+        blockReader.close();
+      }
+
+      //check if we write the data correctly
+      for (int i = 0; i < dataBlockBytes.length; i++) {
+        byte[] cells = dataBlockBytes[i];
+        if (cells == null) {
+          continue;
+        }
+        for (int j = 0; j < cells.length; j++) {
+          byte expected;
+          //calculate the postion of this byte in the file
+          long pos = group * dataBlocks * blockSize
+              + (i * cellSize + j / cellSize * cellSize * dataBlocks)
+              + j % cellSize;
+          if (pos >= writeBytes) {
+            expected = 0;
+          } else {
+            expected = getByte(pos);
+          }
+
+          if (expected != cells[j]) {
+            Assert.fail("Unexpected byte " + cells[j] + ", expect " + expected
+                + ". Block group index is " + group +
+                ", stripe index is " + j / cellSize +
+                ", cell index is " + i + ", byte index is " + j % cellSize);
+          }
+        }
+      }
+    }
+  }
+
+}


[46/50] [abbrv] hadoop git commit: HDFS-8104 Make hard-coded values consistent with the system default schema first before remove them. Contributed by Kai Zheng

Posted by zh...@apache.org.
HDFS-8104 Make hard-coded values consistent with the system default schema first before remove them. Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cb237e47
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cb237e47
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cb237e47

Branch: refs/heads/HDFS-7285
Commit: cb237e47df211c97f4d60227ccae89eda2bb1ea8
Parents: fa773c3
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Apr 10 00:16:28 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:57 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |   4 +-
 .../hadoop/hdfs/protocol/HdfsConstants.java     |  12 +-
 .../hadoop/hdfs/TestPlanReadPortions.java       | 142 +++++++++++++++++++
 .../apache/hadoop/hdfs/TestReadStripedFile.java | 112 ---------------
 4 files changed, 154 insertions(+), 116 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb237e47/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 5078a15..1e695c4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -54,4 +54,6 @@
     HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from
     NameNode (vinayakumarb)
     
-    HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
\ No newline at end of file
+    HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
+
+    HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb237e47/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index a888aa4..11c5260 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -180,11 +180,17 @@ public class HdfsConstants {
   public static final byte WARM_STORAGE_POLICY_ID = 5;
   public static final byte COLD_STORAGE_POLICY_ID = 2;
 
-  public static final byte NUM_DATA_BLOCKS = 3;
-  public static final byte NUM_PARITY_BLOCKS = 2;
+
   public static final long BLOCK_GROUP_INDEX_MASK = 15;
   public static final byte MAX_BLOCKS_IN_GROUP = 16;
 
+  /*
+   * These values correspond to the values used by the system default schema.
+   * TODO: to be removed once all places use schema.
+   */
+
+  public static final byte NUM_DATA_BLOCKS = 6;
+  public static final byte NUM_PARITY_BLOCKS = 3;
   // The chunk size for striped block which is used by erasure coding
-  public static final int BLOCK_STRIPED_CELL_SIZE = 128 * 1024;
+  public static final int BLOCK_STRIPED_CELL_SIZE = 256 * 1024;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb237e47/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
new file mode 100644
index 0000000..cf84b30
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestPlanReadPortions.java
@@ -0,0 +1,142 @@
+/**
+ * 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.junit.Test;
+
+import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion;
+import static org.junit.Assert.*;
+
+public class TestPlanReadPortions {
+
+  // We only support this as num of data blocks. It might be good enough for now
+  // for the purpose, even not flexible yet for any number in a schema.
+  private final short GROUP_SIZE = 3;
+  private final int CELLSIZE = 128 * 1024;
+
+  private void testPlanReadPortions(int startInBlk, int length,
+      int bufferOffset, int[] readLengths, int[] offsetsInBlock,
+      int[][] bufferOffsets, int[][] bufferLengths) {
+    ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE,
+        CELLSIZE, startInBlk, length, bufferOffset);
+    assertEquals(GROUP_SIZE, results.length);
+
+    for (int i = 0; i < GROUP_SIZE; i++) {
+      assertEquals(readLengths[i], results[i].getReadLength());
+      assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock());
+      final int[] bOffsets = results[i].getOffsets();
+      assertArrayEquals(bufferOffsets[i], bOffsets);
+      final int[] bLengths = results[i].getLengths();
+      assertArrayEquals(bufferLengths[i], bLengths);
+    }
+  }
+
+  /**
+   * Test {@link DFSStripedInputStream#planReadPortions}
+   */
+  @Test
+  public void testPlanReadPortions() {
+    /**
+     * start block offset is 0, read cellSize - 10
+     */
+    testPlanReadPortions(0, CELLSIZE - 10, 0,
+        new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0},
+        new int[][]{new int[]{0}, new int[]{}, new int[]{}},
+        new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}});
+
+    /**
+     * start block offset is 0, read 3 * cellSize
+     */
+    testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0,
+        new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0},
+        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}},
+        new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}});
+
+    /**
+     * start block offset is 0, read cellSize + 10
+     */
+    testPlanReadPortions(0, CELLSIZE + 10, 0,
+        new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0},
+        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}},
+        new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}});
+
+    /**
+     * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100
+     */
+    testPlanReadPortions(0, 5 * CELLSIZE + 10, 100,
+        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0},
+        new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE},
+            new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4},
+            new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}},
+        new int[][]{new int[]{CELLSIZE, CELLSIZE},
+            new int[]{CELLSIZE, CELLSIZE},
+            new int[]{CELLSIZE, 10}});
+
+    /**
+     * start block offset is 2, read 3 * cellSize
+     */
+    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100,
+        new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
+        new int[]{2, 0, 0},
+        new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2},
+            new int[]{100 + CELLSIZE - 2},
+            new int[]{100 + CELLSIZE * 2 - 2}},
+        new int[][]{new int[]{CELLSIZE - 2, 2},
+            new int[]{CELLSIZE},
+            new int[]{CELLSIZE}});
+
+    /**
+     * start block offset is 2, read 3 * cellSize + 10
+     */
+    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0,
+        new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE},
+        new int[]{2, 0, 0},
+        new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2},
+            new int[]{CELLSIZE - 2},
+            new int[]{CELLSIZE * 2 - 2}},
+        new int[][]{new int[]{CELLSIZE - 2, 12},
+            new int[]{CELLSIZE},
+            new int[]{CELLSIZE}});
+
+    /**
+     * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10
+     */
+    testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0,
+        new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2},
+        new int[]{CELLSIZE, CELLSIZE - 1, 0},
+        new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1},
+            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1},
+            new int[]{1, 3 * CELLSIZE + 1}},
+        new int[][]{new int[]{CELLSIZE, CELLSIZE},
+            new int[]{1, CELLSIZE, 9},
+            new int[]{CELLSIZE, CELLSIZE}});
+
+    /**
+     * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10
+     */
+    testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0,
+        new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1},
+        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1},
+        new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1},
+            new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1},
+            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}},
+        new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
+            new int[]{CELLSIZE, CELLSIZE, 9},
+            new int[]{1, CELLSIZE, CELLSIZE}});
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cb237e47/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
index 113b73c..09da0cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -29,7 +29,6 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
-import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion;
 import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -78,117 +77,6 @@ public class TestReadStripedFile {
     }
   }
 
-  private void testPlanReadPortions(int startInBlk, int length,
-      int bufferOffset, int[] readLengths, int[] offsetsInBlock,
-      int[][] bufferOffsets, int[][] bufferLengths) {
-    ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE,
-        CELLSIZE, startInBlk, length, bufferOffset);
-    assertEquals(GROUP_SIZE, results.length);
-
-    for (int i = 0; i < GROUP_SIZE; i++) {
-      assertEquals(readLengths[i], results[i].getReadLength());
-      assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock());
-      final int[] bOffsets = results[i].getOffsets();
-      assertArrayEquals(bufferOffsets[i], bOffsets);
-      final int[] bLengths = results[i].getLengths();
-      assertArrayEquals(bufferLengths[i], bLengths);
-    }
-  }
-
-  /**
-   * Test {@link DFSStripedInputStream#planReadPortions}
-   */
-  @Test
-  public void testPlanReadPortions() {
-    /**
-     * start block offset is 0, read cellSize - 10
-     */
-    testPlanReadPortions(0, CELLSIZE - 10, 0,
-        new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0},
-        new int[][]{new int[]{0}, new int[]{}, new int[]{}},
-        new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}});
-
-    /**
-     * start block offset is 0, read 3 * cellSize
-     */
-    testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0,
-        new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0},
-        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}},
-        new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}});
-
-    /**
-     * start block offset is 0, read cellSize + 10
-     */
-    testPlanReadPortions(0, CELLSIZE + 10, 0,
-        new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0},
-        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}},
-        new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}});
-
-    /**
-     * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100
-     */
-    testPlanReadPortions(0, 5 * CELLSIZE + 10, 100,
-        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0},
-        new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE},
-            new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4},
-            new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}},
-        new int[][]{new int[]{CELLSIZE, CELLSIZE},
-            new int[]{CELLSIZE, CELLSIZE},
-            new int[]{CELLSIZE, 10}});
-
-    /**
-     * start block offset is 2, read 3 * cellSize
-     */
-    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100,
-        new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
-        new int[]{2, 0, 0},
-        new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2},
-            new int[]{100 + CELLSIZE - 2},
-            new int[]{100 + CELLSIZE * 2 - 2}},
-        new int[][]{new int[]{CELLSIZE - 2, 2},
-            new int[]{CELLSIZE},
-            new int[]{CELLSIZE}});
-
-    /**
-     * start block offset is 2, read 3 * cellSize + 10
-     */
-    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0,
-        new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE},
-        new int[]{2, 0, 0},
-        new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2},
-            new int[]{CELLSIZE - 2},
-            new int[]{CELLSIZE * 2 - 2}},
-        new int[][]{new int[]{CELLSIZE - 2, 12},
-            new int[]{CELLSIZE},
-            new int[]{CELLSIZE}});
-
-    /**
-     * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10
-     */
-    testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0,
-        new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2},
-        new int[]{CELLSIZE, CELLSIZE - 1, 0},
-        new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1},
-            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1},
-            new int[]{1, 3 * CELLSIZE + 1}},
-        new int[][]{new int[]{CELLSIZE, CELLSIZE},
-            new int[]{1, CELLSIZE, 9},
-            new int[]{CELLSIZE, CELLSIZE}});
-
-    /**
-     * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10
-     */
-    testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0,
-        new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1},
-        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1},
-        new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1},
-            new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1},
-            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}},
-        new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
-            new int[]{CELLSIZE, CELLSIZE, 9},
-            new int[]{1, CELLSIZE, CELLSIZE}});
-  }
-
   private LocatedStripedBlock createDummyLocatedBlock() {
     final long blockGroupID = -1048576;
     DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE];


[27/50] [abbrv] hadoop git commit: HDFS-8005. Erasure Coding: simplify striped block recovery work computation and add tests. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-8005. Erasure Coding: simplify striped block recovery work computation and add tests. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2c14bbe2
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2c14bbe2
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2c14bbe2

Branch: refs/heads/HDFS-7285
Commit: 2c14bbe2692b6b9e203cfd6be7501c651ef8a07f
Parents: 2e39c2b
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 30 13:35:36 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:48:51 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockManager.java    | 134 +++++-------
 .../blockmanagement/DatanodeDescriptor.java     |  14 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   1 +
 .../blockmanagement/TestBlockManager.java       |  33 +--
 .../TestRecoverStripedBlocks.java               | 107 ----------
 .../server/namenode/TestAddStripedBlocks.java   |   2 +-
 .../namenode/TestRecoverStripedBlocks.java      | 210 +++++++++++++++++++
 7 files changed, 290 insertions(+), 211 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c14bbe2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 47865ec..78ada82 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -539,7 +539,7 @@ public class BlockManager {
     // source node returned is not used
     chooseSourceDatanodes(getStoredBlock(block), containingNodes,
         containingLiveReplicasNodes, numReplicas,
-        new LinkedList<Short>(), 1, UnderReplicatedBlocks.LEVEL);
+        new LinkedList<Short>(), UnderReplicatedBlocks.LEVEL);
     
     // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are 
     // not included in the numReplicas.liveReplicas() count
@@ -1376,7 +1376,7 @@ public class BlockManager {
   int computeRecoveryWorkForBlocks(List<List<BlockInfo>> blocksToRecover) {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
-    BlockCollection bc = null;
+    BlockCollection bc;
     int additionalReplRequired;
 
     int scheduledWork = 0;
@@ -1404,13 +1404,10 @@ public class BlockManager {
             containingNodes = new ArrayList<>();
             List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
             NumberReplicas numReplicas = new NumberReplicas();
-            List<Short> missingBlockIndices = new LinkedList<>();
-            DatanodeDescriptor[] srcNodes;
-            int numSourceNodes = bc.isStriped() ?
-                HdfsConstants.NUM_DATA_BLOCKS : 1;
-            srcNodes = chooseSourceDatanodes(
-                block, containingNodes, liveReplicaNodes, numReplicas,
-                missingBlockIndices, numSourceNodes, priority);
+            List<Short> liveBlockIndices = new ArrayList<>();
+            final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block,
+                containingNodes, liveReplicaNodes, numReplicas,
+                liveBlockIndices, priority);
             if(srcNodes == null || srcNodes.length == 0) {
               // block can not be replicated from any node
               LOG.debug("Block " + block + " cannot be recovered " +
@@ -1442,15 +1439,14 @@ public class BlockManager {
             } else {
               additionalReplRequired = 1; // Needed on a new rack
             }
-            if (bc.isStriped()) {
+            if (block.isStriped()) {
+              short[] indices = new short[liveBlockIndices.size()];
+              for (int i = 0 ; i < liveBlockIndices.size(); i++) {
+                indices[i] = liveBlockIndices.get(i);
+              }
               ErasureCodingWork ecw = new ErasureCodingWork(block, bc, srcNodes,
                   containingNodes, liveReplicaNodes, additionalReplRequired,
-                  priority);
-              short[] missingBlockArray = new short[missingBlockIndices.size()];
-              for (int i = 0 ; i < missingBlockIndices.size(); i++) {
-                missingBlockArray[i] = missingBlockIndices.get(i);
-              }
-              ecw.setMissingBlockIndices(missingBlockArray);
+                  priority, indices);
               recovWork.add(ecw);
             } else {
               recovWork.add(new ReplicationWork(block, bc, srcNodes,
@@ -1530,15 +1526,14 @@ public class BlockManager {
           }
 
           // Add block to the to be replicated list
-          if (bc.isStriped()) {
+          if (block.isStriped()) {
             assert rw instanceof ErasureCodingWork;
             assert rw.targets.length > 0;
             rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
                 new ExtendedBlock(namesystem.getBlockPoolId(), block),
                 rw.srcNodes, rw.targets,
-                ((ErasureCodingWork)rw).getMissingBlockIndicies());
-          }
-          else {
+                ((ErasureCodingWork) rw).liveBlockIndicies);
+          } else {
             rw.srcNodes[0].addBlockToBeReplicated(block, targets);
           }
           scheduledWork++;
@@ -1568,9 +1563,9 @@ public class BlockManager {
         DatanodeStorageInfo[] targets = rw.targets;
         if (targets != null && targets.length != 0) {
           StringBuilder targetList = new StringBuilder("datanode(s)");
-          for (int k = 0; k < targets.length; k++) {
+          for (DatanodeStorageInfo target : targets) {
             targetList.append(' ');
-            targetList.append(targets[k].getDatanodeDescriptor());
+            targetList.append(target.getDatanodeDescriptor());
           }
           blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNodes,
               rw.block, targetList);
@@ -1681,11 +1676,8 @@ public class BlockManager {
    * @param numReplicas NumberReplicas instance to be initialized with the
    *                    counts of live, corrupt, excess, and decommissioned
    *                    replicas of the given block.
-   * @param missingBlockIndices List to be populated with indices of missing
-   *                            blocks in a striped block group or missing
-   *                            replicas of a replicated block
-   * @param numSourceNodes integer specifying the number of source nodes to
-   *                       choose
+   * @param liveBlockIndices List to be populated with indices of healthy
+   *                         blocks in a striped block group
    * @param priority integer representing replication priority of the given
    *                 block
    * @return the array of DatanodeDescriptor of the chosen nodes from which to
@@ -1696,24 +1688,20 @@ public class BlockManager {
       List<DatanodeDescriptor> containingNodes,
       List<DatanodeStorageInfo> nodesContainingLiveReplicas,
       NumberReplicas numReplicas,
-      List<Short> missingBlockIndices, int numSourceNodes, int priority) {
+      List<Short> liveBlockIndices, int priority) {
     containingNodes.clear();
     nodesContainingLiveReplicas.clear();
-    LinkedList<DatanodeDescriptor> srcNodes = new LinkedList<>();
+    List<DatanodeDescriptor> srcNodes = new ArrayList<>();
     int live = 0;
     int decommissioned = 0;
     int decommissioning = 0;
     int corrupt = 0;
     int excess = 0;
-    missingBlockIndices.clear();
-    Set<Short> healthyIndices = new HashSet<>();
+    liveBlockIndices.clear();
+    final boolean isStriped = block.isStriped();
 
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
-      if (block.isStriped()) {
-        healthyIndices.add((short) ((BlockInfoStriped) block).
-            getStorageBlockIndex(storage));
-      }
+    for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       LightWeightLinkedSet<BlockInfo> excessBlocks =
         excessReplicateMap.get(node.getDatanodeUuid());
@@ -1752,27 +1740,19 @@ public class BlockManager {
       if(node.isDecommissioned())
         continue;
 
-      // We got this far, current node is a reasonable choice
-      if(srcNodes.size() < numSourceNodes) {
+      if(isStriped || srcNodes.isEmpty()) {
         srcNodes.add(node);
+        if (isStriped) {
+          liveBlockIndices.add((short) ((BlockInfoStriped) block).
+              getStorageBlockIndex(storage));
+        }
         continue;
       }
-      // switch to a different node randomly
+      // for replicated block, switch to a different node randomly
       // this to prevent from deterministically selecting the same node even
       // if the node failed to replicate the block on previous iterations
-      if(DFSUtil.getRandom().nextBoolean()) {
-        int pos = DFSUtil.getRandom().nextInt(numSourceNodes);
-        if(!srcNodes.get(pos).isDecommissionInProgress()) {
-          srcNodes.set(pos, node);
-        }
-      }
-    }
-    if (block.isStriped()) {
-      for (short i = 0; i < HdfsConstants.NUM_DATA_BLOCKS +
-          HdfsConstants.NUM_PARITY_BLOCKS; i++) {
-        if (!healthyIndices.contains(i)) {
-          missingBlockIndices.add(i);
-        }
+      if (!isStriped && DFSUtil.getRandom().nextBoolean()) {
+        srcNodes.set(0, node);
       }
     }
     if(numReplicas != null)
@@ -3833,25 +3813,25 @@ public class BlockManager {
    * to represent a task to recover a block through replication or erasure
    * coding. Recovery is done by transferring data from srcNodes to targets
    */
-  private static class BlockRecoveryWork {
-    protected final BlockInfo block;
-    protected final BlockCollection bc;
+  private abstract static class BlockRecoveryWork {
+    final BlockInfo block;
+    final BlockCollection bc;
 
     /**
      * An erasure coding recovery task has multiple source nodes.
      * A replication task only has 1 source node, stored on top of the array
      */
-    protected final DatanodeDescriptor[] srcNodes;
+    final DatanodeDescriptor[] srcNodes;
     /** Nodes containing the block; avoid them in choosing new targets */
-    protected final List<DatanodeDescriptor> containingNodes;
+    final List<DatanodeDescriptor> containingNodes;
     /** Required by {@link BlockPlacementPolicy#chooseTarget} */
-    protected final List<DatanodeStorageInfo> liveReplicaStorages;
-    protected final int additionalReplRequired;
+    final List<DatanodeStorageInfo> liveReplicaStorages;
+    final int additionalReplRequired;
 
-    protected DatanodeStorageInfo[] targets;
-    protected final int priority;
+    DatanodeStorageInfo[] targets;
+    final int priority;
 
-    public BlockRecoveryWork(BlockInfo block,
+    BlockRecoveryWork(BlockInfo block,
         BlockCollection bc,
         DatanodeDescriptor[] srcNodes,
         List<DatanodeDescriptor> containingNodes,
@@ -3868,15 +3848,13 @@ public class BlockManager {
       this.targets = null;
     }
 
-    protected void chooseTargets(BlockPlacementPolicy blockplacement,
+    abstract void chooseTargets(BlockPlacementPolicy blockplacement,
         BlockStoragePolicySuite storagePolicySuite,
-        Set<Node> excludedNodes) {
-    }
+        Set<Node> excludedNodes);
   }
 
   private static class ReplicationWork extends BlockRecoveryWork {
-
-    public ReplicationWork(BlockInfo block,
+    ReplicationWork(BlockInfo block,
         BlockCollection bc,
         DatanodeDescriptor[] srcNodes,
         List<DatanodeDescriptor> containingNodes,
@@ -3888,7 +3866,8 @@ public class BlockManager {
       LOG.debug("Creating a ReplicationWork to recover " + block);
     }
 
-    protected void chooseTargets(BlockPlacementPolicy blockplacement,
+    @Override
+    void chooseTargets(BlockPlacementPolicy blockplacement,
         BlockStoragePolicySuite storagePolicySuite,
         Set<Node> excludedNodes) {
       assert srcNodes.length > 0
@@ -3905,30 +3884,23 @@ public class BlockManager {
   }
 
   private static class ErasureCodingWork extends BlockRecoveryWork {
+    final short[] liveBlockIndicies;
 
-    private short[] missingBlockIndicies = null;
-
-    public ErasureCodingWork(BlockInfo block,
+    ErasureCodingWork(BlockInfo block,
         BlockCollection bc,
         DatanodeDescriptor[] srcNodes,
         List<DatanodeDescriptor> containingNodes,
         List<DatanodeStorageInfo> liveReplicaStorages,
         int additionalReplRequired,
-        int priority) {
+        int priority, short[] liveBlockIndicies) {
       super(block, bc, srcNodes, containingNodes,
           liveReplicaStorages, additionalReplRequired, priority);
+      this.liveBlockIndicies = liveBlockIndicies;
       LOG.debug("Creating an ErasureCodingWork to recover " + block);
     }
 
-    public short[] getMissingBlockIndicies() {
-      return missingBlockIndicies;
-    }
-
-    public void setMissingBlockIndices(short[] missingBlockIndicies) {
-      this.missingBlockIndicies = missingBlockIndicies;
-    }
-
-    protected void chooseTargets(BlockPlacementPolicy blockplacement,
+    @Override
+    void chooseTargets(BlockPlacementPolicy blockplacement,
         BlockStoragePolicySuite storagePolicySuite,
         Set<Node> excludedNodes) {
       try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c14bbe2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 7bc5e7e..15427f7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -106,14 +106,14 @@ public class DatanodeDescriptor extends DatanodeInfo {
     public final ExtendedBlock block;
     public final DatanodeDescriptor[] sources;
     public final DatanodeStorageInfo[] targets;
-    public final short[] missingBlockIndices;
+    public final short[] liveBlockIndices;
 
     BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources,
-        DatanodeStorageInfo[] targets, short[] missingBlockIndices) {
+        DatanodeStorageInfo[] targets, short[] liveBlockIndices) {
       this.block = block;
       this.sources = sources;
       this.targets = targets;
-      this.missingBlockIndices = missingBlockIndices;
+      this.liveBlockIndices = liveBlockIndices;
     }
 
     @Override
@@ -122,6 +122,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
           append("Recovering ").append(block).
           append(" From: ").append(Arrays.asList(sources)).
           append(" To: ").append(Arrays.asList(targets)).append(")\n").
+          append(" Block Indices: ").append(Arrays.asList(liveBlockIndices)).
           toString();
     }
   }
@@ -635,10 +636,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
    * Store block erasure coding work.
    */
   void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources,
-      DatanodeStorageInfo[] targets, short[] missingBlockIndicies) {
+      DatanodeStorageInfo[] targets, short[] liveBlockIndices) {
     assert(block != null && sources != null && sources.length > 0);
     BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets,
-        missingBlockIndicies);
+        liveBlockIndices);
     erasurecodeBlocks.offer(task);
     BlockManager.LOG.debug("Adding block recovery task " + task +
         "to " + getName() + ", current queue size is " +
@@ -679,7 +680,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * The number of work items that are pending to be replicated
    */
-  int getNumberOfBlocksToBeErasureCoded() {
+  @VisibleForTesting
+  public int getNumberOfBlocksToBeErasureCoded() {
     return erasurecodeBlocks.size();
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c14bbe2/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index b40022e8..2aa44cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -402,6 +402,7 @@ public class INodeFile extends INodeWithAdditionalFields
 
   /** The same as getFileReplication(null). */
   @Override // INodeFileAttributes
+  // TODO striped
   public final short getFileReplication() {
     return getFileReplication(CURRENT_STATE_ID);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c14bbe2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 43f4607..f7504ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -508,30 +508,33 @@ public class TestBlockManager {
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
+            new ArrayList<Short>(),
+            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
 
-    assertNull("Does not choose a source node for a less-than-highest-priority"
-        + " replication since all available source nodes have reached"
-        + " their replication limits.",
+    assertEquals("Does not choose a source node for a less-than-highest-priority"
+            + " replication since all available source nodes have reached"
+            + " their replication limits.", 0,
         bm.chooseSourceDatanodes(
             bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)[0]);
+            new ArrayList<Short>(),
+            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED).length);
 
     // Increase the replication count to test replication count > hard limit
     DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
     origNodes.get(0).addBlockToBeReplicated(aBlock, targets);
 
-    assertNull("Does not choose a source node for a highest-priority"
-        + " replication when all available nodes exceed the hard limit.",
+    assertEquals("Does not choose a source node for a highest-priority"
+            + " replication when all available nodes exceed the hard limit.", 0,
         bm.chooseSourceDatanodes(
             bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
+            new ArrayList<Short>(),
+            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY).length);
   }
 
   @Test
@@ -556,26 +559,24 @@ public class TestBlockManager {
             bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
-            new NumberReplicas(), new LinkedList<Short>(), 1,
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+            new NumberReplicas(), new LinkedList<Short>(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)[0]);
 
 
     // Increase the replication count to test replication count > hard limit
     DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
     origNodes.get(0).addBlockToBeReplicated(aBlock, targets);
 
-    assertNull("Does not choose a source decommissioning node for a normal"
-        + " replication when all available nodes exceed the hard limit.",
+    assertEquals("Does not choose a source decommissioning node for a normal"
+        + " replication when all available nodes exceed the hard limit.", 0,
         bm.chooseSourceDatanodes(
             bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
-            new NumberReplicas(), new LinkedList<Short>(), 1,
-            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
+            new NumberReplicas(), new LinkedList<Short>(),
+            UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED).length);
   }
 
-
-
   @Test
   public void testSafeModeIBR() throws Exception {
     DatanodeDescriptor node = spy(nodes.get(0));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c14bbe2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
deleted file mode 100644
index d883c9b..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
+++ /dev/null
@@ -1,107 +0,0 @@
-/**
- * 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.server.blockmanagement;
-
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.HdfsConfiguration;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
-import org.apache.hadoop.hdfs.client.HdfsAdmin;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-import java.util.Iterator;
-
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
-import static org.junit.Assert.assertTrue;
-
-public class TestRecoverStripedBlocks {
-  private final short GROUP_SIZE =
-      HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short NUM_OF_DATANODES = GROUP_SIZE + 1;
-  private Configuration conf;
-  private MiniDFSCluster cluster;
-  private DistributedFileSystem fs;
-  private static final int BLOCK_SIZE = 1024;
-  private HdfsAdmin dfsAdmin;
-  private FSNamesystem namesystem;
-  private Path ECFilePath;
-
-  @Before
-  public void setupCluster() throws IOException {
-    conf = new HdfsConfiguration();
-    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
-    // Large value to make sure the pending replication request can stay in
-    // DatanodeDescriptor.replicateBlocks before test timeout.
-    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100);
-    // Make sure BlockManager can pull all blocks from UnderReplicatedBlocks via
-    // chooseUnderReplicatedBlocks at once.
-    conf.setInt(
-        DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5);
-
-    cluster = new MiniDFSCluster.Builder(conf).
-        numDataNodes(NUM_OF_DATANODES).build();
-    cluster.waitActive();
-    fs = cluster.getFileSystem();
-    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
-    namesystem = cluster.getNamesystem();
-    ECFilePath = new Path("/ecfile");
-    DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, GROUP_SIZE, 0);
-    dfsAdmin.setStoragePolicy(ECFilePath, EC_STORAGE_POLICY_NAME);
-  }
-
-  @Test
-  public void testMissingStripedBlock() throws Exception {
-    final BlockManager bm = cluster.getNamesystem().getBlockManager();
-    ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, ECFilePath);
-    Iterator<DatanodeStorageInfo> storageInfos =
-        bm.blocksMap.getStorages(b.getLocalBlock())
-            .iterator();
-
-    DatanodeDescriptor firstDn = storageInfos.next().getDatanodeDescriptor();
-    Iterator<BlockInfo> it = firstDn.getBlockIterator();
-    int missingBlkCnt = 0;
-    while (it.hasNext()) {
-      BlockInfo blk = it.next();
-      BlockManager.LOG.debug("Block " + blk + " will be lost");
-      missingBlkCnt++;
-    }
-    BlockManager.LOG.debug("Missing in total " + missingBlkCnt + " blocks");
-
-    bm.getDatanodeManager().removeDatanode(firstDn);
-
-    bm.computeDatanodeWork();
-
-    short cnt = 0;
-    for (DataNode dn : cluster.getDataNodes()) {
-      DatanodeDescriptor dnDescriptor =
-          bm.getDatanodeManager().getDatanode(dn.getDatanodeUuid());
-      cnt += dnDescriptor.getNumberOfBlocksToBeErasureCoded();
-    }
-
-    assertTrue("Counting the number of outstanding EC tasks", cnt == missingBlkCnt);
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c14bbe2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 7d7c81e..215a4e4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -224,7 +224,7 @@ public class TestAddStripedBlocks {
       int i = 0;
       for (DataNode dn : cluster.getDataNodes()) {
         final Block block = new Block(lastBlock.getBlockId() + i++,
-            lastBlock.getGenerationStamp(), 0);
+            0, lastBlock.getGenerationStamp());
         DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
         storageIDs.add(storage.getStorageID());
         StorageReceivedDeletedBlocks[] reports = DFSTestUtil

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c14bbe2/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
new file mode 100644
index 0000000..b9fd4fe
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -0,0 +1,210 @@
+/**
+ * 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.server.namenode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Iterator;
+import java.util.List;
+import java.util.UUID;
+
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+public class TestRecoverStripedBlocks {
+  private final short GROUP_SIZE =
+      NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
+  private MiniDFSCluster cluster;
+  private final Path dirPath = new Path("/dir");
+  private Path filePath = new Path(dirPath, "file");
+
+  @Before
+  public void setup() throws IOException {
+    final Configuration conf = new HdfsConfiguration();
+    // Large value to make sure the pending replication request can stay in
+    // DatanodeDescriptor.replicateBlocks before test timeout.
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100);
+    // Make sure BlockManager can pull all blocks from UnderReplicatedBlocks via
+    // chooseUnderReplicatedBlocks at once.
+    conf.setInt(
+        DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5);
+
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE + 1)
+        .build();
+    cluster.waitActive();
+  }
+
+  @After
+  public void tearDown() throws Exception {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  public static void createECFile(MiniDFSCluster cluster, Path file, Path dir,
+      int numBlocks) throws Exception {
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    dfs.mkdirs(dir);
+    dfs.setStoragePolicy(dir, EC_STORAGE_POLICY_NAME);
+
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1); // create an empty file
+
+      FSNamesystem ns = cluster.getNamesystem();
+      FSDirectory fsdir = ns.getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+
+      ExtendedBlock previous = null;
+      for (int i = 0; i < numBlocks; i++) {
+        Block newBlock = createBlock(cluster.getDataNodes(), ns,
+            file.toString(), fileNode, dfs.getClient().getClientName(),
+            previous);
+        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
+      }
+
+      ns.completeFile(file.toString(), dfs.getClient().getClientName(),
+          previous, fileNode.getId());
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+
+  static Block createBlock(List<DataNode> dataNodes, FSNamesystem ns,
+      String file, INodeFile fileNode, String clientName,
+      ExtendedBlock previous) throws Exception {
+    ns.getAdditionalBlock(file, fileNode.getId(), clientName, previous, null,
+        null);
+
+    final BlockInfo lastBlock = fileNode.getLastBlock();
+    final int groupSize = fileNode.getBlockReplication();
+    // 1. RECEIVING_BLOCK IBR
+    int i = 0;
+    for (DataNode dn : dataNodes) {
+      if (i < groupSize) {
+        final Block block = new Block(lastBlock.getBlockId() + i++, 0,
+            lastBlock.getGenerationStamp());
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block,
+                ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+        }
+      }
+    }
+
+    // 2. RECEIVED_BLOCK IBR
+    i = 0;
+    for (DataNode dn : dataNodes) {
+      if (i < groupSize) {
+        final Block block = new Block(lastBlock.getBlockId() + i++,
+            BLOCK_STRIPED_CHUNK_SIZE, lastBlock.getGenerationStamp());
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block,
+                ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+        }
+      }
+    }
+
+    lastBlock.setNumBytes(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS);
+    return lastBlock;
+  }
+
+  @Test
+  public void testMissingStripedBlock() throws Exception {
+    final int numBlocks = 4;
+    createECFile(cluster, filePath, dirPath, numBlocks);
+
+    // make sure the file is complete in NN
+    final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
+        .getINode4Write(filePath.toString()).asFile();
+    assertFalse(fileNode.isUnderConstruction());
+    assertTrue(fileNode.isWithStripedBlocks());
+    BlockInfo[] blocks = fileNode.getBlocks();
+    assertEquals(numBlocks, blocks.length);
+    for (BlockInfo blk : blocks) {
+      assertTrue(blk.isStriped());
+      assertTrue(blk.isComplete());
+      assertEquals(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes());
+      final BlockInfoStriped sb = (BlockInfoStriped) blk;
+      assertEquals(GROUP_SIZE, sb.numNodes());
+    }
+
+    final BlockManager bm = cluster.getNamesystem().getBlockManager();
+    BlockInfo firstBlock = fileNode.getBlocks()[0];
+    DatanodeStorageInfo[] storageInfos = bm.getStorages(firstBlock);
+
+    DatanodeDescriptor secondDn = storageInfos[1].getDatanodeDescriptor();
+    assertEquals(numBlocks, secondDn.numBlocks());
+
+    bm.getDatanodeManager().removeDatanode(secondDn);
+
+    BlockManagerTestUtil.getComputedDatanodeWork(bm);
+
+    // all the recovery work will be scheduled on the last DN
+    DataNode lastDn = cluster.getDataNodes().get(GROUP_SIZE);
+    DatanodeDescriptor last =
+          bm.getDatanodeManager().getDatanode(lastDn.getDatanodeId());
+    assertEquals("Counting the number of outstanding EC tasks", numBlocks,
+        last.getNumberOfBlocksToBeErasureCoded());
+    List<BlockECRecoveryInfo> recovery = last.getErasureCodeCommand(numBlocks);
+    for (BlockECRecoveryInfo info : recovery) {
+      assertEquals(1, info.targets.length);
+      assertEquals(last, info.targets[0].getDatanodeDescriptor());
+      assertEquals(GROUP_SIZE - 1, info.sources.length);
+      assertEquals(GROUP_SIZE - 1, info.liveBlockIndices.length);
+    }
+  }
+}


[50/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit mainly addresses HDFS-8081 and HDFS-8048. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging trunk changes (this commit mainly addresses HDFS-8081 and HDFS-8048. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/01af25fb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/01af25fb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/01af25fb

Branch: refs/heads/HDFS-7285
Commit: 01af25fbe1198080ddb905657090a4e273548f16
Parents: b155a2e
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 13 10:56:24 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:58 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSInputStream.java |  4 ++--
 .../apache/hadoop/hdfs/DFSStripedInputStream.java   | 16 +++++++++-------
 .../apache/hadoop/hdfs/DFSStripedOutputStream.java  |  3 ++-
 .../hadoop/hdfs/server/namenode/FSNamesystem.java   |  5 +++--
 .../hadoop/hdfs/TestDFSStripedOutputStream.java     |  3 ++-
 5 files changed, 18 insertions(+), 13 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/01af25fb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index e94c41a..6006693 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -1099,7 +1099,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
     final int length = (int) (end - start + 1);
-    actualGetFromOneDataNode(datanode, block, start, end, buf,
+    actualGetFromOneDataNode(datanode, blockStartOffset, start, end, buf,
         new int[]{offset}, new int[]{length}, corruptedBlockMap);
   }
 
@@ -1118,7 +1118,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    *                          block replica
    */
   void actualGetFromOneDataNode(final DNAddrPair datanode,
-      LocatedBlock block, final long startInBlk, final long endInBlk,
+      long blockStartOffset, final long startInBlk, final long endInBlk,
       byte[] buf, int[] offsets, int[] lengths,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01af25fb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
index 077b0f8..8a431b1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -224,7 +224,7 @@ public class DFSStripedInputStream extends DFSInputStream {
    * Real implementation of pread.
    */
   @Override
-  protected void fetchBlockByteRange(LocatedBlock block, long start,
+  protected void fetchBlockByteRange(long blockStartOffset, long start,
       long end, byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
@@ -234,7 +234,7 @@ public class DFSStripedInputStream extends DFSInputStream {
     int len = (int) (end - start + 1);
 
     // Refresh the striped block group
-    block = getBlockGroupAt(block.getStartOffset());
+    LocatedBlock block = getBlockGroupAt(blockStartOffset);
     assert block instanceof LocatedStripedBlock : "NameNode" +
         " should return a LocatedStripedBlock for a striped file";
     LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
@@ -254,9 +254,11 @@ public class DFSStripedInputStream extends DFSInputStream {
       DatanodeInfo loc = blks[i].getLocations()[0];
       StorageType type = blks[i].getStorageTypes()[0];
       DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr(
-          loc.getXferAddr(dfsClient.getConf().connectToDnViaHostname)), type);
-      Callable<Void> readCallable = getFromOneDataNode(dnAddr, blks[i],
-          rp.startOffsetInBlock, rp.startOffsetInBlock + rp.readLength - 1, buf,
+          loc.getXferAddr(dfsClient.getConf().isConnectToDnViaHostname())),
+          type);
+      Callable<Void> readCallable = getFromOneDataNode(dnAddr,
+          blks[i].getStartOffset(), rp.startOffsetInBlock,
+          rp.startOffsetInBlock + rp.readLength - 1, buf,
           rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i);
       Future<Void> getFromDNRequest = stripedReadsService.submit(readCallable);
       DFSClient.LOG.debug("Submitting striped read request for " + blks[i]);
@@ -272,7 +274,7 @@ public class DFSStripedInputStream extends DFSInputStream {
   }
 
   private Callable<Void> getFromOneDataNode(final DNAddrPair datanode,
-      final LocatedBlock block, final long start, final long end,
+      final long blockStartOffset, final long start, final long end,
       final byte[] buf, final int[] offsets, final int[] lengths,
       final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
       final int hedgedReadId) {
@@ -283,7 +285,7 @@ public class DFSStripedInputStream extends DFSInputStream {
         TraceScope scope =
             Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan);
         try {
-          actualGetFromOneDataNode(datanode, block, start,
+          actualGetFromOneDataNode(datanode, blockStartOffset, start,
               end, buf, offsets, lengths, corruptedBlockMap);
         } finally {
           scope.close();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01af25fb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
index aded4fe..1d0e1be 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java
@@ -284,7 +284,8 @@ public class DFSStripedOutputStream extends DFSOutputStream {
     }
     for (StripedDataStreamer streamer : streamers) {
       streamer.setLastException(new IOException("Lease timeout of "
-          + (dfsClient.getHdfsTimeout()/1000) + " seconds expired."));
+          + (dfsClient.getConf().getHdfsTimeout()/1000) +
+          " seconds expired."));
     }
     closeThreads(true);
     dfsClient.endFileLease(fileId);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01af25fb/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index a504907..6d89b0c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3067,7 +3067,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     final long blockSize;
     final short numTargets;
     final byte storagePolicyID;
-    final boolean isStriped;
     Node clientNode = null;
     String clientMachine = null;
 
@@ -3109,7 +3108,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       clientNode = blockManager.getDatanodeManager().getDatanodeByHost(
           clientMachine);
       // TODO: make block group size configurable (HDFS-7337)
-      isStriped = pendingFile.isStriped();
+      boolean isStriped = pendingFile.isStriped();
       numTargets = isStriped ?
           HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS :
           pendingFile.getFileReplication();
@@ -3138,6 +3137,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException {
     Block newBlock = null;
     long offset;
+    boolean isStriped;
     checkOperation(OperationCategory.WRITE);
     waitForLoadingFSImage();
     writeLock();
@@ -3168,6 +3168,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       commitOrCompleteLastBlock(pendingFile, fileState.iip,
                                 ExtendedBlock.getLocalBlock(previous));
 
+      isStriped = pendingFile.isStriped();
       // allocate new block, record block locations in INode.
       newBlock = createNewBlock(isStriped);
       saveAllocatedBlock(src, fileState.iip, newBlock, targets,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/01af25fb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
index f5a37f3..ee6998b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java
@@ -5,6 +5,7 @@ import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.client.impl.DfsClientConf;
 import org.apache.hadoop.hdfs.net.Peer;
 import org.apache.hadoop.hdfs.net.TcpPeerServer;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
@@ -241,7 +242,7 @@ public class TestDFSStripedOutputStream {
         }
 
         block.setNumBytes(lenOfBlock);
-        BlockReader blockReader = new BlockReaderFactory(new DFSClient.Conf(conf)).
+        BlockReader blockReader = new BlockReaderFactory(new DfsClientConf(conf)).
             setFileName(src).
             setBlock(block).
             setBlockToken(lblock.getBlockToken()).


[44/50] [abbrv] hadoop git commit: HDFS-8074 Define a system-wide default EC schema. Contributed by Kai Zheng

Posted by zh...@apache.org.
HDFS-8074 Define a system-wide default EC schema. Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/fa773c36
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/fa773c36
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/fa773c36

Branch: refs/heads/HDFS-7285
Commit: fa773c36cfeb094a7a9f174d4b9aa6d3039142a8
Parents: 228c6b4
Author: Kai Zheng <ka...@intel.com>
Authored: Thu Apr 9 01:30:02 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:57 2015 -0700

----------------------------------------------------------------------
 .../src/main/conf/ecschema-def.xml              |  5 --
 .../apache/hadoop/io/erasurecode/ECSchema.java  | 57 +++++++++++++++++-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  4 +-
 .../hdfs/server/namenode/ECSchemaManager.java   | 62 ++++++++++++++++++++
 4 files changed, 120 insertions(+), 8 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa773c36/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
index e619485..e36d386 100644
--- a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
+++ b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
@@ -27,11 +27,6 @@ You can modify and remove those not used yet, or add new ones.
 -->
 
 <schemas>
-  <schema name="RS-6-3">
-    <k>6</k>
-    <m>3</m>
-    <codec>RS</codec>
-  </schema>
   <schema name="RS-10-4">
     <k>10</k>
     <m>4</m>

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa773c36/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
index 27be00e..8c3310e 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -23,12 +23,12 @@ import java.util.Map;
 /**
  * Erasure coding schema to housekeeper relevant information.
  */
-public class ECSchema {
+public final class ECSchema {
   public static final String NUM_DATA_UNITS_KEY = "k";
   public static final String NUM_PARITY_UNITS_KEY = "m";
   public static final String CODEC_NAME_KEY = "codec";
   public static final String CHUNK_SIZE_KEY = "chunkSize";
-  public static final int DEFAULT_CHUNK_SIZE = 64 * 1024; // 64K
+  public static final int DEFAULT_CHUNK_SIZE = 256 * 1024; // 256K
 
   private String schemaName;
   private String codecName;
@@ -82,6 +82,18 @@ public class ECSchema {
   }
 
   /**
+   * Constructor with key parameters provided.
+   * @param schemaName
+   * @param codecName
+   * @param numDataUnits
+   * @param numParityUnits
+   */
+  public ECSchema(String schemaName, String codecName,
+                  int numDataUnits, int numParityUnits) {
+    this(schemaName, codecName, numDataUnits, numParityUnits, null);
+  }
+
+  /**
    * Constructor with key parameters provided. Note the options may contain
    * additional information for the erasure codec to interpret further.
    * @param schemaName
@@ -200,4 +212,45 @@ public class ECSchema {
 
     return sb.toString();
   }
+
+  @Override
+  public boolean equals(Object o) {
+    if (this == o) {
+      return true;
+    }
+    if (o == null || getClass() != o.getClass()) {
+      return false;
+    }
+
+    ECSchema ecSchema = (ECSchema) o;
+
+    if (numDataUnits != ecSchema.numDataUnits) {
+      return false;
+    }
+    if (numParityUnits != ecSchema.numParityUnits) {
+      return false;
+    }
+    if (chunkSize != ecSchema.chunkSize) {
+      return false;
+    }
+    if (!schemaName.equals(ecSchema.schemaName)) {
+      return false;
+    }
+    if (!codecName.equals(ecSchema.codecName)) {
+      return false;
+    }
+    return options.equals(ecSchema.options);
+  }
+
+  @Override
+  public int hashCode() {
+    int result = schemaName.hashCode();
+    result = 31 * result + codecName.hashCode();
+    result = 31 * result + options.hashCode();
+    result = 31 * result + numDataUnits;
+    result = 31 * result + numParityUnits;
+    result = 31 * result + chunkSize;
+
+    return result;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa773c36/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 7423033..5078a15 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -52,4 +52,6 @@
     manage EC zones (Zhe Zhang)
 
     HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from
-    NameNode (vinayakumarb)
\ No newline at end of file
+    NameNode (vinayakumarb)
+    
+    HDFS-8074. Define a system-wide default EC schema. (Kai Zheng)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/fa773c36/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
new file mode 100644
index 0000000..b001c57
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ECSchemaManager.java
@@ -0,0 +1,62 @@
+/**
+ * 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.server.namenode;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * This manages EC schemas predefined and activated in the system. It loads from
+ * predefined ones in XML and syncs with persisted ones in NameNode image.
+ *
+ * This class is instantiated by the FSNamesystem.
+ */
+@InterfaceAudience.LimitedPrivate({"HDFS"})
+public final class ECSchemaManager {
+
+  private static final int DEFAULT_DATA_BLOCKS = 6;
+  private static final int DEFAULT_PARITY_BLOCKS = 3;
+  private static final String DEFAULT_CODEC_NAME = "rs";
+  private static final String DEFAULT_SCHEMA_NAME = "SYS-DEFAULT-RS-6-3";
+
+  private static ECSchema SYS_DEFAULT_SCHEMA = new ECSchema(DEFAULT_SCHEMA_NAME,
+      DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS);
+
+  /**
+   * Get system-wide default EC schema, which can be used by default when no
+   * schema is specified for an EC zone.
+   * @return schema
+   */
+  public static ECSchema getSystemDefaultSchema() {
+    return SYS_DEFAULT_SCHEMA;
+  }
+
+  /**
+   * Tell the specified schema is the system default one or not.
+   * @param schema
+   * @return true if it's the default false otherwise
+   */
+  public static boolean isSystemDefault(ECSchema schema) {
+    if (schema == null) {
+      throw new IllegalArgumentException("Invalid schema parameter");
+    }
+
+    // schema name is the identifier, but for safety we check all properties.
+    return SYS_DEFAULT_SCHEMA.equals(schema);
+  }
+}


[02/50] [abbrv] hadoop git commit: HDFS-7749. Erasure Coding: Add striped block support in INodeFile. Contributed by Jing Zhao.

Posted by zh...@apache.org.
http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
new file mode 100644
index 0000000..47445be
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FileWithStripedBlocksFeature.java
@@ -0,0 +1,112 @@
+/**
+ * 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.server.namenode;
+
+import com.google.common.base.Preconditions;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+
+/**
+ * Feature for file with striped blocks
+ */
+class FileWithStripedBlocksFeature implements INode.Feature {
+  private BlockInfoStriped[] blocks;
+
+  FileWithStripedBlocksFeature() {
+    blocks = new BlockInfoStriped[0];
+  }
+
+  FileWithStripedBlocksFeature(BlockInfoStriped[] blocks) {
+    Preconditions.checkArgument(blocks != null);
+    this.blocks = blocks;
+  }
+
+  BlockInfoStriped[] getBlocks() {
+    return this.blocks;
+  }
+
+  void setBlock(int index, BlockInfoStriped blk) {
+    blocks[index] = blk;
+  }
+
+  BlockInfoStriped getLastBlock() {
+    return blocks == null || blocks.length == 0 ?
+        null : blocks[blocks.length - 1];
+  }
+
+  int numBlocks() {
+    return blocks == null ? 0 : blocks.length;
+  }
+
+  void updateBlockCollection(INodeFile file) {
+    if (blocks != null) {
+      for (BlockInfoStriped blk : blocks) {
+        blk.setBlockCollection(file);
+      }
+    }
+  }
+
+  private void setBlocks(BlockInfoStriped[] blocks) {
+    this.blocks = blocks;
+  }
+
+  void addBlock(BlockInfoStriped newBlock) {
+    if (this.blocks == null) {
+      this.setBlocks(new BlockInfoStriped[]{newBlock});
+    } else {
+      int size = this.blocks.length;
+      BlockInfoStriped[] newlist = new BlockInfoStriped[size + 1];
+      System.arraycopy(this.blocks, 0, newlist, 0, size);
+      newlist[size] = newBlock;
+      this.setBlocks(newlist);
+    }
+  }
+
+  boolean removeLastBlock(Block oldblock) {
+    if (blocks == null || blocks.length == 0) {
+      return false;
+    }
+    int newSize = blocks.length - 1;
+    if (!blocks[newSize].equals(oldblock)) {
+      return false;
+    }
+
+    //copy to a new list
+    BlockInfoStriped[] newlist = new BlockInfoStriped[newSize];
+    System.arraycopy(blocks, 0, newlist, 0, newSize);
+    setBlocks(newlist);
+    return true;
+  }
+
+  void truncateStripedBlocks(int n) {
+    final BlockInfoStriped[] newBlocks;
+    if (n == 0) {
+      newBlocks = new BlockInfoStriped[0];
+    } else {
+      newBlocks = new BlockInfoStriped[n];
+      System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
+    }
+    // set new blocks
+    setBlocks(newBlocks);
+  }
+
+  void clear() {
+    this.blocks = null;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index b09463d..5462da7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -18,6 +18,7 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import static org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite.ID_UNSPECIFIED;
+import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.CURRENT_STATE_ID;
 import static org.apache.hadoop.hdfs.server.namenode.snapshot.Snapshot.NO_SNAPSHOT_ID;
 
@@ -37,12 +38,12 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiff;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileDiffList;
 import org.apache.hadoop.hdfs.server.namenode.snapshot.FileWithSnapshotFeature;
@@ -174,6 +175,31 @@ public class INodeFile extends INodeWithAdditionalFields
         && getXAttrFeature() == other.getXAttrFeature();
   }
 
+  /* Start of StripedBlock Feature */
+
+  public final FileWithStripedBlocksFeature getStripedBlocksFeature() {
+    return getFeature(FileWithStripedBlocksFeature.class);
+  }
+
+  public FileWithStripedBlocksFeature addStripedBlocksFeature() {
+    assert blocks == null || blocks.length == 0:
+        "The file contains contiguous blocks";
+    assert !isWithStripedBlocks();
+    this.setFileReplication((short) 0);
+    FileWithStripedBlocksFeature sb = new FileWithStripedBlocksFeature();
+    addFeature(sb);
+    return sb;
+  }
+
+  public boolean isWithStripedBlocks() {
+    return getStripedBlocksFeature() != null;
+  }
+
+  /** Used to make sure there is no contiguous block related info */
+  private boolean hasNoContiguousBlock() {
+    return (blocks == null || blocks.length == 0) && getFileReplication() == 0;
+  }
+
   /* Start of Under-Construction Feature */
 
   /**
@@ -208,7 +234,7 @@ public class INodeFile extends INodeWithAdditionalFields
         "file is no longer under construction");
     FileUnderConstructionFeature uc = getFileUnderConstructionFeature();
     if (uc != null) {
-      assertAllBlocksComplete();
+      assertAllBlocksComplete(getBlocks());
       removeFeature(uc);
       this.setModificationTime(mtime);
     }
@@ -216,37 +242,56 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Assert all blocks are complete. */
-  private void assertAllBlocksComplete() {
-    if (blocks == null) {
+  private void assertAllBlocksComplete(BlockInfo[] blks) {
+    if (blks == null) {
       return;
     }
-    for (int i = 0; i < blocks.length; i++) {
-      Preconditions.checkState(blocks[i].isComplete(), "Failed to finalize"
+    for (int i = 0; i < blks.length; i++) {
+      Preconditions.checkState(blks[i].isComplete(), "Failed to finalize"
           + " %s %s since blocks[%s] is non-complete, where blocks=%s.",
-          getClass().getSimpleName(), this, i, Arrays.asList(blocks));
+          getClass().getSimpleName(), this, i, Arrays.asList(blks));
     }
   }
 
+  /**
+   * Instead of adding a new block, this function is usually used while loading
+   * fsimage or converting the last block to UC/Complete.
+   */
   @Override // BlockCollection
-  public void setBlock(int index, BlockInfoContiguous blk) {
-    this.blocks[index] = blk;
+  public void setBlock(int index, BlockInfo blk) {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      assert blk instanceof BlockInfoContiguous;
+      this.blocks[index] = (BlockInfoContiguous) blk;
+    } else {
+      assert blk instanceof BlockInfoStriped;
+      assert hasNoContiguousBlock();
+      sb.setBlock(index, (BlockInfoStriped) blk);
+    }
   }
 
   @Override // BlockCollection, the file should be under construction
-  public BlockInfoContiguousUnderConstruction setLastBlock(
-      BlockInfoContiguous lastBlock, DatanodeStorageInfo[] locations)
-      throws IOException {
+  public void convertLastBlockToUC(BlockInfo lastBlock,
+      DatanodeStorageInfo[] locations) throws IOException {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
-
     if (numBlocks() == 0) {
       throw new IOException("Failed to set last block: File is empty.");
     }
-    BlockInfoContiguousUnderConstruction ucBlock =
-      lastBlock.convertToBlockUnderConstruction(
-          BlockUCState.UNDER_CONSTRUCTION, locations);
+
+    final BlockInfo ucBlock;
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      assert lastBlock instanceof BlockInfoContiguous;
+      ucBlock = ((BlockInfoContiguous) lastBlock)
+          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
+    } else {
+      assert hasNoContiguousBlock();
+      assert lastBlock instanceof BlockInfoStriped;
+      ucBlock = ((BlockInfoStriped) lastBlock)
+          .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
+    }
     setBlock(numBlocks() - 1, ucBlock);
-    return ucBlock;
   }
 
   /**
@@ -256,19 +301,25 @@ public class INodeFile extends INodeWithAdditionalFields
   boolean removeLastBlock(Block oldblock) {
     Preconditions.checkState(isUnderConstruction(),
         "file is no longer under construction");
-    if (blocks == null || blocks.length == 0) {
-      return false;
-    }
-    int size_1 = blocks.length - 1;
-    if (!blocks[size_1].equals(oldblock)) {
-      return false;
-    }
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      if (blocks == null || blocks.length == 0) {
+        return false;
+      }
+      int size_1 = blocks.length - 1;
+      if (!blocks[size_1].equals(oldblock)) {
+        return false;
+      }
 
-    //copy to a new list
-    BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
-    System.arraycopy(blocks, 0, newlist, 0, size_1);
-    setBlocks(newlist);
-    return true;
+      //copy to a new list
+      BlockInfoContiguous[] newlist = new BlockInfoContiguous[size_1];
+      System.arraycopy(blocks, 0, newlist, 0, size_1);
+      setContiguousBlocks(newlist);
+      return true;
+    } else {
+      assert hasNoContiguousBlock();
+      return sb.removeLastBlock(oldblock);
+    }
   }
 
   /* End of Under-Construction Feature */
@@ -369,13 +420,15 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   /** Set the replication factor of this file. */
-  public final void setFileReplication(short replication) {
+  private void setFileReplication(short replication) {
     header = HeaderFormat.REPLICATION.BITS.combine(replication, header);
   }
 
   /** Set the replication factor of this file. */
   public final INodeFile setFileReplication(short replication,
       int latestSnapshotId) throws QuotaExceededException {
+    Preconditions.checkState(!isWithStripedBlocks(),
+        "Cannot set replication to a file with striped blocks");
     recordModification(latestSnapshotId);
     setFileReplication(replication);
     return this;
@@ -413,42 +466,57 @@ public class INodeFile extends INodeWithAdditionalFields
     setStoragePolicyID(storagePolicyId);
   }
 
-  @Override
+  @Override // INodeFileAttributes
   public long getHeaderLong() {
     return header;
   }
 
-  /** @return the storagespace required for a full block. */
-  final long getPreferredBlockStoragespace() {
-    return getPreferredBlockSize() * getBlockReplication();
+  /** @return the blocks of the file. */
+  @Override // BlockCollection
+  public BlockInfo[] getBlocks() {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb != null) {
+      assert hasNoContiguousBlock();
+      return sb.getBlocks();
+    } else {
+      return this.blocks;
+    }
   }
 
-  /** @return the blocks of the file. */
-  @Override
-  public BlockInfoContiguous[] getBlocks() {
+  /** Used by snapshot diff */
+  public BlockInfoContiguous[] getContiguousBlocks() {
     return this.blocks;
   }
 
   /** @return blocks of the file corresponding to the snapshot. */
-  public BlockInfoContiguous[] getBlocks(int snapshot) {
-    if(snapshot == CURRENT_STATE_ID || getDiffs() == null)
+  public BlockInfo[] getBlocks(int snapshot) {
+    if (snapshot == CURRENT_STATE_ID || getDiffs() == null) {
       return getBlocks();
+    }
+    // find blocks stored in snapshot diffs (for truncate)
     FileDiff diff = getDiffs().getDiffById(snapshot);
-    BlockInfoContiguous[] snapshotBlocks =
-        diff == null ? getBlocks() : diff.getBlocks();
-    if(snapshotBlocks != null)
+    // note that currently FileDiff can only store contiguous blocks
+    BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks();
+    if (snapshotBlocks != null) {
       return snapshotBlocks;
+    }
     // Blocks are not in the current snapshot
     // Find next snapshot with blocks present or return current file blocks
     snapshotBlocks = getDiffs().findLaterSnapshotBlocks(snapshot);
     return (snapshotBlocks == null) ? getBlocks() : snapshotBlocks;
   }
 
-  void updateBlockCollection() {
-    if (blocks != null) {
+  /** Used during concat to update the BlockCollection for each block */
+  private void updateBlockCollection() {
+    if (blocks != null && blocks.length > 0) {
       for(BlockInfoContiguous b : blocks) {
         b.setBlockCollection(this);
       }
+    } else {
+      FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+      if (sb != null) {
+        sb.updateBlockCollection(this);
+      }
     }
   }
 
@@ -471,33 +539,33 @@ public class INodeFile extends INodeWithAdditionalFields
       size += in.blocks.length;
     }
 
-    setBlocks(newlist);
+    setContiguousBlocks(newlist);
     updateBlockCollection();
   }
   
   /**
-   * add a block to the block list
+   * add a contiguous block to the block list
    */
   void addBlock(BlockInfoContiguous newblock) {
     if (this.blocks == null) {
-      this.setBlocks(new BlockInfoContiguous[]{newblock});
+      this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
     } else {
       int size = this.blocks.length;
       BlockInfoContiguous[] newlist = new BlockInfoContiguous[size + 1];
       System.arraycopy(this.blocks, 0, newlist, 0, size);
       newlist[size] = newblock;
-      this.setBlocks(newlist);
+      this.setContiguousBlocks(newlist);
     }
   }
 
   /** Set the blocks. */
-  public void setBlocks(BlockInfoContiguous[] blocks) {
+  public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
     this.blocks = blocks;
   }
 
   @Override
-  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps, final int snapshot,
-                                  int priorSnapshotId,
+  public QuotaCounts cleanSubtree(BlockStoragePolicySuite bsps,
+      final int snapshot, int priorSnapshotId,
       final BlocksMapUpdateInfo collectedBlocks,
       final List<INode> removedINodes) {
     FileWithSnapshotFeature sf = getFileWithSnapshotFeature();
@@ -527,13 +595,19 @@ public class INodeFile extends INodeWithAdditionalFields
   @Override
   public void destroyAndCollectBlocks(BlockStoragePolicySuite bsps,
       BlocksMapUpdateInfo collectedBlocks, final List<INode> removedINodes) {
-    if (blocks != null && collectedBlocks != null) {
-      for (BlockInfoContiguous blk : blocks) {
+    BlockInfo[] blks = getBlocks();
+    if (blks != null && collectedBlocks != null) {
+      for (BlockInfo blk : blks) {
         collectedBlocks.addDeleteBlock(blk);
         blk.setBlockCollection(null);
       }
     }
-    setBlocks(null);
+    setContiguousBlocks(null);
+
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb != null) {
+      sb.clear();
+    }
     if (getAclFeature() != null) {
       AclStorage.removeAclFeature(getAclFeature());
     }
@@ -726,7 +800,7 @@ public class INodeFile extends INodeWithAdditionalFields
       size += block.getNumBytes();
     }
     // check if the last block is under construction
-    BlockInfoContiguous lastBlock = getLastBlock();
+    BlockInfo lastBlock = getLastBlock();
     if(lastBlock != null &&
         lastBlock instanceof BlockInfoContiguousUnderConstruction) {
       size += getPreferredBlockSize() - lastBlock.getNumBytes();
@@ -734,15 +808,6 @@ public class INodeFile extends INodeWithAdditionalFields
     return size;
   }
 
-  public final long storagespaceConsumed(int lastSnapshotId) {
-    if (lastSnapshotId != CURRENT_STATE_ID) {
-      return computeFileSize(lastSnapshotId)
-        * getFileReplication(lastSnapshotId);
-    } else {
-      return storagespaceConsumed();
-    }
-  }
-
   public final short getReplication(int lastSnapshotId) {
     if (lastSnapshotId != CURRENT_STATE_ID) {
       return getFileReplication(lastSnapshotId);
@@ -762,21 +827,33 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * Return the penultimate allocated block for this file.
    */
-  BlockInfoContiguous getPenultimateBlock() {
-    if (blocks == null || blocks.length <= 1) {
-      return null;
-    }
-    return blocks[blocks.length - 2];
+  BlockInfo getPenultimateBlock() {
+    BlockInfo[] blks = getBlocks();
+    return (blks == null || blks.length <= 1) ?
+        null : blks[blks.length - 2];
   }
 
   @Override
-  public BlockInfoContiguous getLastBlock() {
-    return blocks == null || blocks.length == 0? null: blocks[blocks.length-1];
+  public BlockInfo getLastBlock() {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      return blocks == null || blocks.length == 0 ?
+          null : blocks[blocks.length - 1];
+    } else {
+      assert hasNoContiguousBlock();
+      return sb.getLastBlock();
+    }
   }
 
   @Override
   public int numBlocks() {
-    return blocks == null ? 0 : blocks.length;
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      return blocks == null ? 0 : blocks.length;
+    } else {
+      assert hasNoContiguousBlock();
+      return sb.numBlocks();
+    }
   }
 
   @VisibleForTesting
@@ -788,6 +865,7 @@ public class INodeFile extends INodeWithAdditionalFields
     // only compare the first block
     out.print(", blocks=");
     out.print(blocks == null || blocks.length == 0? null: blocks[0]);
+    // TODO print striped blocks
     out.println();
   }
 
@@ -797,9 +875,10 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   public long collectBlocksBeyondMax(final long max,
       final BlocksMapUpdateInfo collectedBlocks) {
-    final BlockInfoContiguous[] oldBlocks = getBlocks();
-    if (oldBlocks == null)
+    final BlockInfo[] oldBlocks = getBlocks();
+    if (oldBlocks == null) {
       return 0;
+    }
     // find the minimum n such that the size of the first n blocks > max
     int n = 0;
     long size = 0;
@@ -860,21 +939,36 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   void truncateBlocksTo(int n) {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      truncateContiguousBlocks(n);
+    } else {
+      sb.truncateStripedBlocks(n);
+    }
+  }
+
+  private void truncateContiguousBlocks(int n) {
     final BlockInfoContiguous[] newBlocks;
     if (n == 0) {
       newBlocks = BlockInfoContiguous.EMPTY_ARRAY;
     } else {
       newBlocks = new BlockInfoContiguous[n];
-      System.arraycopy(getBlocks(), 0, newBlocks, 0, n);
+      System.arraycopy(blocks, 0, newBlocks, 0, n);
     }
     // set new blocks
-    setBlocks(newBlocks);
+    setContiguousBlocks(newBlocks);
   }
 
+  /**
+   * This function is only called when block list is stored in snapshot
+   * diffs. Note that this can only happen when truncation happens with
+   * snapshots. Since we do not support truncation with striped blocks,
+   * we only need to handle contiguous blocks here.
+   */
   public void collectBlocksBeyondSnapshot(BlockInfoContiguous[] snapshotBlocks,
                                           BlocksMapUpdateInfo collectedBlocks) {
-    BlockInfoContiguous[] oldBlocks = getBlocks();
-    if(snapshotBlocks == null || oldBlocks == null)
+    BlockInfoContiguous[] oldBlocks = this.blocks;
+    if (snapshotBlocks == null || oldBlocks == null)
       return;
     // Skip blocks in common between the file and the snapshot
     int n = 0;
@@ -882,7 +976,7 @@ public class INodeFile extends INodeWithAdditionalFields
           oldBlocks[n] == snapshotBlocks[n]) {
       n++;
     }
-    truncateBlocksTo(n);
+    truncateContiguousBlocks(n);
     // Collect the remaining blocks of the file
     while(n < oldBlocks.length) {
       collectedBlocks.addDeleteBlock(oldBlocks[n++]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
index 9ce8ebc..6cd11b5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/LeaseManager.java
@@ -37,7 +37,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.UnresolvedLinkException;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.util.Daemon;
 
@@ -120,10 +120,10 @@ public class LeaseManager {
         } catch (UnresolvedLinkException e) {
           throw new AssertionError("Lease files should reside on this FS");
         }
-        BlockInfoContiguous[] blocks = cons.getBlocks();
+        BlockInfo[] blocks = cons.getBlocks();
         if(blocks == null)
           continue;
-        for(BlockInfoContiguous b : blocks) {
+        for(BlockInfo b : blocks) {
           if(!b.isComplete())
             numUCBlocks++;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
index be412a0..5f5a1e0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NamenodeFsck.java
@@ -63,6 +63,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactor
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.DataEncryptionKey;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy;
@@ -235,8 +236,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory {
       //get blockInfo
       Block block = new Block(Block.getBlockId(blockId));
       //find which file this block belongs to
-      BlockInfoContiguous blockInfo = namenode.getNamesystem()
-          .getStoredBlock(block);
+      BlockInfo blockInfo = namenode.getNamesystem().getStoredBlock(block);
       if(blockInfo == null) {
         out.println("Block "+ blockId +" " + NONEXISTENT_STATUS);
         LOG.warn("Block "+ blockId + " " + NONEXISTENT_STATUS);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
index 3442e7b..4695c3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/Namesystem.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory;
 import org.apache.hadoop.hdfs.util.RwLock;
@@ -45,5 +46,5 @@ public interface Namesystem extends RwLock, SafeMode {
 
   public void checkOperation(OperationCategory read) throws StandbyException;
 
-  public boolean isInSnapshot(BlockInfoContiguousUnderConstruction blockUC);
+  public boolean isInSnapshot(BlockCollection bc);
 }
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
index 87b370a..74baec5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FSImageFormatPBSnapshot.java
@@ -239,15 +239,16 @@ public class FSImageFormatPBSnapshot {
         FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null,
             pbf.getFileSize());
         List<BlockProto> bpl = pbf.getBlocksList();
-        // TODO: also persist striped blocks
+        // in file diff there can only be contiguous blocks
         BlockInfoContiguous[] blocks = new BlockInfoContiguous[bpl.size()];
         for(int j = 0, e = bpl.size(); j < e; ++j) {
           Block blk = PBHelper.convert(bpl.get(j));
           BlockInfoContiguous storedBlock =
               (BlockInfoContiguous) fsn.getBlockManager().getStoredBlock(blk);
           if(storedBlock == null) {
-            storedBlock = fsn.getBlockManager().addBlockCollection(
-                new BlockInfoContiguous(blk, copy.getFileReplication()), file);
+            storedBlock = (BlockInfoContiguous) fsn.getBlockManager()
+                .addBlockCollection(new BlockInfoContiguous(blk,
+                    copy.getFileReplication()), file);
           }
           blocks[j] = storedBlock;
         }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
index 5c9e121..a1263c5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/snapshot/FileDiffList.java
@@ -54,8 +54,11 @@ public class FileDiffList extends
       INodeFileAttributes snapshotCopy, boolean withBlocks) {
     final FileDiff diff =
         super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy);
-    if(withBlocks)  // Store blocks if this is the first update
-      diff.setBlocks(iNodeFile.getBlocks());
+    if (withBlocks) {  // Store blocks if this is the first update
+      BlockInfoContiguous[] blks = iNodeFile.getContiguousBlocks();
+      assert blks != null;
+      diff.setBlocks(blks);
+    }
   }
 
   public BlockInfoContiguous[] findEarlierSnapshotBlocks(int snapshotId) {
@@ -118,7 +121,7 @@ public class FileDiffList extends
         (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks());
     // Find later snapshot (or file itself) with blocks
     BlockInfoContiguous[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId());
-    laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks;
+    laterBlocks = (laterBlocks == null) ? file.getContiguousBlocks() : laterBlocks;
     // Skip blocks, which belong to either the earlier or the later lists
     int i = 0;
     for(; i < removedBlocks.length; i++) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index 3bd1d91..b6fd033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -91,6 +91,10 @@ message INodeSection {
     optional string clientMachine = 2;
   }
 
+  message StripedBlocksFeature {
+    repeated StripedBlockProto blocks = 1;
+  }
+
   message AclFeatureProto {
     /**
      * An ACL entry is represented by a 32-bit integer in Big Endian
@@ -139,6 +143,7 @@ message INodeSection {
     optional AclFeatureProto acl = 8;
     optional XAttrFeatureProto xAttrs = 9;
     optional uint32 storagePolicyID = 10;
+    optional StripedBlocksFeature stripedBlocks = 11;
   }
 
   message QuotaByStorageTypeEntryProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 86fb462..d281a64 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -491,6 +491,16 @@ message BlockProto {
 }
 
 /**
+ * Striped block information. Besides the basic information for a block,
+ * it also contains the number of data/parity blocks.
+ */
+message StripedBlockProto {
+  required BlockProto block = 1;
+  optional uint32 dataBlockNum = 2;
+  optional uint32 parityBlockNum = 3;
+}
+
+/**
  * Block and datanodes where is it located
  */
 message BlockWithLocationsProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 0d472b6..c3b5b44 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -105,6 +105,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
@@ -1601,7 +1602,7 @@ public class DFSTestUtil {
   public static DatanodeDescriptor getExpectedPrimaryNode(NameNode nn,
       ExtendedBlock blk) {
     FSNamesystem fsn = nn.getNamesystem();
-    BlockInfoContiguous storedBlock = fsn.getStoredBlock(blk.getLocalBlock());
+    BlockInfo storedBlock = fsn.getStoredBlock(blk.getLocalBlock());
     assertTrue("Block " + blk + " should be under construction, " +
         "got: " + storedBlock,
         storedBlock instanceof BlockInfoContiguousUnderConstruction);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
index fd28ded..6cd0d1c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestReplicationPolicy.java
@@ -1240,8 +1240,8 @@ public class TestReplicationPolicy {
         (DatanodeStorageInfo.AddBlockResult.ADDED);
     ucBlock.addStorage(storage, ucBlock);
 
-    when(mbc.setLastBlock((BlockInfoContiguous) any(), (DatanodeStorageInfo[]) any()))
-    .thenReturn(ucBlock);
+    BlockInfo lastBlk = mbc.getLastBlock();
+    when(mbc.getLastBlock()).thenReturn(lastBlk, ucBlock);
 
     bm.convertLastBlockToUnderConstruction(mbc, 0L);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
index a417c3d..301ee25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlock.java
@@ -31,7 +31,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.junit.After;
 import org.junit.Before;
@@ -87,21 +87,21 @@ public class TestAddBlock {
     
     // check file1
     INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
+    BlockInfo[] file1Blocks = file1Node.getBlocks();
     assertEquals(1, file1Blocks.length);
     assertEquals(BLOCKSIZE - 1, file1Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file1Blocks[0].getBlockUCState());
     
     // check file2
     INodeFile file2Node = fsdir.getINode4Write(file2.toString()).asFile();
-    BlockInfoContiguous[] file2Blocks = file2Node.getBlocks();
+    BlockInfo[] file2Blocks = file2Node.getBlocks();
     assertEquals(1, file2Blocks.length);
     assertEquals(BLOCKSIZE, file2Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file2Blocks[0].getBlockUCState());
     
     // check file3
     INodeFile file3Node = fsdir.getINode4Write(file3.toString()).asFile();
-    BlockInfoContiguous[] file3Blocks = file3Node.getBlocks();
+    BlockInfo[] file3Blocks = file3Node.getBlocks();
     assertEquals(2, file3Blocks.length);
     assertEquals(BLOCKSIZE, file3Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file3Blocks[0].getBlockUCState());
@@ -110,7 +110,7 @@ public class TestAddBlock {
     
     // check file4
     INodeFile file4Node = fsdir.getINode4Write(file4.toString()).asFile();
-    BlockInfoContiguous[] file4Blocks = file4Node.getBlocks();
+    BlockInfo[] file4Blocks = file4Node.getBlocks();
     assertEquals(2, file4Blocks.length);
     assertEquals(BLOCKSIZE, file4Blocks[0].getNumBytes());
     assertEquals(BlockUCState.COMPLETE, file4Blocks[0].getBlockUCState());
@@ -141,7 +141,7 @@ public class TestAddBlock {
       FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
       
       INodeFile fileNode = fsdir.getINode4Write(file1.toString()).asFile();
-      BlockInfoContiguous[] fileBlocks = fileNode.getBlocks();
+      BlockInfo[] fileBlocks = fileNode.getBlocks();
       assertEquals(2, fileBlocks.length);
       assertEquals(BLOCKSIZE, fileBlocks[0].getNumBytes());
       assertEquals(BlockUCState.COMPLETE, fileBlocks[0].getBlockUCState());

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
index 06dfade..a2ef7b2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
@@ -26,6 +26,7 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.junit.After;
 import org.junit.Before;
@@ -75,7 +76,7 @@ public class TestAddBlockgroup {
     final Path file1 = new Path("/file1");
     DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L);
     INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfoContiguous[] file1Blocks = file1Node.getBlocks();
+    BlockInfo[] file1Blocks = file1Node.getBlocks();
     assertEquals(2, file1Blocks.length);
     assertEquals(GROUP_SIZE, file1Blocks[0].numNodes());
     assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP,

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
index 1fbe160..f372bec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestBlockUnderConstruction.java
@@ -35,7 +35,7 @@ import org.apache.hadoop.hdfs.TestFileCreation;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols;
@@ -91,12 +91,12 @@ public class TestBlockUnderConstruction {
         " isUnderConstruction = " + inode.isUnderConstruction() +
         " expected to be " + isFileOpen,
         inode.isUnderConstruction() == isFileOpen);
-    BlockInfoContiguous[] blocks = inode.getBlocks();
+    BlockInfo[] blocks = inode.getBlocks();
     assertTrue("File does not have blocks: " + inode.toString(),
         blocks != null && blocks.length > 0);
     
     int idx = 0;
-    BlockInfoContiguous curBlock;
+    BlockInfo curBlock;
     // all blocks but the last two should be regular blocks
     for(; idx < blocks.length - 2; idx++) {
       curBlock = blocks[idx];

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 7b9ea93..913e0a7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -24,6 +24,7 @@ import java.io.File;
 import java.io.IOException;
 import java.util.EnumSet;
 
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -39,7 +40,6 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.client.HdfsDataOutputStream.SyncFlag;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager.Lease;
@@ -105,7 +105,7 @@ public class TestFSImage {
       INodeFile file2Node = fsn.dir.getINode4Write(file2.toString()).asFile();
       assertEquals("hello".length(), file2Node.computeFileSize());
       assertTrue(file2Node.isUnderConstruction());
-      BlockInfoContiguous[] blks = file2Node.getBlocks();
+      BlockInfo[] blks = file2Node.getBlocks();
       assertEquals(1, blks.length);
       assertEquals(BlockUCState.UNDER_CONSTRUCTION, blks[0].getBlockUCState());
       // check lease manager

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
index fbcc73f..3e27107 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFileTruncate.java
@@ -55,6 +55,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
@@ -1036,7 +1037,8 @@ public class TestFileTruncate {
     iip = fsn.getFSDirectory().getINodesInPath(src, true);
     file = iip.getLastINode().asFile();
     file.recordModification(iip.getLatestSnapshotId(), true);
-    assertThat(file.isBlockInLatestSnapshot(file.getLastBlock()), is(true));
+    assertThat(file.isBlockInLatestSnapshot(
+        (BlockInfoContiguous) file.getLastBlock()), is(true));
     initialGenStamp = file.getLastBlock().getGenerationStamp();
     // Test that prepareFileForTruncate sets up copy-on-write truncate
     fsn.writeLock();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
index 68b7e38..5579cb6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFsck.java
@@ -78,7 +78,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -802,7 +802,7 @@ public class TestFsck {
       // intentionally corrupt NN data structure
       INodeFile node = (INodeFile) cluster.getNamesystem().dir.getINode
           (fileName, true);
-      final BlockInfoContiguous[] blocks = node.getBlocks();
+      final BlockInfo[] blocks = node.getBlocks();
       assertEquals(blocks.length, 1);
       blocks[0].setNumBytes(-1L);  // set the block length to be negative
       

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
index 85072d1..7bffb33 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotBlocksMap.java
@@ -36,7 +36,7 @@ import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -108,14 +108,14 @@ public class TestSnapshotBlocksMap {
      final FSDirectory dir, final BlockManager blkManager) throws Exception {
     final INodeFile file = INodeFile.valueOf(dir.getINode(path), path);
     assertEquals(numBlocks, file.getBlocks().length);
-    for(BlockInfoContiguous b : file.getBlocks()) {
+    for(BlockInfo b : file.getBlocks()) {
       assertBlockCollection(blkManager, file, b);
     }
     return file;
   }
 
   static void assertBlockCollection(final BlockManager blkManager,
-      final INodeFile file, final BlockInfoContiguous b) {
+      final INodeFile file, final BlockInfo b) {
     Assert.assertSame(b, blkManager.getStoredBlock(b));
     Assert.assertSame(file, blkManager.getBlockCollection(b));
     Assert.assertSame(file, b.getBlockCollection());
@@ -146,10 +146,10 @@ public class TestSnapshotBlocksMap {
     {
       final INodeFile f2 = assertBlockCollection(file2.toString(), 3, fsdir,
           blockmanager);
-      BlockInfoContiguous[] blocks = f2.getBlocks();
+      BlockInfo[] blocks = f2.getBlocks();
       hdfs.delete(sub2, true);
       // The INode should have been removed from the blocksMap
-      for(BlockInfoContiguous b : blocks) {
+      for(BlockInfo b : blocks) {
         assertNull(blockmanager.getBlockCollection(b));
       }
     }
@@ -177,7 +177,7 @@ public class TestSnapshotBlocksMap {
     // Check the block information for file0
     final INodeFile f0 = assertBlockCollection(file0.toString(), 4, fsdir,
         blockmanager);
-    BlockInfoContiguous[] blocks0 = f0.getBlocks();
+    BlockInfo[] blocks0 = f0.getBlocks();
     
     // Also check the block information for snapshot of file0
     Path snapshotFile0 = SnapshotTestHelper.getSnapshotPath(sub1, "s0",
@@ -187,7 +187,7 @@ public class TestSnapshotBlocksMap {
     // Delete file0
     hdfs.delete(file0, true);
     // Make sure the blocks of file0 is still in blocksMap
-    for(BlockInfoContiguous b : blocks0) {
+    for(BlockInfo b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@@ -201,7 +201,7 @@ public class TestSnapshotBlocksMap {
     hdfs.deleteSnapshot(sub1, "s1");
 
     // Make sure the first block of file0 is still in blocksMap
-    for(BlockInfoContiguous b : blocks0) {
+    for(BlockInfo b : blocks0) {
       assertNotNull(blockmanager.getBlockCollection(b));
     }
     assertBlockCollection(snapshotFile0.toString(), 4, fsdir, blockmanager);
@@ -293,7 +293,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     assertEquals(BLOCKSIZE, blks[0].getNumBytes());
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
@@ -331,7 +331,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
     cluster.getNameNodeRpc()
@@ -370,7 +370,7 @@ public class TestSnapshotBlocksMap {
     hdfs.append(bar);
 
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     ExtendedBlock previous = new ExtendedBlock(fsn.getBlockPoolId(), blks[0]);
     cluster.getNameNodeRpc()
@@ -421,7 +421,7 @@ public class TestSnapshotBlocksMap {
     out.write(testData);
     out.close();
     INodeFile barNode = fsdir.getINode4Write(bar.toString()).asFile();
-    BlockInfoContiguous[] blks = barNode.getBlocks();
+    BlockInfo[] blks = barNode.getBlocks();
     assertEquals(1, blks.length);
     assertEquals(testData.length, blks[0].getNumBytes());
     

http://git-wip-us.apache.org/repos/asf/hadoop/blob/5608b43b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
index a679183..452ff3f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/TestSnapshotDeletion.java
@@ -42,7 +42,7 @@ import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSNNTopology;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
@@ -260,12 +260,12 @@ public class TestSnapshotDeletion {
     DFSTestUtil.createFile(hdfs, tempFile, BLOCKSIZE, REPLICATION, seed);
     final INodeFile temp = TestSnapshotBlocksMap.assertBlockCollection(
         tempFile.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks = temp.getBlocks();
+    BlockInfo[] blocks = temp.getBlocks();
     hdfs.delete(tempDir, true);
     // check dir's quota usage
     checkQuotaUsageComputation(dir, 8, BLOCKSIZE * REPLICATION * 3);
     // check blocks of tempFile
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -342,7 +342,7 @@ public class TestSnapshotDeletion {
     // while deletion, we add diff for subsub and metaChangeFile1, and remove
     // newFile
     checkQuotaUsageComputation(dir, 9L, BLOCKSIZE * REPLICATION * 4);
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -479,7 +479,7 @@ public class TestSnapshotDeletion {
     
     final INodeFile toDeleteFileNode = TestSnapshotBlocksMap
         .assertBlockCollection(toDeleteFile.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks = toDeleteFileNode.getBlocks();
+    BlockInfo[] blocks = toDeleteFileNode.getBlocks();
     
     // create snapshot s0 on dir
     SnapshotTestHelper.createSnapshot(hdfs, dir, "s0");
@@ -505,7 +505,7 @@ public class TestSnapshotDeletion {
     // metaChangeDir's diff, dir's diff. diskspace: remove toDeleteFile, and 
     // metaChangeFile's replication factor decreases
     checkQuotaUsageComputation(dir, 6, 2 * BLOCKSIZE * REPLICATION - BLOCKSIZE);
-    for (BlockInfoContiguous b : blocks) {
+    for (BlockInfo b : blocks) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     
@@ -799,7 +799,7 @@ public class TestSnapshotDeletion {
     FileStatus statusBeforeDeletion13 = hdfs.getFileStatus(file13_s1);
     INodeFile file14Node = TestSnapshotBlocksMap.assertBlockCollection(
         file14_s2.toString(), 1, fsdir, blockmanager);
-    BlockInfoContiguous[] blocks_14 = file14Node.getBlocks();
+    BlockInfo[] blocks_14 = file14Node.getBlocks();
     TestSnapshotBlocksMap.assertBlockCollection(file15_s2.toString(), 1, fsdir,
         blockmanager);
     
@@ -836,7 +836,7 @@ public class TestSnapshotDeletion {
         modDirStr + "file15");
     assertFalse(hdfs.exists(file14_s1));
     assertFalse(hdfs.exists(file15_s1));
-    for (BlockInfoContiguous b : blocks_14) {
+    for (BlockInfo b : blocks_14) {
       assertNull(blockmanager.getBlockCollection(b));
     }
     


[41/50] [abbrv] hadoop git commit: HDFS-7782. Erasure coding: pread from files in striped layout. Contributed by Zhe Zhang and Jing Zhao

Posted by zh...@apache.org.
HDFS-7782. Erasure coding: pread from files in striped layout. Contributed by Zhe Zhang and Jing Zhao


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/e67086dc
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/e67086dc
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/e67086dc

Branch: refs/heads/HDFS-7285
Commit: e67086dce29b613036158cb9359a30f56154c87c
Parents: 42cb9d8
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Apr 7 11:20:13 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:47 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/DFSClient.java  |  55 +++
 .../org/apache/hadoop/hdfs/DFSConfigKeys.java   |   8 +-
 .../org/apache/hadoop/hdfs/DFSInputStream.java  |  79 +++-
 .../hadoop/hdfs/DFSStripedInputStream.java      | 367 +++++++++++++++++++
 .../hadoop/hdfs/protocol/HdfsConstants.java     |   2 +-
 .../hadoop/hdfs/protocol/LocatedBlock.java      |   4 +
 .../hdfs/protocol/LocatedStripedBlock.java      |   5 +
 .../blockmanagement/BlockInfoStriped.java       |   6 +-
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  92 ++++-
 .../apache/hadoop/hdfs/TestReadStripedFile.java | 307 ++++++++++++++++
 .../namenode/TestRecoverStripedBlocks.java      |  88 +----
 11 files changed, 900 insertions(+), 113 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index cd06e09..2ef1d36 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -236,6 +236,7 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
   private static final DFSHedgedReadMetrics HEDGED_READ_METRIC =
       new DFSHedgedReadMetrics();
   private static ThreadPoolExecutor HEDGED_READ_THREAD_POOL;
+  private static volatile ThreadPoolExecutor STRIPED_READ_THREAD_POOL;
   private final Sampler<?> traceSampler;
 
   public DfsClientConf getConf() {
@@ -376,6 +377,19 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     if (numThreads > 0) {
       this.initThreadsNumForHedgedReads(numThreads);
     }
+    numThreads = conf.getInt(
+        DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE,
+        DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE);
+    if (numThreads <= 0) {
+      LOG.warn("The value of "
+          + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE
+          + " must be greater than 0. The current setting is " + numThreads
+          + ". Reset it to the default value "
+          + DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE);
+      numThreads =
+          DFSConfigKeys.DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE;
+    }
+    this.initThreadsNumForStripedReads(numThreads);
     this.saslClient = new SaslDataTransferClient(
       conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf),
       TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth);
@@ -3148,6 +3162,43 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  /**
+   * Create thread pool for parallel reading in striped layout,
+   * STRIPED_READ_THREAD_POOL, if it does not already exist.
+   * @param num Number of threads for striped reads thread pool.
+   */
+  private void initThreadsNumForStripedReads(int num) {
+    assert num > 0;
+    if (STRIPED_READ_THREAD_POOL != null) {
+      return;
+    }
+    synchronized (DFSClient.class) {
+      if (STRIPED_READ_THREAD_POOL == null) {
+        STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60,
+            TimeUnit.SECONDS, new SynchronousQueue<Runnable>(),
+            new Daemon.DaemonFactory() {
+          private final AtomicInteger threadIndex = new AtomicInteger(0);
+
+          @Override
+          public Thread newThread(Runnable r) {
+            Thread t = super.newThread(r);
+            t.setName("stripedRead-" + threadIndex.getAndIncrement());
+            return t;
+          }
+        }, new ThreadPoolExecutor.CallerRunsPolicy() {
+          @Override
+          public void rejectedExecution(Runnable runnable, ThreadPoolExecutor e) {
+            LOG.info("Execution for striped reading rejected, "
+                + "Executing in current thread");
+            // will run in the current thread
+            super.rejectedExecution(runnable, e);
+          }
+        });
+        STRIPED_READ_THREAD_POOL.allowCoreThreadTimeOut(true);
+      }
+    }
+  }
+
   long getHedgedReadTimeout() {
     return this.hedgedReadThresholdMillis;
   }
@@ -3161,6 +3212,10 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     return HEDGED_READ_THREAD_POOL;
   }
 
+  ThreadPoolExecutor getStripedReadsThreadPool() {
+    return STRIPED_READ_THREAD_POOL;
+  }
+
   boolean isHedgedReadsEnabled() {
     return (HEDGED_READ_THREAD_POOL != null) &&
       HEDGED_READ_THREAD_POOL.getMaximumPoolSize() > 0;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
index f3ccc3e..7755272 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSConfigKeys.java
@@ -645,7 +645,13 @@ public class DFSConfigKeys extends CommonConfigurationKeys {
       "dfs.namenode.reject-unresolved-dn-topology-mapping";
   public static final boolean DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_DEFAULT =
       false;
-  
+
+  public static final String DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_SIZE =
+      "dfs.client.striped.read.threadpool.size";
+  // With default 3+2 schema, each normal read could span 3 DNs. So this
+  // default value accommodates 6 read streams
+  public static final int DFS_CLIENT_STRIPED_READ_THREADPOOL_MAX_DEFAULT_SIZE = 18;
+
   // Slow io warning log threshold settings for dfsclient and datanode.
   public static final String DFS_DATANODE_SLOW_IO_WARNING_THRESHOLD_KEY =
     "dfs.datanode.slow.io.warning.threshold.ms";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
index dd0f6fe..e94c41a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java
@@ -43,6 +43,7 @@ import java.util.concurrent.Future;
 import java.util.concurrent.TimeUnit;
 import java.util.concurrent.atomic.AtomicBoolean;
 
+import com.google.common.base.Preconditions;
 import org.apache.commons.io.IOUtils;
 import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.fs.ByteBufferReadable;
@@ -93,7 +94,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   @VisibleForTesting
   public static boolean tcpReadsDisabledForTesting = false;
   private long hedgedReadOpsLoopNumForTesting = 0;
-  private final DFSClient dfsClient;
+  protected final DFSClient dfsClient;
   private AtomicBoolean closed = new AtomicBoolean(false);
   private final String src;
   private final boolean verifyChecksum;
@@ -440,7 +441,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * @return located block
    * @throws IOException
    */
-  private LocatedBlock getBlockAt(long offset) throws IOException {
+  protected LocatedBlock getBlockAt(long offset) throws IOException {
     synchronized(infoLock) {
       assert (locatedBlocks != null) : "locatedBlocks is null";
 
@@ -705,7 +706,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
    * Wraps different possible read implementations so that readBuffer can be
    * strategy-agnostic.
    */
-  private interface ReaderStrategy {
+  interface ReaderStrategy {
     public int doRead(BlockReader blockReader, int off, int len)
         throws ChecksumException, IOException;
   }
@@ -1048,7 +1049,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     return errMsgr.toString();
   }
 
-  private void fetchBlockByteRange(long blockStartOffset, long start, long end,
+  protected void fetchBlockByteRange(long blockStartOffset, long start, long end,
       byte[] buf, int offset,
       Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
@@ -1090,13 +1091,42 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
     };
   }
 
+  /**
+   * Used when reading contiguous blocks
+   */
   private void actualGetFromOneDataNode(final DNAddrPair datanode,
       long blockStartOffset, final long start, final long end, byte[] buf,
       int offset, Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
       throws IOException {
+    final int length = (int) (end - start + 1);
+    actualGetFromOneDataNode(datanode, block, start, end, buf,
+        new int[]{offset}, new int[]{length}, corruptedBlockMap);
+  }
+
+  /**
+   * Read data from one DataNode.
+   * @param datanode the datanode from which to read data
+   * @param block the block to read
+   * @param startInBlk the startInBlk offset of the block
+   * @param endInBlk the endInBlk offset of the block
+   * @param buf the given byte array into which the data is read
+   * @param offsets the data may be read into multiple segments of the buf
+   *                (when reading a striped block). this array indicates the
+   *                offset of each buf segment.
+   * @param lengths the length of each buf segment
+   * @param corruptedBlockMap map recording list of datanodes with corrupted
+   *                          block replica
+   */
+  void actualGetFromOneDataNode(final DNAddrPair datanode,
+      LocatedBlock block, final long startInBlk, final long endInBlk,
+      byte[] buf, int[] offsets, int[] lengths,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
     DFSClientFaultInjector.get().startFetchFromDatanode();
     int refetchToken = 1; // only need to get a new access token once
     int refetchEncryptionKey = 1; // only need to get a new encryption key once
+    final int len = (int) (endInBlk - startInBlk + 1);
+    checkReadPortions(offsets, lengths, len);
 
     while (true) {
       // cached block locations may have been updated by chooseDataNode()
@@ -1117,7 +1147,6 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       try {
         DFSClientFaultInjector.get().fetchFromDatanodeException();
         Token<BlockTokenIdentifier> blockToken = block.getBlockToken();
-        int len = (int) (end - start + 1);
         reader = new BlockReaderFactory(dfsClient.getConf()).
             setInetSocketAddress(targetAddr).
             setRemotePeerFactory(dfsClient).
@@ -1126,7 +1155,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             setFileName(src).
             setBlock(block.getBlock()).
             setBlockToken(blockToken).
-            setStartOffset(start).
+            setStartOffset(startInBlk).
             setVerifyChecksum(verifyChecksum).
             setClientName(dfsClient.clientName).
             setLength(len).
@@ -1136,12 +1165,14 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
             setUserGroupInformation(dfsClient.ugi).
             setConfiguration(dfsClient.getConfiguration()).
             build();
-        int nread = reader.readAll(buf, offset, len);
-        updateReadStatistics(readStatistics, nread, reader);
+        for (int i = 0; i < offsets.length; i++) {
+          int nread = reader.readAll(buf, offsets[i], lengths[i]);
+          updateReadStatistics(readStatistics, nread, reader);
 
-        if (nread != len) {
-          throw new IOException("truncated return from reader.read(): " +
-                                "excpected " + len + ", got " + nread);
+          if (nread != len) {
+            throw new IOException("truncated return from reader.read(): " +
+                "excpected " + len + ", got " + nread);
+          }
         }
         DFSClientFaultInjector.get().readFromDatanodeDelay();
         return;
@@ -1187,6 +1218,25 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /**
+   * This method verifies that the read portions are valid and do not overlap
+   * with each other.
+   */
+  private void checkReadPortions(int[] offsets, int[] lengths, int totalLen) {
+    Preconditions.checkArgument(offsets.length == lengths.length &&
+        offsets.length > 0);
+    int sum = 0;
+    for (int i = 0; i < lengths.length; i++) {
+      if (i > 0) {
+        int gap = offsets[i] - offsets[i - 1];
+        // make sure read portions do not overlap with each other
+        Preconditions.checkArgument(gap >= lengths[i - 1]);
+      }
+      sum += lengths[i];
+    }
+    Preconditions.checkArgument(sum == totalLen);
+  }
+
+  /**
    * Like {@link #fetchBlockByteRange(LocatedBlock, long, long, byte[],
    * int, Map)} except we start up a second, parallel, 'hedged' read
    * if the first read is taking longer than configured amount of
@@ -1407,10 +1457,9 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
       long targetStart = position - blk.getStartOffset();
       long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart);
       try {
-        if (dfsClient.isHedgedReadsEnabled()) {
+        if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) {
           hedgedFetchBlockByteRange(blk.getStartOffset(), targetStart,
-              targetStart + bytesToRead - 1, buffer, offset,
-              corruptedBlockMap);
+              targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap);
         } else {
           fetchBlockByteRange(blk.getStartOffset(), targetStart,
               targetStart + bytesToRead - 1, buffer, offset,
@@ -1606,7 +1655,7 @@ implements ByteBufferReadable, CanSetDropBehind, CanSetReadahead,
   }
 
   /** Utility class to encapsulate data node info and its address. */
-  private static final class DNAddrPair {
+  static final class DNAddrPair {
     final DatanodeInfo info;
     final InetSocketAddress addr;
     final StorageType storageType;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
new file mode 100644
index 0000000..077b0f8
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java
@@ -0,0 +1,367 @@
+/**
+ * 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 com.google.common.annotations.VisibleForTesting;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException;
+import org.apache.hadoop.net.NetUtils;
+import org.apache.htrace.Span;
+import org.apache.htrace.Trace;
+import org.apache.htrace.TraceScope;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Set;
+import java.util.Map;
+import java.util.HashMap;
+import java.util.concurrent.CompletionService;
+import java.util.concurrent.ExecutorCompletionService;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.CancellationException;
+import java.util.concurrent.Callable;
+import java.util.concurrent.Future;
+
+
+/******************************************************************************
+ * DFSStripedInputStream reads from striped block groups, illustrated below:
+ *
+ * | <- Striped Block Group -> |
+ *  blk_0      blk_1       blk_2   <- A striped block group has
+ *    |          |           |          {@link #groupSize} blocks
+ *    v          v           v
+ * +------+   +------+   +------+
+ * |cell_0|   |cell_1|   |cell_2|  <- The logical read order should be
+ * +------+   +------+   +------+       cell_0, cell_1, ...
+ * |cell_3|   |cell_4|   |cell_5|
+ * +------+   +------+   +------+
+ * |cell_6|   |cell_7|   |cell_8|
+ * +------+   +------+   +------+
+ * |cell_9|
+ * +------+  <- A cell contains {@link #cellSize} bytes of data
+ *
+ * Three styles of read will eventually be supported:
+ *   1. Stateful read: TODO: HDFS-8033
+ *   2. pread without decode support
+ *     This is implemented by calculating the portion of read from each block and
+ *     issuing requests to each DataNode in parallel.
+ *   3. pread with decode support: TODO: will be supported after HDFS-7678
+ *****************************************************************************/
+public class DFSStripedInputStream extends DFSInputStream {
+  /**
+   * This method plans the read portion from each block in the stripe
+   * @param groupSize The size / width of the striping group
+   * @param cellSize The size of each striping cell
+   * @param startInBlk Starting offset in the striped block
+   * @param len Length of the read request
+   * @param bufOffset  Initial offset in the result buffer
+   * @return array of {@link ReadPortion}, each representing the portion of I/O
+   *         for an individual block in the group
+   */
+  @VisibleForTesting
+  static ReadPortion[] planReadPortions(final int groupSize,
+      final int cellSize, final long startInBlk, final int len, int bufOffset) {
+    ReadPortion[] results = new ReadPortion[groupSize];
+    for (int i = 0; i < groupSize; i++) {
+      results[i] = new ReadPortion();
+    }
+
+    // cellIdxInBlk is the index of the cell in the block
+    // E.g., cell_3 is the 2nd cell in blk_0
+    int cellIdxInBlk = (int) (startInBlk / (cellSize * groupSize));
+
+    // blkIdxInGroup is the index of the block in the striped block group
+    // E.g., blk_2 is the 3rd block in the group
+    final int blkIdxInGroup = (int) (startInBlk / cellSize % groupSize);
+    results[blkIdxInGroup].startOffsetInBlock = cellSize * cellIdxInBlk +
+        startInBlk % cellSize;
+    boolean crossStripe = false;
+    for (int i = 1; i < groupSize; i++) {
+      if (blkIdxInGroup + i >= groupSize && !crossStripe) {
+        cellIdxInBlk++;
+        crossStripe = true;
+      }
+      results[(blkIdxInGroup + i) % groupSize].startOffsetInBlock =
+          cellSize * cellIdxInBlk;
+    }
+
+    int firstCellLen = Math.min(cellSize - (int) (startInBlk % cellSize), len);
+    results[blkIdxInGroup].offsetsInBuf.add(bufOffset);
+    results[blkIdxInGroup].lengths.add(firstCellLen);
+    results[blkIdxInGroup].readLength += firstCellLen;
+
+    int i = (blkIdxInGroup + 1) % groupSize;
+    for (int done = firstCellLen; done < len; done += cellSize) {
+      ReadPortion rp = results[i];
+      rp.offsetsInBuf.add(done + bufOffset);
+      final int readLen = Math.min(len - done, cellSize);
+      rp.lengths.add(readLen);
+      rp.readLength += readLen;
+      i = (i + 1) % groupSize;
+    }
+    return results;
+  }
+
+  /**
+   * This method parses a striped block group into individual blocks.
+   *
+   * @param bg The striped block group
+   * @param dataBlkNum the number of data blocks
+   * @return An array containing the blocks in the group
+   */
+  @VisibleForTesting
+  static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg,
+      int dataBlkNum, int cellSize) {
+    int locatedBGSize = bg.getBlockIndices().length;
+    // TODO not considering missing blocks for now, only identify data blocks
+    LocatedBlock[] lbs = new LocatedBlock[dataBlkNum];
+    for (short i = 0; i < locatedBGSize; i++) {
+      final int idx = bg.getBlockIndices()[i];
+      if (idx < dataBlkNum && lbs[idx] == null) {
+        lbs[idx] = constructInternalBlock(bg, i, cellSize, idx);
+      }
+    }
+    return lbs;
+  }
+
+  private static LocatedBlock constructInternalBlock(LocatedStripedBlock bg,
+      int idxInReturnedLocs, int cellSize, int idxInBlockGroup) {
+    final ExtendedBlock blk = new ExtendedBlock(bg.getBlock());
+    blk.setBlockId(bg.getBlock().getBlockId() + idxInBlockGroup);
+    // TODO: fix the numBytes computation
+
+    return new LocatedBlock(blk,
+        new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]},
+        new String[]{bg.getStorageIDs()[idxInReturnedLocs]},
+        new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]},
+        bg.getStartOffset() + idxInBlockGroup * cellSize, bg.isCorrupt(),
+        null);
+  }
+
+
+  private int cellSize = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final short groupSize = HdfsConstants.NUM_DATA_BLOCKS;
+
+  DFSStripedInputStream(DFSClient dfsClient, String src, boolean verifyChecksum)
+      throws IOException {
+    super(dfsClient, src, verifyChecksum);
+    DFSClient.LOG.debug("Creating an striped input stream for file " + src);
+  }
+
+  @Override
+  public synchronized int read(final ByteBuffer buf) throws IOException {
+    throw new UnsupportedActionException("Stateful read is not supported");
+  }
+
+  @Override
+  public synchronized int read(final byte buf[], int off, int len)
+      throws IOException {
+    throw new UnsupportedActionException("Stateful read is not supported");
+  }
+
+  /**
+   * | <--------- LocatedStripedBlock (ID = 0) ---------> |
+   * LocatedBlock (0) | LocatedBlock (1) | LocatedBlock (2)
+   *                      ^
+   *                    offset
+   * On a striped file, the super method {@link DFSInputStream#getBlockAt}
+   * treats a striped block group as a single {@link LocatedBlock} object,
+   * which includes target in its range. This method adds the logic of:
+   *   1. Analyzing the index of required block based on offset
+   *   2. Parsing the block group to obtain the block location on that index
+   */
+  @Override
+  protected LocatedBlock getBlockAt(long blkStartOffset) throws IOException {
+    LocatedBlock lb = super.getBlockAt(blkStartOffset);
+    assert lb instanceof LocatedStripedBlock : "NameNode should return a " +
+        "LocatedStripedBlock for a striped file";
+
+    int idx = (int) (((blkStartOffset - lb.getStartOffset()) / cellSize)
+        % groupSize);
+    // If indexing information is returned, iterate through the index array
+    // to find the entry for position idx in the group
+    LocatedStripedBlock lsb = (LocatedStripedBlock) lb;
+    int i = 0;
+    for (; i < lsb.getBlockIndices().length; i++) {
+      if (lsb.getBlockIndices()[i] == idx) {
+        break;
+      }
+    }
+    if (DFSClient.LOG.isDebugEnabled()) {
+      DFSClient.LOG.debug("getBlockAt for striped blocks, offset="
+          + blkStartOffset + ". Obtained block " + lb + ", idx=" + idx);
+    }
+    return constructInternalBlock(lsb, i, cellSize, idx);
+  }
+
+  private LocatedBlock getBlockGroupAt(long offset) throws IOException {
+    return super.getBlockAt(offset);
+  }
+
+  /**
+   * Real implementation of pread.
+   */
+  @Override
+  protected void fetchBlockByteRange(LocatedBlock block, long start,
+      long end, byte[] buf, int offset,
+      Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap)
+      throws IOException {
+    Map<Future<Void>, Integer> futures = new HashMap<>();
+    CompletionService<Void> stripedReadsService =
+        new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool());
+    int len = (int) (end - start + 1);
+
+    // Refresh the striped block group
+    block = getBlockGroupAt(block.getStartOffset());
+    assert block instanceof LocatedStripedBlock : "NameNode" +
+        " should return a LocatedStripedBlock for a striped file";
+    LocatedStripedBlock blockGroup = (LocatedStripedBlock) block;
+
+    // Planning the portion of I/O for each shard
+    ReadPortion[] readPortions = planReadPortions(groupSize, cellSize, start,
+        len, offset);
+
+    // Parse group to get chosen DN location
+    LocatedBlock[] blks = parseStripedBlockGroup(blockGroup, groupSize, cellSize);
+
+    for (short i = 0; i < groupSize; i++) {
+      ReadPortion rp = readPortions[i];
+      if (rp.readLength <= 0) {
+        continue;
+      }
+      DatanodeInfo loc = blks[i].getLocations()[0];
+      StorageType type = blks[i].getStorageTypes()[0];
+      DNAddrPair dnAddr = new DNAddrPair(loc, NetUtils.createSocketAddr(
+          loc.getXferAddr(dfsClient.getConf().connectToDnViaHostname)), type);
+      Callable<Void> readCallable = getFromOneDataNode(dnAddr, blks[i],
+          rp.startOffsetInBlock, rp.startOffsetInBlock + rp.readLength - 1, buf,
+          rp.getOffsets(), rp.getLengths(), corruptedBlockMap, i);
+      Future<Void> getFromDNRequest = stripedReadsService.submit(readCallable);
+      DFSClient.LOG.debug("Submitting striped read request for " + blks[i]);
+      futures.put(getFromDNRequest, (int) i);
+    }
+    while (!futures.isEmpty()) {
+      try {
+        waitNextCompletion(stripedReadsService, futures);
+      } catch (InterruptedException ie) {
+        // Ignore and retry
+      }
+    }
+  }
+
+  private Callable<Void> getFromOneDataNode(final DNAddrPair datanode,
+      final LocatedBlock block, final long start, final long end,
+      final byte[] buf, final int[] offsets, final int[] lengths,
+      final Map<ExtendedBlock, Set<DatanodeInfo>> corruptedBlockMap,
+      final int hedgedReadId) {
+    final Span parentSpan = Trace.currentSpan();
+    return new Callable<Void>() {
+      @Override
+      public Void call() throws Exception {
+        TraceScope scope =
+            Trace.startSpan("Parallel reading " + hedgedReadId, parentSpan);
+        try {
+          actualGetFromOneDataNode(datanode, block, start,
+              end, buf, offsets, lengths, corruptedBlockMap);
+        } finally {
+          scope.close();
+        }
+        return null;
+      }
+    };
+  }
+
+  private void waitNextCompletion(CompletionService<Void> stripedReadsService,
+      Map<Future<Void>, Integer> futures) throws InterruptedException {
+    if (futures.isEmpty()) {
+      throw new InterruptedException("Futures already empty");
+    }
+    Future<Void> future = null;
+    try {
+      future = stripedReadsService.take();
+      future.get();
+      futures.remove(future);
+    } catch (ExecutionException | CancellationException e) {
+      // already logged in the Callable
+      futures.remove(future);
+    }
+    throw new InterruptedException("let's retry");
+  }
+
+  public void setCellSize(int cellSize) {
+    this.cellSize = cellSize;
+  }
+
+  /**
+   * This class represents the portion of I/O associated with each block in the
+   * striped block group.
+   */
+  static class ReadPortion {
+    /**
+     * startOffsetInBlock
+     *     |
+     *     v
+     *     |<-lengths[0]->|<-  lengths[1]  ->|<-lengths[2]->|
+     * +------------------+------------------+----------------+
+     * |      cell_0      |      cell_3      |     cell_6     |  <- blk_0
+     * +------------------+------------------+----------------+
+     *   _/                \_______________________
+     *  |                                          |
+     *  v offsetsInBuf[0]                          v offsetsInBuf[1]
+     * +------------------------------------------------------+
+     * |  cell_0     |      cell_1 and cell_2      |cell_3 ...|   <- buf
+     * |  (partial)  |    (from blk_1 and blk_2)   |          |
+     * +------------------------------------------------------+
+     */
+    private long startOffsetInBlock = 0;
+    private long readLength = 0;
+    private final List<Integer> offsetsInBuf = new ArrayList<>();
+    private final List<Integer> lengths = new ArrayList<>();
+
+    int[] getOffsets() {
+      int[] offsets = new int[offsetsInBuf.size()];
+      for (int i = 0; i < offsets.length; i++) {
+        offsets[i] = offsetsInBuf.get(i);
+      }
+      return offsets;
+    }
+
+    int[] getLengths() {
+      int[] lens = new int[this.lengths.size()];
+      for (int i = 0; i < lens.length; i++) {
+        lens[i] = this.lengths.get(i);
+      }
+      return lens;
+    }
+
+    long getReadLength() {
+      return readLength;
+    }
+
+    long getStartOffsetInBlock() {
+      return startOffsetInBlock;
+    }
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
index b2882af..a888aa4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsConstants.java
@@ -186,5 +186,5 @@ public class HdfsConstants {
   public static final byte MAX_BLOCKS_IN_GROUP = 16;
 
   // The chunk size for striped block which is used by erasure coding
-  public static final int BLOCK_STRIPED_CHUNK_SIZE = 64 * 1024;
+  public static final int BLOCK_STRIPED_CELL_SIZE = 128 * 1024;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index a38e8f2..4ba8193 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -212,5 +212,9 @@ public class LocatedBlock {
         + "; locs=" + Arrays.asList(locs)
         + "}";
   }
+
+  public boolean isStriped() {
+    return false;
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
index 97e3a69..98614db 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
@@ -65,4 +65,9 @@ public class LocatedStripedBlock extends LocatedBlock {
   public int[] getBlockIndices() {
     return this.blockIndices;
   }
+
+  @Override
+  public boolean isStriped() {
+    return true;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 4a85efb..20b0c5c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -23,7 +23,7 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 import java.io.DataOutput;
 import java.io.IOException;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
@@ -203,8 +203,8 @@ public class BlockInfoStriped extends BlockInfo {
     // In case striped blocks, total usage by this striped blocks should
     // be the total of data blocks and parity blocks because
     // `getNumBytes` is the total of actual data block size.
-    return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CHUNK_SIZE) + 1)
-        * BLOCK_STRIPED_CHUNK_SIZE * parityBlockNum + getNumBytes();
+    return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CELL_SIZE) + 1)
+        * BLOCK_STRIPED_CELL_SIZE * parityBlockNum + getNumBytes();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index 4f37fbb..987f7d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -64,6 +64,12 @@ import java.util.Random;
 import java.util.Set;
 import java.util.UUID;
 import java.util.concurrent.TimeoutException;
+import com.google.common.base.Charsets;
+import com.google.common.base.Joiner;
+import com.google.common.base.Preconditions;
+import com.google.common.base.Supplier;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
 
 import org.apache.commons.io.FileUtils;
 import org.apache.commons.logging.Log;
@@ -99,6 +105,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeInfo.AdminStates;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LayoutVersion;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
 import org.apache.hadoop.hdfs.protocol.datatransfer.Sender;
@@ -106,7 +113,6 @@ import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseP
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.ExportedBlockKeys;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil;
@@ -120,8 +126,10 @@ import org.apache.hadoop.hdfs.server.datanode.DataNode;
 import org.apache.hadoop.hdfs.server.datanode.DataNodeLayoutVersion;
 import org.apache.hadoop.hdfs.server.datanode.TestTransferRbw;
 import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi;
+import org.apache.hadoop.hdfs.server.namenode.FSDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLog;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.apache.hadoop.hdfs.server.namenode.INodeFile;
 import org.apache.hadoop.hdfs.server.namenode.LeaseManager;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
@@ -129,7 +137,6 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
 import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
-import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
 import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
@@ -149,12 +156,8 @@ import org.apache.log4j.Level;
 import org.junit.Assume;
 import org.mockito.internal.util.reflection.Whitebox;
 
-import com.google.common.base.Charsets;
-import com.google.common.base.Joiner;
-import com.google.common.base.Preconditions;
-import com.google.common.base.Supplier;
-import com.google.common.collect.Lists;
-import com.google.common.collect.Maps;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 
 /** Utilities for HDFS tests */
 public class DFSTestUtil {
@@ -1782,4 +1785,77 @@ public class DFSTestUtil {
     reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
     return reports;
   }
+
+  public static void createECFile(MiniDFSCluster cluster, Path file, Path dir,
+      int numBlocks, int numStripesPerBlk) throws Exception {
+    DistributedFileSystem dfs = cluster.getFileSystem();
+    dfs.mkdirs(dir);
+    dfs.getClient().createErasureCodingZone(dir.toString());
+
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1); // create an empty file
+
+      FSNamesystem ns = cluster.getNamesystem();
+      FSDirectory fsdir = ns.getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+
+      ExtendedBlock previous = null;
+      for (int i = 0; i < numBlocks; i++) {
+        Block newBlock = createBlock(cluster.getDataNodes(), dfs, ns,
+            file.toString(), fileNode, dfs.getClient().getClientName(),
+            previous, numStripesPerBlk);
+        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
+      }
+
+      dfs.getClient().namenode.complete(file.toString(),
+          dfs.getClient().getClientName(), previous, fileNode.getId());
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+
+  static Block createBlock(List<DataNode> dataNodes, DistributedFileSystem fs,
+      FSNamesystem ns, String file, INodeFile fileNode, String clientName,
+      ExtendedBlock previous, int numStripes) throws Exception {
+    fs.getClient().namenode.addBlock(file, clientName, previous, null,
+        fileNode.getId(), null);
+
+    final BlockInfo lastBlock = fileNode.getLastBlock();
+    final int groupSize = fileNode.getBlockReplication();
+    // 1. RECEIVING_BLOCK IBR
+    int i = 0;
+    for (DataNode dn : dataNodes) {
+      if (i < groupSize) {
+        final Block block = new Block(lastBlock.getBlockId() + i++, 0,
+            lastBlock.getGenerationStamp());
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block,
+                ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+        }
+      }
+    }
+
+    // 2. RECEIVED_BLOCK IBR
+    i = 0;
+    for (DataNode dn : dataNodes) {
+      if (i < groupSize) {
+        final Block block = new Block(lastBlock.getBlockId() + i++,
+            numStripes * BLOCK_STRIPED_CELL_SIZE, lastBlock.getGenerationStamp());
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block,
+                ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
+        }
+      }
+    }
+
+    lastBlock.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS);
+    return lastBlock;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
new file mode 100644
index 0000000..113b73c
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFile.java
@@ -0,0 +1,307 @@
+/**
+ * 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.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
+import static org.apache.hadoop.hdfs.DFSStripedInputStream.ReadPortion;
+import static org.junit.Assert.assertArrayEquals;
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertFalse;
+import static org.junit.Assert.assertTrue;
+
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
+import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Arrays;
+import java.util.List;
+
+// TODO: read results not automatically verified, to extend SimulatedFSDataset
+public class TestReadStripedFile {
+
+  public static final Log LOG = LogFactory.getLog(TestReadStripedFile.class);
+
+  private MiniDFSCluster cluster;
+  private Configuration conf = new Configuration();
+  private DistributedFileSystem fs;
+  private final Path dirPath = new Path("/striped");
+  private Path filePath = new Path(dirPath, "file");
+  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS;
+  private final short TOTAL_SIZE = HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
+  private final int CELLSIZE = HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
+  private final int NUM_STRIPE_PER_BLOCK = 2;
+  private final int BLOCKSIZE = 2 * GROUP_SIZE * CELLSIZE;
+
+  @Before
+  public void setup() throws IOException {
+    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
+    SimulatedFSDataset.setFactory(conf);
+    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(TOTAL_SIZE)
+        .build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  private void testPlanReadPortions(int startInBlk, int length,
+      int bufferOffset, int[] readLengths, int[] offsetsInBlock,
+      int[][] bufferOffsets, int[][] bufferLengths) {
+    ReadPortion[] results = DFSStripedInputStream.planReadPortions(GROUP_SIZE,
+        CELLSIZE, startInBlk, length, bufferOffset);
+    assertEquals(GROUP_SIZE, results.length);
+
+    for (int i = 0; i < GROUP_SIZE; i++) {
+      assertEquals(readLengths[i], results[i].getReadLength());
+      assertEquals(offsetsInBlock[i], results[i].getStartOffsetInBlock());
+      final int[] bOffsets = results[i].getOffsets();
+      assertArrayEquals(bufferOffsets[i], bOffsets);
+      final int[] bLengths = results[i].getLengths();
+      assertArrayEquals(bufferLengths[i], bLengths);
+    }
+  }
+
+  /**
+   * Test {@link DFSStripedInputStream#planReadPortions}
+   */
+  @Test
+  public void testPlanReadPortions() {
+    /**
+     * start block offset is 0, read cellSize - 10
+     */
+    testPlanReadPortions(0, CELLSIZE - 10, 0,
+        new int[]{CELLSIZE - 10, 0, 0}, new int[]{0, 0, 0},
+        new int[][]{new int[]{0}, new int[]{}, new int[]{}},
+        new int[][]{new int[]{CELLSIZE - 10}, new int[]{}, new int[]{}});
+
+    /**
+     * start block offset is 0, read 3 * cellSize
+     */
+    testPlanReadPortions(0, GROUP_SIZE * CELLSIZE, 0,
+        new int[]{CELLSIZE, CELLSIZE, CELLSIZE}, new int[]{0, 0, 0},
+        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{CELLSIZE * 2}},
+        new int[][]{new int[]{CELLSIZE}, new int[]{CELLSIZE}, new int[]{CELLSIZE}});
+
+    /**
+     * start block offset is 0, read cellSize + 10
+     */
+    testPlanReadPortions(0, CELLSIZE + 10, 0,
+        new int[]{CELLSIZE, 10, 0}, new int[]{0, 0, 0},
+        new int[][]{new int[]{0}, new int[]{CELLSIZE}, new int[]{}},
+        new int[][]{new int[]{CELLSIZE}, new int[]{10}, new int[]{}});
+
+    /**
+     * start block offset is 0, read 5 * cellSize + 10, buffer start offset is 100
+     */
+    testPlanReadPortions(0, 5 * CELLSIZE + 10, 100,
+        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE + 10}, new int[]{0, 0, 0},
+        new int[][]{new int[]{100, 100 + CELLSIZE * GROUP_SIZE},
+            new int[]{100 + CELLSIZE, 100 + CELLSIZE * 4},
+            new int[]{100 + CELLSIZE * 2, 100 + CELLSIZE * 5}},
+        new int[][]{new int[]{CELLSIZE, CELLSIZE},
+            new int[]{CELLSIZE, CELLSIZE},
+            new int[]{CELLSIZE, 10}});
+
+    /**
+     * start block offset is 2, read 3 * cellSize
+     */
+    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE, 100,
+        new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
+        new int[]{2, 0, 0},
+        new int[][]{new int[]{100, 100 + GROUP_SIZE * CELLSIZE - 2},
+            new int[]{100 + CELLSIZE - 2},
+            new int[]{100 + CELLSIZE * 2 - 2}},
+        new int[][]{new int[]{CELLSIZE - 2, 2},
+            new int[]{CELLSIZE},
+            new int[]{CELLSIZE}});
+
+    /**
+     * start block offset is 2, read 3 * cellSize + 10
+     */
+    testPlanReadPortions(2, GROUP_SIZE * CELLSIZE + 10, 0,
+        new int[]{CELLSIZE + 10, CELLSIZE, CELLSIZE},
+        new int[]{2, 0, 0},
+        new int[][]{new int[]{0, GROUP_SIZE * CELLSIZE - 2},
+            new int[]{CELLSIZE - 2},
+            new int[]{CELLSIZE * 2 - 2}},
+        new int[][]{new int[]{CELLSIZE - 2, 12},
+            new int[]{CELLSIZE},
+            new int[]{CELLSIZE}});
+
+    /**
+     * start block offset is cellSize * 2 - 1, read 5 * cellSize + 10
+     */
+    testPlanReadPortions(CELLSIZE * 2 - 1, 5 * CELLSIZE + 10, 0,
+        new int[]{CELLSIZE * 2, CELLSIZE + 10, CELLSIZE * 2},
+        new int[]{CELLSIZE, CELLSIZE - 1, 0},
+        new int[][]{new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1},
+            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1},
+            new int[]{1, 3 * CELLSIZE + 1}},
+        new int[][]{new int[]{CELLSIZE, CELLSIZE},
+            new int[]{1, CELLSIZE, 9},
+            new int[]{CELLSIZE, CELLSIZE}});
+
+    /**
+     * start block offset is cellSize * 6 - 1, read 7 * cellSize + 10
+     */
+    testPlanReadPortions(CELLSIZE * 6 - 1, 7 * CELLSIZE + 10, 0,
+        new int[]{CELLSIZE * 3, CELLSIZE * 2 + 9, CELLSIZE * 2 + 1},
+        new int[]{CELLSIZE * 2, CELLSIZE * 2, CELLSIZE * 2 - 1},
+        new int[][]{new int[]{1, 3 * CELLSIZE + 1, 6 * CELLSIZE + 1},
+            new int[]{CELLSIZE + 1, 4 * CELLSIZE + 1, 7 * CELLSIZE + 1},
+            new int[]{0, 2 * CELLSIZE + 1, 5 * CELLSIZE + 1}},
+        new int[][]{new int[]{CELLSIZE, CELLSIZE, CELLSIZE},
+            new int[]{CELLSIZE, CELLSIZE, 9},
+            new int[]{1, CELLSIZE, CELLSIZE}});
+  }
+
+  private LocatedStripedBlock createDummyLocatedBlock() {
+    final long blockGroupID = -1048576;
+    DatanodeInfo[] locs = new DatanodeInfo[TOTAL_SIZE];
+    String[] storageIDs = new String[TOTAL_SIZE];
+    StorageType[] storageTypes = new StorageType[TOTAL_SIZE];
+    int[] indices = new int[TOTAL_SIZE];
+    for (int i = 0; i < TOTAL_SIZE; i++) {
+      locs[i] = new DatanodeInfo(cluster.getDataNodes().get(i).getDatanodeId());
+      storageIDs[i] = cluster.getDataNodes().get(i).getDatanodeUuid();
+      storageTypes[i] = StorageType.DISK;
+      indices[i] = (i + 2) % GROUP_SIZE;
+    }
+    return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID),
+        locs, storageIDs, storageTypes, indices, 0, false, null);
+  }
+
+  @Test
+  public void testParseDummyStripedBlock() {
+    LocatedStripedBlock lsb = createDummyLocatedBlock();
+    LocatedBlock[] blocks = DFSStripedInputStream.parseStripedBlockGroup(
+        lsb, GROUP_SIZE, CELLSIZE);
+    assertEquals(GROUP_SIZE, blocks.length);
+    for (int j = 0; j < GROUP_SIZE; j++) {
+      assertFalse(blocks[j].isStriped());
+      assertEquals(j,
+          BlockIdManager.getBlockIndex(blocks[j].getBlock().getLocalBlock()));
+      assertEquals(j * CELLSIZE, blocks[j].getStartOffset());
+    }
+  }
+
+  @Test
+  public void testParseStripedBlock() throws Exception {
+    final int numBlocks = 4;
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCKSIZE * numBlocks);
+
+    assertEquals(4, lbs.locatedBlockCount());
+    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
+    for (LocatedBlock lb : lbList) {
+      assertTrue(lb.isStriped());
+    }
+
+    for (int i = 0; i < numBlocks; i++) {
+      LocatedStripedBlock lsb = (LocatedStripedBlock) (lbs.get(i));
+      LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
+          GROUP_SIZE, CELLSIZE);
+      assertEquals(GROUP_SIZE, blks.length);
+      for (int j = 0; j < GROUP_SIZE; j++) {
+        assertFalse(blks[j].isStriped());
+        assertEquals(j,
+            BlockIdManager.getBlockIndex(blks[j].getBlock().getLocalBlock()));
+        assertEquals(i * BLOCKSIZE + j * CELLSIZE, blks[j].getStartOffset());
+      }
+    }
+  }
+
+  /**
+   * Test {@link DFSStripedInputStream#getBlockAt(long)}
+   */
+  @Test
+  public void testGetBlock() throws Exception {
+    final int numBlocks = 4;
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCKSIZE * numBlocks);
+    final DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
+
+    List<LocatedBlock> lbList = lbs.getLocatedBlocks();
+    for (LocatedBlock aLbList : lbList) {
+      LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList;
+      LocatedBlock[] blks = DFSStripedInputStream.parseStripedBlockGroup(lsb,
+          GROUP_SIZE, CELLSIZE);
+      for (int j = 0; j < GROUP_SIZE; j++) {
+        LocatedBlock refreshed = in.getBlockAt(blks[j].getStartOffset());
+        assertEquals(blks[j].getBlock(), refreshed.getBlock());
+        assertEquals(blks[j].getStartOffset(), refreshed.getStartOffset());
+        assertArrayEquals(blks[j].getLocations(), refreshed.getLocations());
+      }
+    }
+  }
+
+  @Test
+  public void testPread() throws Exception {
+    final int numBlocks = 4;
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks,
+        NUM_STRIPE_PER_BLOCK);
+    LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(
+        filePath.toString(), 0, BLOCKSIZE);
+
+    assert lbs.get(0) instanceof LocatedStripedBlock;
+    LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0));
+    for (int i = 0; i < GROUP_SIZE; i++) {
+      Block blk = new Block(bg.getBlock().getBlockId() + i, BLOCKSIZE,
+          bg.getBlock().getGenerationStamp());
+      blk.setGenerationStamp(bg.getBlock().getGenerationStamp());
+      cluster.injectBlocks(i, Arrays.asList(blk),
+          bg.getBlock().getBlockPoolId());
+    }
+    DFSStripedInputStream in =
+        new DFSStripedInputStream(fs.getClient(), filePath.toString(), false);
+    in.setCellSize(CELLSIZE);
+    int readSize = GROUP_SIZE * CELLSIZE * NUM_STRIPE_PER_BLOCK;
+    byte[] readBuffer = new byte[readSize];
+    int ret = in.read(0, readBuffer, 0, readSize);
+
+    assertEquals(readSize, ret);
+    // Verify correctly read data from first DN
+    assertEquals(SimulatedFSDataset.DEFAULT_DATABYTE, readBuffer[0]);
+    // Verify correctly read data from second DN
+    assertEquals(SimulatedFSDataset.DEFAULT_DATABYTE, readBuffer[CELLSIZE + 1]);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/e67086dc/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
index d965ae7..b2ff6c8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java
@@ -18,15 +18,11 @@
 package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
@@ -36,19 +32,14 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
-import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
-import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
-import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
-import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
 import java.util.List;
-import java.util.UUID;
 
-import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CELL_SIZE;
 import static org.apache.hadoop.hdfs.protocol.HdfsConstants.NUM_DATA_BLOCKS;
 import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertFalse;
@@ -84,83 +75,10 @@ public class TestRecoverStripedBlocks {
     }
   }
 
-  public static void createECFile(MiniDFSCluster cluster, Path file, Path dir,
-      int numBlocks) throws Exception {
-    DistributedFileSystem dfs = cluster.getFileSystem();
-    dfs.mkdirs(dir);
-    dfs.getClient().getNamenode().createErasureCodingZone(dir.toString());
-
-    FSDataOutputStream out = null;
-    try {
-      out = dfs.create(file, (short) 1); // create an empty file
-
-      FSNamesystem ns = cluster.getNamesystem();
-      FSDirectory fsdir = ns.getFSDirectory();
-      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
-
-      ExtendedBlock previous = null;
-      for (int i = 0; i < numBlocks; i++) {
-        Block newBlock = createBlock(cluster.getDataNodes(), ns,
-            file.toString(), fileNode, dfs.getClient().getClientName(),
-            previous);
-        previous = new ExtendedBlock(ns.getBlockPoolId(), newBlock);
-      }
-
-      ns.completeFile(file.toString(), dfs.getClient().getClientName(),
-          previous, fileNode.getId());
-    } finally {
-      IOUtils.cleanup(null, out);
-    }
-  }
-
-  static Block createBlock(List<DataNode> dataNodes, FSNamesystem ns,
-      String file, INodeFile fileNode, String clientName,
-      ExtendedBlock previous) throws Exception {
-    ns.getAdditionalBlock(file, fileNode.getId(), clientName, previous, null,
-        null);
-
-    final BlockInfo lastBlock = fileNode.getLastBlock();
-    final int groupSize = fileNode.getBlockReplication();
-    // 1. RECEIVING_BLOCK IBR
-    int i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        final Block block = new Block(lastBlock.getBlockId() + i++, 0,
-            lastBlock.getGenerationStamp());
-        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
-        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
-            .makeReportForReceivedBlock(block,
-                ReceivedDeletedBlockInfo.BlockStatus.RECEIVING_BLOCK, storage);
-        for (StorageReceivedDeletedBlocks report : reports) {
-          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
-        }
-      }
-    }
-
-    // 2. RECEIVED_BLOCK IBR
-    i = 0;
-    for (DataNode dn : dataNodes) {
-      if (i < groupSize) {
-        final Block block = new Block(lastBlock.getBlockId() + i++,
-            BLOCK_STRIPED_CHUNK_SIZE, lastBlock.getGenerationStamp());
-        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
-        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
-            .makeReportForReceivedBlock(block,
-                ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage);
-        for (StorageReceivedDeletedBlocks report : reports) {
-          ns.processIncrementalBlockReport(dn.getDatanodeId(), report);
-        }
-      }
-    }
-
-    lastBlock.setNumBytes(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS);
-    return lastBlock;
-  }
-
   @Test
   public void testMissingStripedBlock() throws Exception {
     final int numBlocks = 4;
-    createECFile(cluster, filePath, dirPath, numBlocks);
+    DFSTestUtil.createECFile(cluster, filePath, dirPath, numBlocks, 1);
 
     // make sure the file is complete in NN
     final INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
@@ -172,7 +90,7 @@ public class TestRecoverStripedBlocks {
     for (BlockInfo blk : blocks) {
       assertTrue(blk.isStriped());
       assertTrue(blk.isComplete());
-      assertEquals(BLOCK_STRIPED_CHUNK_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes());
+      assertEquals(BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, blk.getNumBytes());
       final BlockInfoStriped sb = (BlockInfoStriped) blk;
       assertEquals(GROUP_SIZE, sb.numNodes());
     }


[26/50] [abbrv] hadoop git commit: HDFS-7907. Erasure Coding: track invalid, corrupt, and under-recovery striped blocks in NameNode. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7907. Erasure Coding: track invalid, corrupt, and under-recovery striped blocks in NameNode. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2e39c2b7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2e39c2b7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2e39c2b7

Branch: refs/heads/HDFS-7285
Commit: 2e39c2b7fd477e77c0cd649a94ac67f785b87d6a
Parents: f6975ae
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 30 11:25:09 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:46:02 2015 -0700

----------------------------------------------------------------------
 .../blockmanagement/BlockInfoStriped.java       |  25 ++-
 .../server/blockmanagement/BlockManager.java    | 203 ++++++++++---------
 .../blockmanagement/DecommissionManager.java    |  85 ++++----
 .../hdfs/server/namenode/FSNamesystem.java      |   8 +-
 .../server/blockmanagement/TestNodeCount.java   |   2 +-
 .../TestOverReplicatedBlocks.java               |   4 +-
 6 files changed, 172 insertions(+), 155 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e39c2b7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 30b5ee7..4a85efb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -18,11 +18,13 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
+
 import java.io.DataOutput;
 import java.io.IOException;
 
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
+
 /**
  * Subclass of {@link BlockInfo}, presenting a block group in erasure coding.
  *
@@ -37,7 +39,6 @@ import java.io.IOException;
  * array to record the block index for each triplet.
  */
 public class BlockInfoStriped extends BlockInfo {
-  private final int   chunkSize = HdfsConstants.BLOCK_STRIPED_CHUNK_SIZE;
   private final short dataBlockNum;
   private final short parityBlockNum;
   /**
@@ -132,6 +133,22 @@ public class BlockInfoStriped extends BlockInfo {
     return i == -1 ? -1 : indices[i];
   }
 
+  /**
+   * Identify the block stored in the given datanode storage. Note that
+   * the returned block has the same block Id with the one seen/reported by the
+   * DataNode.
+   */
+  Block getBlockOnStorage(DatanodeStorageInfo storage) {
+    int index = getStorageBlockIndex(storage);
+    if (index < 0) {
+      return null;
+    } else {
+      Block block = new Block(this);
+      block.setBlockId(this.getBlockId() + index);
+      return block;
+    }
+  }
+
   @Override
   boolean removeStorage(DatanodeStorageInfo storage) {
     int dnIndex = findStorageInfoFromEnd(storage);
@@ -186,8 +203,8 @@ public class BlockInfoStriped extends BlockInfo {
     // In case striped blocks, total usage by this striped blocks should
     // be the total of data blocks and parity blocks because
     // `getNumBytes` is the total of actual data block size.
-    return ((getNumBytes() - 1) / (dataBlockNum * chunkSize) + 1)
-        * chunkSize * parityBlockNum + getNumBytes();
+    return ((getNumBytes() - 1) / (dataBlockNum * BLOCK_STRIPED_CHUNK_SIZE) + 1)
+        * BLOCK_STRIPED_CHUNK_SIZE * parityBlockNum + getNumBytes();
   }
 
   @Override

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e39c2b7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index a3c5512..47865ec 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -178,7 +178,11 @@ public class BlockManager {
   /** Store blocks -> datanodedescriptor(s) map of corrupt replicas */
   final CorruptReplicasMap corruptReplicas = new CorruptReplicasMap();
 
-  /** Blocks to be invalidated. */
+  /**
+   * Blocks to be invalidated.
+   * For a striped block to invalidate, we should track its individual internal
+   * blocks.
+   */
   private final InvalidateBlocks invalidateBlocks;
   
   /**
@@ -194,8 +198,8 @@ public class BlockManager {
    * Maps a StorageID to the set of blocks that are "extra" for this
    * DataNode. We'll eventually remove these extras.
    */
-  public final Map<String, LightWeightLinkedSet<Block>> excessReplicateMap =
-    new TreeMap<String, LightWeightLinkedSet<Block>>();
+  public final Map<String, LightWeightLinkedSet<BlockInfo>> excessReplicateMap =
+    new TreeMap<>();
 
   /**
    * Store set of Blocks that need to be replicated 1 or more times.
@@ -592,11 +596,11 @@ public class BlockManager {
         ((BlockInfoStriped) block).getDataBlockNum() : minReplication;
   }
 
-  public boolean checkMinStorage(BlockInfo block) {
+  public boolean hasMinStorage(BlockInfo block) {
     return countNodes(block).liveReplicas() >= getMinStorageNum(block);
   }
 
-  public boolean checkMinStorage(BlockInfo block, int liveNum) {
+  public boolean hasMinStorage(BlockInfo block, int liveNum) {
     return liveNum >= getMinStorageNum(block);
   }
 
@@ -641,7 +645,7 @@ public class BlockManager {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(lastBlock, commitBlock);
-    if (checkMinStorage(lastBlock)) {
+    if (hasMinStorage(lastBlock)) {
       completeBlock(bc, bc.numBlocks() - 1, false);
     }
     return b;
@@ -665,7 +669,7 @@ public class BlockManager {
     }
 
     int numNodes = curBlock.numNodes();
-    if (!force && !checkMinStorage(curBlock, numNodes)) {
+    if (!force && !hasMinStorage(curBlock, numNodes)) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
     }
@@ -763,7 +767,7 @@ public class BlockManager {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        checkMinStorage(oldBlock, targets.length) ? -1 : 0,
+        hasMinStorage(oldBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
@@ -1090,7 +1094,7 @@ public class BlockManager {
    
   /** Remove the blocks associated to the given datanode. */
   void removeBlocksAssociatedTo(final DatanodeDescriptor node) {
-    final Iterator<? extends Block> it = node.getBlockIterator();
+    final Iterator<BlockInfo> it = node.getBlockIterator();
     while(it.hasNext()) {
       removeStoredBlock(it.next(), node);
     }
@@ -1104,10 +1108,10 @@ public class BlockManager {
   /** Remove the blocks associated to the given DatanodeStorageInfo. */
   void removeBlocksAssociatedTo(final DatanodeStorageInfo storageInfo) {
     assert namesystem.hasWriteLock();
-    final Iterator<? extends Block> it = storageInfo.getBlockIterator();
+    final Iterator<BlockInfo> it = storageInfo.getBlockIterator();
     DatanodeDescriptor node = storageInfo.getDatanodeDescriptor();
     while(it.hasNext()) {
-      Block block = it.next();
+      BlockInfo block = it.next();
       removeStoredBlock(block, node);
       invalidateBlocks.remove(node, block);
     }
@@ -1129,21 +1133,32 @@ public class BlockManager {
    * Adds block to list of blocks which will be invalidated on all its
    * datanodes.
    */
-  private void addToInvalidates(Block b) {
+  private void addToInvalidates(BlockInfo storedBlock) {
     if (!namesystem.isPopulatingReplQueues()) {
       return;
     }
     StringBuilder datanodes = new StringBuilder();
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(b, State.NORMAL)) {
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock,
+        State.NORMAL)) {
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      invalidateBlocks.add(b, node, false);
-      datanodes.append(node).append(" ");
+      final Block b = getBlockToInvalidate(storedBlock, storage);
+      if (b != null) {
+        invalidateBlocks.add(b, node, false);
+        datanodes.append(node).append(" ");
+      }
     }
     if (datanodes.length() != 0) {
-      blockLog.info("BLOCK* addToInvalidates: {} {}", b, datanodes.toString());
+      blockLog.info("BLOCK* addToInvalidates: {} {}", storedBlock,
+          datanodes.toString());
     }
   }
 
+  private Block getBlockToInvalidate(BlockInfo storedBlock,
+      DatanodeStorageInfo storage) {
+    return storedBlock.isStriped() ?
+        ((BlockInfoStriped) storedBlock).getBlockOnStorage(storage) : storedBlock;
+  }
+
   /**
    * Remove all block invalidation tasks under this datanode UUID;
    * used when a datanode registers with a new UUID and the old one
@@ -1201,7 +1216,7 @@ public class BlockManager {
       DatanodeStorageInfo storageInfo,
       DatanodeDescriptor node) throws IOException {
 
-    BlockCollection bc = b.corrupted.getBlockCollection();
+    BlockCollection bc = b.stored.getBlockCollection();
     if (bc == null) {
       blockLog.info("BLOCK markBlockAsCorrupt: {} cannot be marked as" +
           " corrupt as it does not belong to any file", b);
@@ -1211,7 +1226,7 @@ public class BlockManager {
 
     // Add replica to the data-node if it is not already there
     if (storageInfo != null) {
-      storageInfo.addBlock(b.stored, b.reportedBlock);
+      storageInfo.addBlock(b.stored, b.corrupted);
     }
 
     // Add this replica to corruptReplicas Map
@@ -1221,8 +1236,7 @@ public class BlockManager {
     NumberReplicas numberOfReplicas = countNodes(b.stored);
     boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= bc
         .getBlockReplication();
-    boolean minReplicationSatisfied = checkMinStorage(b.stored,
-        numberOfReplicas.liveReplicas());
+    boolean minReplicationSatisfied = hasMinStorage(b.stored, numberOfReplicas.liveReplicas());
     boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
         (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
         bc.getBlockReplication();
@@ -1414,7 +1428,7 @@ public class BlockManager {
 
             if (numEffectiveReplicas >= requiredReplication) {
               if ( (pendingReplications.getNumReplicas(block) > 0) ||
-                   (blockHasEnoughRacks(block)) ) {
+                   (blockHasEnoughRacks(block, requiredReplication)) ) {
                 neededReplications.remove(block, priority); // remove from neededReplications
                 blockLog.info("BLOCK* Removing {} from neededReplications as" +
                         " it has enough replicas", block);
@@ -1497,7 +1511,7 @@ public class BlockManager {
 
           if (numEffectiveReplicas >= requiredReplication) {
             if ( (pendingReplications.getNumReplicas(block) > 0) ||
-                 (blockHasEnoughRacks(block)) ) {
+                 (blockHasEnoughRacks(block, requiredReplication)) ) {
               neededReplications.remove(block, priority); // remove from neededReplications
               rw.targets = null;
               blockLog.info("BLOCK* Removing {} from neededReplications as" +
@@ -1507,7 +1521,7 @@ public class BlockManager {
           }
 
           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
-               (!blockHasEnoughRacks(block)) ) {
+               (!blockHasEnoughRacks(block, requiredReplication)) ) {
             if (rw.srcNodes[0].getNetworkLocation().equals(
                 targets[0].getDatanodeDescriptor().getNetworkLocation())) {
               //No use continuing, unless a new rack in this case
@@ -1701,7 +1715,7 @@ public class BlockManager {
             getStorageBlockIndex(storage));
       }
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
-      LightWeightLinkedSet<Block> excessBlocks =
+      LightWeightLinkedSet<BlockInfo> excessBlocks =
         excessReplicateMap.get(node.getDatanodeUuid());
       int countableReplica = storage.getState() == State.NORMAL ? 1 : 0;
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
@@ -1832,39 +1846,32 @@ public class BlockManager {
    * list of blocks that should be considered corrupt due to a block report.
    */
   private static class BlockToMarkCorrupt {
-    /** The corrupted block in a datanode. */
-    final BlockInfo corrupted;
+    /**
+     * The corrupted block in a datanode. This is the one reported by the
+     * datanode.
+     */
+    final Block corrupted;
     /** The corresponding block stored in the BlockManager. */
     final BlockInfo stored;
-    /** The block reported from a datanode */
-    final Block reportedBlock;
     /** The reason to mark corrupt. */
     final String reason;
     /** The reason code to be stored */
     final Reason reasonCode;
 
-    BlockToMarkCorrupt(Block reported, BlockInfo corrupted,
-        BlockInfo stored, String reason, Reason reasonCode) {
-      Preconditions.checkNotNull(reported, "reported is null");
+    BlockToMarkCorrupt(Block corrupted, BlockInfo stored, String reason,
+        Reason reasonCode) {
       Preconditions.checkNotNull(corrupted, "corrupted is null");
       Preconditions.checkNotNull(stored, "stored is null");
 
-      this.reportedBlock = reported;
       this.corrupted = corrupted;
       this.stored = stored;
       this.reason = reason;
       this.reasonCode = reasonCode;
     }
 
-    BlockToMarkCorrupt(Block reported, BlockInfo stored, String reason,
-        Reason reasonCode) {
-      this(reported, stored, stored, reason, reasonCode);
-    }
-
-    BlockToMarkCorrupt(Block reported, BlockInfo stored, long gs,
+    BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs,
         String reason, Reason reasonCode) {
-      this(reported, BlockInfo.copyOf(stored), stored, reason,
-          reasonCode);
+      this(corrupted, stored, reason, reasonCode);
       //the corrupted block in datanode has a different generation stamp
       corrupted.setGenerationStamp(gs);
     }
@@ -2080,10 +2087,10 @@ public class BlockManager {
     // between the old and new block report.
     //
     Collection<BlockInfoToAdd> toAdd = new LinkedList<>();
-    Collection<Block> toRemove = new TreeSet<Block>();
-    Collection<Block> toInvalidate = new LinkedList<Block>();
-    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<BlockToMarkCorrupt>();
-    Collection<StatefulBlockInfo> toUC = new LinkedList<StatefulBlockInfo>();
+    Collection<BlockInfo> toRemove = new TreeSet<>();
+    Collection<Block> toInvalidate = new LinkedList<>();
+    Collection<BlockToMarkCorrupt> toCorrupt = new LinkedList<>();
+    Collection<StatefulBlockInfo> toUC = new LinkedList<>();
     reportDiff(storageInfo, report,
         toAdd, toRemove, toInvalidate, toCorrupt, toUC);
    
@@ -2092,7 +2099,7 @@ public class BlockManager {
     for (StatefulBlockInfo b : toUC) { 
       addStoredBlockUnderConstruction(b, storageInfo);
     }
-    for (Block b : toRemove) {
+    for (BlockInfo b : toRemove) {
       removeStoredBlock(b, node);
     }
     int numBlocksLogged = 0;
@@ -2227,7 +2234,7 @@ public class BlockManager {
   private void reportDiff(DatanodeStorageInfo storageInfo, 
       BlockListAsLongs newReport, 
       Collection<BlockInfoToAdd> toAdd,     // add to DatanodeDescriptor
-      Collection<Block> toRemove,           // remove from DatanodeDescriptor
+      Collection<BlockInfo> toRemove,       // remove from DatanodeDescriptor
       Collection<Block> toInvalidate,       // should be removed from DN
       Collection<BlockToMarkCorrupt> toCorrupt, // add to corrupt replicas list
       Collection<StatefulBlockInfo> toUC) { // add to under-construction list
@@ -2262,8 +2269,9 @@ public class BlockManager {
     // collect blocks that have not been reported
     // all of them are next to the delimiter
     Iterator<BlockInfo> it = storageInfo.new BlockIterator(delimiter.getNext(0));
-    while(it.hasNext())
+    while (it.hasNext()) {
       toRemove.add(it.next());
+    }
     storageInfo.removeBlock(delimiter);
   }
 
@@ -2587,7 +2595,7 @@ public class BlockManager {
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
-        && checkMinStorage(storedBlock, numCurrentReplica)) {
+        && hasMinStorage(storedBlock, numCurrentReplica)) {
       completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
@@ -2662,7 +2670,7 @@ public class BlockManager {
       + pendingReplications.getNumReplicas(storedBlock);
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
-        checkMinStorage(storedBlock, numLiveReplicas)) {
+        hasMinStorage(storedBlock, numLiveReplicas)) {
       storedBlock = completeBlock(bc, storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -2700,7 +2708,7 @@ public class BlockManager {
     int numCorruptNodes = num.corruptReplicas();
     if (numCorruptNodes != corruptReplicasCount) {
       LOG.warn("Inconsistent number of corrupt replicas for " +
-          storedBlock + "blockMap has " + numCorruptNodes + 
+          storedBlock + ". blockMap has " + numCorruptNodes +
           " but corrupt replicas map has " + corruptReplicasCount);
     }
     if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) {
@@ -2974,14 +2982,14 @@ public class BlockManager {
    * If there are any extras, call chooseExcessReplicates() to
    * mark them in the excessReplicateMap.
    */
-  private void processOverReplicatedBlock(final Block block,
+  private void processOverReplicatedBlock(final BlockInfo block,
       final short replication, final DatanodeDescriptor addedNode,
       DatanodeDescriptor delNodeHint) {
     assert namesystem.hasWriteLock();
     if (addedNode == delNodeHint) {
       delNodeHint = null;
     }
-    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<DatanodeStorageInfo>();
+    Collection<DatanodeStorageInfo> nonExcess = new ArrayList<>();
     Collection<DatanodeDescriptor> corruptNodes = corruptReplicas
         .getNodes(block);
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) {
@@ -2995,8 +3003,8 @@ public class BlockManager {
         postponeBlock(block);
         return;
       }
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(cur
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+          cur.getDatanodeUuid());
       if (excessBlocks == null || !excessBlocks.contains(block)) {
         if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
           // exclude corrupt replicas
@@ -3026,22 +3034,22 @@ public class BlockManager {
    * then pick a node with least free space
    */
   private void chooseExcessReplicates(final Collection<DatanodeStorageInfo> nonExcess, 
-                              Block b, short replication,
+                              BlockInfo storedBlock, short replication,
                               DatanodeDescriptor addedNode,
                               DatanodeDescriptor delNodeHint,
                               BlockPlacementPolicy replicator) {
     assert namesystem.hasWriteLock();
     // first form a rack to datanodes map and
-    BlockCollection bc = getBlockCollection(b);
-    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID());
+    BlockCollection bc = getBlockCollection(storedBlock);
+    final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(
+        bc.getStoragePolicyID());
     final List<StorageType> excessTypes = storagePolicy.chooseExcess(
         replication, DatanodeStorageInfo.toStorageTypes(nonExcess));
 
 
-    final Map<String, List<DatanodeStorageInfo>> rackMap
-        = new HashMap<String, List<DatanodeStorageInfo>>();
-    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<DatanodeStorageInfo>();
-    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<DatanodeStorageInfo>();
+    final Map<String, List<DatanodeStorageInfo>> rackMap = new HashMap<>();
+    final List<DatanodeStorageInfo> moreThanOne = new ArrayList<>();
+    final List<DatanodeStorageInfo> exactlyOne = new ArrayList<>();
     
     // split nodes into two sets
     // moreThanOne contains nodes on rack with more than one replica
@@ -3062,7 +3070,7 @@ public class BlockManager {
           moreThanOne, excessTypes)) {
         cur = delNodeHintStorage;
       } else { // regular excessive replica removal
-        cur = replicator.chooseReplicaToDelete(bc, b, replication,
+        cur = replicator.chooseReplicaToDelete(bc, storedBlock, replication,
             moreThanOne, exactlyOne, excessTypes);
       }
       firstOne = false;
@@ -3072,7 +3080,7 @@ public class BlockManager {
           exactlyOne, cur);
 
       nonExcess.remove(cur);
-      addToExcessReplicate(cur.getDatanodeDescriptor(), b);
+      addToExcessReplicate(cur.getDatanodeDescriptor(), storedBlock);
 
       //
       // The 'excessblocks' tracks blocks until we get confirmation
@@ -3081,11 +3089,12 @@ public class BlockManager {
       //
       // The 'invalidate' list is used to inform the datanode the block 
       // should be deleted.  Items are removed from the invalidate list
-      // upon giving instructions to the namenode.
+      // upon giving instructions to the datanodes.
       //
-      addToInvalidates(b, cur.getDatanodeDescriptor());
+      final Block blockToInvalidate = getBlockToInvalidate(storedBlock, cur);
+      addToInvalidates(blockToInvalidate, cur.getDatanodeDescriptor());
       blockLog.info("BLOCK* chooseExcessReplicates: "
-                +"({}, {}) is added to invalidated blocks set", cur, b);
+                +"({}, {}) is added to invalidated blocks set", cur, storedBlock);
     }
   }
 
@@ -3110,17 +3119,18 @@ public class BlockManager {
     }
   }
 
-  private void addToExcessReplicate(DatanodeInfo dn, Block block) {
+  private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) {
     assert namesystem.hasWriteLock();
-    LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid());
+    LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+        dn.getDatanodeUuid());
     if (excessBlocks == null) {
-      excessBlocks = new LightWeightLinkedSet<Block>();
+      excessBlocks = new LightWeightLinkedSet<>();
       excessReplicateMap.put(dn.getDatanodeUuid(), excessBlocks);
     }
-    if (excessBlocks.add(block)) {
+    if (excessBlocks.add(storedBlock)) {
       excessBlocksCount.incrementAndGet();
       blockLog.debug("BLOCK* addToExcessReplicate: ({}, {}) is added to"
-          + " excessReplicateMap", dn, block);
+          + " excessReplicateMap", dn, storedBlock);
     }
   }
 
@@ -3128,14 +3138,13 @@ public class BlockManager {
    * Modify (block-->datanode) map. Possibly generate replication tasks, if the
    * removed block is still valid.
    */
-  public void removeStoredBlock(Block block, DatanodeDescriptor node) {
-    blockLog.debug("BLOCK* removeStoredBlock: {} from {}", block, node);
+  public void removeStoredBlock(BlockInfo storedBlock, DatanodeDescriptor node) {
+    blockLog.debug("BLOCK* removeStoredBlock: {} from {}", storedBlock, node);
     assert (namesystem.hasWriteLock());
     {
-      BlockInfo storedBlock = getStoredBlock(block);
       if (storedBlock == null || !blocksMap.removeNode(storedBlock, node)) {
         blockLog.debug("BLOCK* removeStoredBlock: {} has already been" +
-            " removed from node {}", block, node);
+            " removed from node {}", storedBlock, node);
         return;
       }
 
@@ -3145,7 +3154,7 @@ public class BlockManager {
       // necessary. In that case, put block on a possibly-will-
       // be-replicated list.
       //
-      BlockCollection bc = blocksMap.getBlockCollection(block);
+      BlockCollection bc = storedBlock.getBlockCollection();
       if (bc != null) {
         namesystem.decrementSafeBlockCount(storedBlock);
         updateNeededReplications(storedBlock, -1, 0);
@@ -3155,13 +3164,13 @@ public class BlockManager {
       // We've removed a block from a node, so it's definitely no longer
       // in "excess" there.
       //
-      LightWeightLinkedSet<Block> excessBlocks = excessReplicateMap.get(node
-          .getDatanodeUuid());
+      LightWeightLinkedSet<BlockInfo> excessBlocks = excessReplicateMap.get(
+          node.getDatanodeUuid());
       if (excessBlocks != null) {
-        if (excessBlocks.remove(block)) {
+        if (excessBlocks.remove(storedBlock)) {
           excessBlocksCount.decrementAndGet();
           blockLog.debug("BLOCK* removeStoredBlock: {} is removed from " +
-              "excessBlocks", block);
+              "excessBlocks", storedBlock);
           if (excessBlocks.size() == 0) {
             excessReplicateMap.remove(node.getDatanodeUuid());
           }
@@ -3169,7 +3178,7 @@ public class BlockManager {
       }
 
       // Remove the replica from corruptReplicas
-      corruptReplicas.removeFromCorruptReplicasMap(block, node);
+      corruptReplicas.removeFromCorruptReplicasMap(storedBlock, node);
     }
   }
 
@@ -3303,7 +3312,7 @@ public class BlockManager {
     for (ReceivedDeletedBlockInfo rdbi : srdb.getBlocks()) {
       switch (rdbi.getStatus()) {
       case DELETED_BLOCK:
-        removeStoredBlock(rdbi.getBlock(), node);
+        removeStoredBlock(getStoredBlock(rdbi.getBlock()), node);
         deleted++;
         break;
       case RECEIVED_BLOCK:
@@ -3354,8 +3363,8 @@ public class BlockManager {
       } else if (node.isDecommissioned()) {
         decommissioned++;
       } else {
-        LightWeightLinkedSet<Block> blocksExcess = excessReplicateMap.get(node
-            .getDatanodeUuid());
+        LightWeightLinkedSet<BlockInfo> blocksExcess = excessReplicateMap.get(
+            node.getDatanodeUuid());
         if (blocksExcess != null && blocksExcess.contains(b)) {
           excess++;
         } else {
@@ -3408,13 +3417,13 @@ public class BlockManager {
     int numOverReplicated = 0;
     while(it.hasNext()) {
       final BlockInfo block = it.next();
-      BlockCollection bc = blocksMap.getBlockCollection(block);
-      short expectedReplication = bc.getBlockReplication();
+      int expectedReplication = this.getReplication(block);
       NumberReplicas num = countNodes(block);
       int numCurrentReplica = num.liveReplicas();
       if (numCurrentReplica > expectedReplication) {
         // over-replicated block 
-        processOverReplicatedBlock(block, expectedReplication, null, null);
+        processOverReplicatedBlock(block, (short) expectedReplication, null,
+            null);
         numOverReplicated++;
       }
     }
@@ -3614,21 +3623,20 @@ public class BlockManager {
     return toInvalidate.size();
   }
 
-  boolean blockHasEnoughRacks(Block b) {
+  // TODO: update the enough rack logic for striped blocks
+  boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) {
     if (!this.shouldCheckForEnoughRacks) {
       return true;
     }
     boolean enoughRacks = false;
-    Collection<DatanodeDescriptor> corruptNodes = 
-                                  corruptReplicas.getNodes(b);
-    int numExpectedReplicas = getReplication(b);
+    Collection<DatanodeDescriptor> corruptNodes =
+        corruptReplicas.getNodes(storedBlock);
     String rackName = null;
-    for(DatanodeStorageInfo storage : blocksMap.getStorages(b)) {
+    for(DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) {
       final DatanodeDescriptor cur = storage.getDatanodeDescriptor();
       if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) {
         if ((corruptNodes == null ) || !corruptNodes.contains(cur)) {
-          if (numExpectedReplicas == 1 ||
-              (numExpectedReplicas > 1 &&
+          if (expectedStorageNum == 1 || (expectedStorageNum > 1 &&
                   !datanodeManager.hasClusterEverBeenMultiRack())) {
             enoughRacks = true;
             break;
@@ -3650,8 +3658,8 @@ public class BlockManager {
    * A block needs replication if the number of replicas is less than expected
    * or if it does not have enough racks.
    */
-  boolean isNeededReplication(Block b, int expected, int current) {
-    return current < expected || !blockHasEnoughRacks(b);
+  boolean isNeededReplication(BlockInfo storedBlock, int expected, int current) {
+    return current < expected || !blockHasEnoughRacks(storedBlock, expected);
   }
   
   public long getMissingBlocksCount() {
@@ -3823,8 +3831,7 @@ public class BlockManager {
   /**
    * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
    * to represent a task to recover a block through replication or erasure
-   * coding. Recovery is done by transferring data from {@link srcNodes} to
-   * {@link targets}
+   * coding. Recovery is done by transferring data from srcNodes to targets
    */
   private static class BlockRecoveryWork {
     protected final BlockInfo block;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e39c2b7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 546cc30..480f8c0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -100,7 +100,7 @@ public class DecommissionManager {
    * reports or other events. Before being finally marking as decommissioned,
    * another check is done with the actual block map.
    */
-  private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfoContiguous>>
+  private final TreeMap<DatanodeDescriptor, AbstractList<BlockInfo>>
       decomNodeBlocks;
 
   /**
@@ -244,12 +244,12 @@ public class DecommissionManager {
   }
 
   /**
-   * Checks whether a block is sufficiently replicated for decommissioning.
-   * Full-strength replication is not always necessary, hence "sufficient".
+   * Checks whether a block is sufficiently replicated/stored for
+   * decommissioning. For replicated blocks or striped blocks, full-strength
+   * replication or storage is not always necessary, hence "sufficient".
    * @return true if sufficient, else false.
    */
-  private boolean isSufficientlyReplicated(BlockInfoContiguous block, 
-      BlockCollection bc,
+  private boolean isSufficient(BlockInfo block, BlockCollection bc,
       NumberReplicas numberReplicas) {
     final int numExpected = bc.getBlockReplication();
     final int numLive = numberReplicas.liveReplicas();
@@ -265,18 +265,19 @@ public class DecommissionManager {
     if (numExpected > numLive) {
       if (bc.isUnderConstruction() && block.equals(bc.getLastBlock())) {
         // Can decom a UC block as long as there will still be minReplicas
-        if (numLive >= blockManager.minReplication) {
+        if (blockManager.hasMinStorage(block, numLive)) {
           LOG.trace("UC block {} sufficiently-replicated since numLive ({}) "
-              + ">= minR ({})", block, numLive, blockManager.minReplication);
+              + ">= minR ({})", block, numLive,
+              blockManager.getMinStorageNum(block));
           return true;
         } else {
           LOG.trace("UC block {} insufficiently-replicated since numLive "
               + "({}) < minR ({})", block, numLive,
-              blockManager.minReplication);
+              blockManager.getMinStorageNum(block));
         }
       } else {
         // Can decom a non-UC as long as the default replication is met
-        if (numLive >= blockManager.defaultReplication) {
+        if (numLive >= blockManager.getDefaultStorageNum(block)) {
           return true;
         }
       }
@@ -412,7 +413,7 @@ public class DecommissionManager {
     }
 
     private void check() {
-      final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfoContiguous>>>
+      final Iterator<Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>>
           it = new CyclicIteration<>(decomNodeBlocks, iterkey).iterator();
       final LinkedList<DatanodeDescriptor> toRemove = new LinkedList<>();
 
@@ -420,10 +421,10 @@ public class DecommissionManager {
           && !exceededNumBlocksPerCheck()
           && !exceededNumNodesPerCheck()) {
         numNodesChecked++;
-        final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfoContiguous>>
+        final Map.Entry<DatanodeDescriptor, AbstractList<BlockInfo>>
             entry = it.next();
         final DatanodeDescriptor dn = entry.getKey();
-        AbstractList<BlockInfoContiguous> blocks = entry.getValue();
+        AbstractList<BlockInfo> blocks = entry.getValue();
         boolean fullScan = false;
         if (blocks == null) {
           // This is a newly added datanode, run through its list to schedule 
@@ -431,14 +432,14 @@ public class DecommissionManager {
           // that are insufficiently replicated for further tracking
           LOG.debug("Newly-added node {}, doing full scan to find " +
               "insufficiently-replicated blocks.", dn);
-          blocks = handleInsufficientlyReplicated(dn);
+          blocks = handleInsufficientlyStored(dn);
           decomNodeBlocks.put(dn, blocks);
           fullScan = true;
         } else {
           // This is a known datanode, check if its # of insufficiently 
           // replicated blocks has dropped to zero and if it can be decommed
           LOG.debug("Processing decommission-in-progress node {}", dn);
-          pruneSufficientlyReplicated(dn, blocks);
+          pruneReliableBlocks(dn, blocks);
         }
         if (blocks.size() == 0) {
           if (!fullScan) {
@@ -450,7 +451,7 @@ public class DecommissionManager {
             // marking the datanode as decommissioned 
             LOG.debug("Node {} has finished replicating current set of "
                 + "blocks, checking with the full block map.", dn);
-            blocks = handleInsufficientlyReplicated(dn);
+            blocks = handleInsufficientlyStored(dn);
             decomNodeBlocks.put(dn, blocks);
           }
           // If the full scan is clean AND the node liveness is okay, 
@@ -491,27 +492,25 @@ public class DecommissionManager {
     }
 
     /**
-     * Removes sufficiently replicated blocks from the block list of a 
-     * datanode.
+     * Removes reliable blocks from the block list of a datanode.
      */
-    private void pruneSufficientlyReplicated(final DatanodeDescriptor datanode,
-        AbstractList<BlockInfoContiguous> blocks) {
+    private void pruneReliableBlocks(final DatanodeDescriptor datanode,
+        AbstractList<BlockInfo> blocks) {
       processBlocksForDecomInternal(datanode, blocks.iterator(), null, true);
     }
 
     /**
-     * Returns a list of blocks on a datanode that are insufficiently 
-     * replicated, i.e. are under-replicated enough to prevent decommission.
+     * Returns a list of blocks on a datanode that are insufficiently replicated
+     * or require recovery, i.e. requiring recovery and should prevent
+     * decommission.
      * <p/>
-     * As part of this, it also schedules replication work for 
-     * any under-replicated blocks.
+     * As part of this, it also schedules replication/recovery work.
      *
-     * @param datanode
-     * @return List of insufficiently replicated blocks 
+     * @return List of blocks requiring recovery
      */
-    private AbstractList<BlockInfoContiguous> handleInsufficientlyReplicated(
+    private AbstractList<BlockInfo> handleInsufficientlyStored(
         final DatanodeDescriptor datanode) {
-      AbstractList<BlockInfoContiguous> insufficient = new ChunkedArrayList<>();
+      AbstractList<BlockInfo> insufficient = new ChunkedArrayList<>();
       processBlocksForDecomInternal(datanode, datanode.getBlockIterator(),
           insufficient, false);
       return insufficient;
@@ -520,24 +519,22 @@ public class DecommissionManager {
     /**
      * Used while checking if decommission-in-progress datanodes can be marked
      * as decommissioned. Combines shared logic of 
-     * pruneSufficientlyReplicated and handleInsufficientlyReplicated.
+     * pruneReliableBlocks and handleInsufficientlyStored.
      *
      * @param datanode                    Datanode
      * @param it                          Iterator over the blocks on the
      *                                    datanode
-     * @param insufficientlyReplicated    Return parameter. If it's not null,
+     * @param insufficientList            Return parameter. If it's not null,
      *                                    will contain the insufficiently
      *                                    replicated-blocks from the list.
-     * @param pruneSufficientlyReplicated whether to remove sufficiently
-     *                                    replicated blocks from the iterator
-     * @return true if there are under-replicated blocks in the provided block
-     * iterator, else false.
+     * @param pruneReliableBlocks         whether to remove blocks reliable
+     *                                    enough from the iterator
      */
     private void processBlocksForDecomInternal(
         final DatanodeDescriptor datanode,
-        final Iterator<? extends BlockInfo> it,
-        final List<BlockInfoContiguous> insufficientlyReplicated,
-        boolean pruneSufficientlyReplicated) {
+        final Iterator<BlockInfo> it,
+        final List<BlockInfo> insufficientList,
+        boolean pruneReliableBlocks) {
       boolean firstReplicationLog = true;
       int underReplicatedBlocks = 0;
       int decommissionOnlyReplicas = 0;
@@ -552,7 +549,7 @@ public class DecommissionManager {
           it.remove();
           continue;
         }
-        BlockCollection bc = blockManager.blocksMap.getBlockCollection(block);
+        BlockCollection bc = blockManager.getBlockCollection(block);
         if (bc == null) {
           // Orphan block, will be invalidated eventually. Skip.
           continue;
@@ -560,7 +557,6 @@ public class DecommissionManager {
 
         final NumberReplicas num = blockManager.countNodes(block);
         final int liveReplicas = num.liveReplicas();
-        final int curReplicas = liveReplicas;
 
         // Schedule under-replicated blocks for replication if not already
         // pending
@@ -571,7 +567,7 @@ public class DecommissionManager {
               namesystem.isPopulatingReplQueues()) {
             // Process these blocks only when active NN is out of safe mode.
             blockManager.neededReplications.add(block,
-                curReplicas,
+                liveReplicas,
                 num.decommissionedAndDecommissioning(),
                 bc.getBlockReplication());
           }
@@ -579,17 +575,16 @@ public class DecommissionManager {
 
         // Even if the block is under-replicated, 
         // it doesn't block decommission if it's sufficiently replicated
-        BlockInfoContiguous blk = (BlockInfoContiguous) block;
-        if (isSufficientlyReplicated(blk, bc, num)) {
-          if (pruneSufficientlyReplicated) {
+        if (isSufficient(block, bc, num)) {
+          if (pruneReliableBlocks) {
             it.remove();
           }
           continue;
         }
 
         // We've found an insufficiently replicated block.
-        if (insufficientlyReplicated != null) {
-          insufficientlyReplicated.add(blk);
+        if (insufficientList != null) {
+          insufficientList.add(block);
         }
         // Log if this is our first time through
         if (firstReplicationLog) {
@@ -602,7 +597,7 @@ public class DecommissionManager {
         if (bc.isUnderConstruction()) {
           underReplicatedInOpenFiles++;
         }
-        if ((curReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
+        if ((liveReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) {
           decommissionOnlyReplicas++;
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e39c2b7/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 76ac2aa..5fd956e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -3803,7 +3803,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       if (trackBlockCounts) {
         if (b.isComplete()) {
           numRemovedComplete++;
-          if (blockManager.checkMinStorage(b, b.numNodes())) {
+          if (blockManager.hasMinStorage(b, b.numNodes())) {
             numRemovedSafe++;
           }
         }
@@ -4032,7 +4032,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
         break;
-      assert blockManager.checkMinStorage(curBlock) :
+      assert blockManager.hasMinStorage(curBlock) :
               "A COMPLETE block is not minimally replicated in " + src;
     }
 
@@ -4068,7 +4068,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     // If penultimate block doesn't exist then its minReplication is met
     boolean penultimateBlockMinStorage = penultimateBlock == null ||
-        blockManager.checkMinStorage(penultimateBlock);
+        blockManager.hasMinStorage(penultimateBlock);
 
     switch(lastBlockState) {
     case COMPLETE:
@@ -4077,7 +4077,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     case COMMITTED:
       // Close file if committed blocks are minimally replicated
       if(penultimateBlockMinStorage &&
-          blockManager.checkMinStorage(lastBlock)) {
+          blockManager.hasMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId());
         NameNode.stateChangeLog.warn("BLOCK*"

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e39c2b7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
index 1c3f075..c33667d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNodeCount.java
@@ -100,7 +100,7 @@ public class TestNodeCount {
       DatanodeDescriptor nonExcessDN = null;
       for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) {
         final DatanodeDescriptor dn = storage.getDatanodeDescriptor();
-        Collection<Block> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
+        Collection<BlockInfo> blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid());
         if (blocks == null || !blocks.contains(block.getLocalBlock()) ) {
           nonExcessDN = dn;
           break;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2e39c2b7/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
index 2d7bb44..83b3aa0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestOverReplicatedBlocks.java
@@ -34,7 +34,6 @@ import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
 import org.apache.hadoop.hdfs.MiniDFSCluster.DataNodeProperties;
-import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
@@ -42,7 +41,6 @@ import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
-import org.apache.hadoop.util.Time;
 import org.junit.Test;
 
 public class TestOverReplicatedBlocks {
@@ -185,7 +183,7 @@ public class TestOverReplicatedBlocks {
       // All replicas for deletion should be scheduled on lastDN.
       // And should not actually be deleted, because lastDN does not heartbeat.
       namesystem.readLock();
-      Collection<Block> dnBlocks = 
+      Collection<BlockInfo> dnBlocks =
         namesystem.getBlockManager().excessReplicateMap.get(lastDNid);
       assertEquals("Replicas on node " + lastDNid + " should have been deleted",
           SMALL_FILE_LENGTH / SMALL_BLOCK_SIZE, dnBlocks.size());


[48/50] [abbrv] hadoop git commit: HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all ECSchemas loaded in Namenode. (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all ECSchemas loaded in Namenode. (Contributed by Vinayakumar B)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/b155a2e3
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/b155a2e3
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/b155a2e3

Branch: refs/heads/HDFS-7285
Commit: b155a2e395916fea7b822879a591fc0faab18af9
Parents: e006f68
Author: Vinayakumar B <vi...@apache.org>
Authored: Fri Apr 10 15:07:32 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:58 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/io/erasurecode/ECSchema.java  |  4 +-
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 +-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 11 ++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 10 ++++
 ...tNamenodeProtocolServerSideTranslatorPB.java | 19 +++++++
 .../ClientNamenodeProtocolTranslatorPB.java     | 26 ++++++++-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  5 +-
 .../hdfs/server/namenode/FSNamesystem.java      | 17 ++++++
 .../hdfs/server/namenode/NameNodeRpcServer.java |  9 +++-
 .../src/main/proto/ClientNamenodeProtocol.proto |  9 ++++
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |  3 +-
 .../org/apache/hadoop/hdfs/TestECSchemas.java   | 57 ++++++++++++++++++++
 12 files changed, 164 insertions(+), 11 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
index 8c3310e..32077f6 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -123,12 +123,12 @@ public final class ECSchema {
 
     this.chunkSize = DEFAULT_CHUNK_SIZE;
     try {
-      if (options.containsKey(CHUNK_SIZE_KEY)) {
+      if (this.options.containsKey(CHUNK_SIZE_KEY)) {
         this.chunkSize = Integer.parseInt(options.get(CHUNK_SIZE_KEY));
       }
     } catch (NumberFormatException e) {
       throw new IllegalArgumentException("Option value " +
-          options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          this.options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
           " is found. It should be an integer");
     }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 753795a..5250dfa 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -58,4 +58,7 @@
 
     HDFS-8104. Make hard-coded values consistent with the system default schema first before remove them. (Kai Zheng)
 
-    HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
\ No newline at end of file
+    HDFS-7889. Subclass DFSOutputStream to support writing striping layout files. (Li Bo via Kai Zheng)
+
+    HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all
+    ECSchemas loaded in Namenode. (vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index b2a69dd..6339d30 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -163,6 +163,7 @@ import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.MD5Hash;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.retry.LossyRetryInvocationHandler;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RemoteException;
@@ -3105,6 +3106,16 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public ECSchema[] getECSchemas() throws IOException {
+    checkOpen();
+    TraceScope scope = Trace.startSpan("getECSchemas", traceSampler);
+    try {
+      return namenode.getECSchemas();
+    } finally {
+      scope.close();
+    }
+  }
+
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
     return new DFSInotifyEventInputStream(traceSampler, namenode);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 45d92f3..7f5ac49 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.retry.AtMostOnce;
 import org.apache.hadoop.io.retry.Idempotent;
 import org.apache.hadoop.security.AccessControlException;
@@ -1474,4 +1475,13 @@ public interface ClientProtocol {
    */
   @Idempotent
   public ECInfo getErasureCodingInfo(String src) throws IOException;
+
+  /**
+   * Gets list of ECSchemas loaded in Namenode
+   *
+   * @return Returns the list of ECSchemas loaded at Namenode
+   * @throws IOException
+   */
+  @Idempotent
+  public ECSchema[] getECSchemas() throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index 1493e52..33207ab 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -106,6 +106,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
@@ -218,6 +220,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifie
 import org.apache.hadoop.hdfs.server.namenode.INodeId;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto;
 import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto;
 import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto;
@@ -1530,4 +1533,20 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetECSchemasResponseProto getECSchemas(RpcController controller,
+      GetECSchemasRequestProto request) throws ServiceException {
+    try {
+      ECSchema[] ecSchemas = server.getECSchemas();
+      GetECSchemasResponseProto.Builder resBuilder = GetECSchemasResponseProto
+          .newBuilder();
+      for (ECSchema ecSchema : ecSchemas) {
+        resBuilder.addSchemas(PBHelper.convertECSchema(ecSchema));
+      }
+      return resBuilder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 568da68..0211522 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -107,6 +107,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDataEncryptionKeyResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetECSchemasResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
@@ -165,10 +167,11 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Update
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SetStoragePolicyRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.CreateErasureCodingZoneResponseProto;
-import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos;
+import org.apache.hadoop.hdfs.protocol.proto.*;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.CreateEncryptionZoneRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.GetXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.ListXAttrsRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.XAttrProtos.RemoveXAttrRequestProto;
@@ -180,6 +183,7 @@ import org.apache.hadoop.hdfs.server.namenode.SafeModeException;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.ProtobufHelper;
 import org.apache.hadoop.ipc.ProtocolMetaInterface;
 import org.apache.hadoop.ipc.ProtocolTranslator;
@@ -237,6 +241,10 @@ public class ClientNamenodeProtocolTranslatorPB implements
   VOID_GET_STORAGE_POLICIES_REQUEST =
       GetStoragePoliciesRequestProto.newBuilder().build();
 
+  private final static GetECSchemasRequestProto
+  VOID_GET_ECSCHEMAS_REQUEST = GetECSchemasRequestProto
+      .newBuilder().build();
+
   public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) {
     rpcProxy = proxy;
   }
@@ -1550,4 +1558,20 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public ECSchema[] getECSchemas() throws IOException {
+    try {
+      GetECSchemasResponseProto response = rpcProxy.getECSchemas(null,
+          VOID_GET_ECSCHEMAS_REQUEST);
+      ECSchema[] schemas = new ECSchema[response.getSchemasCount()];
+      int i = 0;
+      for (ECSchemaProto schemaProto : response.getSchemasList()) {
+        schemas[i++] = PBHelper.convertECSchema(schemaProto);
+      }
+      return schemas;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index afdb302..3d0e9bd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -3122,8 +3122,6 @@ public class PBHelper {
     for (ECSchemaOptionEntryProto option : optionsList) {
       options.put(option.getKey(), option.getValue());
     }
-    // include chunksize in options.
-    options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(schema.getChunkSize()));
     return new ECSchema(schema.getSchemaName(), schema.getCodecName(),
         schema.getDataUnits(), schema.getParityUnits(), options);
   }
@@ -3133,8 +3131,7 @@ public class PBHelper {
         .setSchemaName(schema.getSchemaName())
         .setCodecName(schema.getCodecName())
         .setDataUnits(schema.getNumDataUnits())
-        .setParityUnits(schema.getNumParityUnits())
-        .setChunkSize(schema.getChunkSize());
+        .setParityUnits(schema.getNumParityUnits());
     Set<Entry<String, String>> entrySet = schema.getOptions().entrySet();
     for (Entry<String, String> entry : entrySet) {
       builder.addOptions(ECSchemaOptionEntryProto.newBuilder()

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 4c86bd3..a504907 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -8169,6 +8169,23 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return null;
   }
 
+  /**
+   * Get available ECSchemas
+   */
+  ECSchema[] getECSchemas() throws IOException {
+    checkOperation(OperationCategory.READ);
+    waitForLoadingFSImage();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      // TODO HDFS-7866 Need to return all schemas maintained by Namenode
+      ECSchema defaultSchema = ECSchemaManager.getSystemDefaultSchema();
+      return new ECSchema[] { defaultSchema };
+    } finally {
+      readUnlock();
+    }
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index fe9b901..85ac1e5 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -142,6 +142,7 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReport;
 import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.ProtobufRpcEngine;
 import org.apache.hadoop.ipc.RPC;
 import org.apache.hadoop.ipc.RetryCache;
@@ -2044,9 +2045,15 @@ class NameNodeRpcServer implements NamenodeProtocols {
     nn.spanReceiverHost.removeSpanReceiver(id);
   }
 
-  @Override // ClientNameNodeProtocol
+  @Override // ClientProtocol
   public ECInfo getErasureCodingInfo(String src) throws IOException {
     checkNNStartup();
     return namesystem.getErasureCodingInfo(src);
   }
+
+  @Override // ClientProtocol
+  public ECSchema[] getECSchemas() throws IOException {
+    checkNNStartup();
+    return namesystem.getECSchemas();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index 9488aed..3389a22 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -729,6 +729,13 @@ message GetErasureCodingInfoResponseProto {
   optional ECInfoProto ECInfo = 1;
 }
 
+message GetECSchemasRequestProto { // void request
+}
+
+message GetECSchemasResponseProto {
+  repeated ECSchemaProto schemas = 1;
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -879,4 +886,6 @@ service ClientNamenodeProtocol {
       returns(GetEditsFromTxidResponseProto);
   rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto)
       returns(GetErasureCodingInfoResponseProto);
+  rpc getECSchemas(GetECSchemasRequestProto)
+      returns(GetECSchemasResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 1314ea0..0507538 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -637,8 +637,7 @@ message ECSchemaProto {
   required string codecName = 2;
   required uint32 dataUnits = 3;
   required uint32 parityUnits = 4;
-  required uint32 chunkSize = 5;
-  repeated ECSchemaOptionEntryProto options = 6;
+  repeated ECSchemaOptionEntryProto options = 5;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/b155a2e3/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
new file mode 100644
index 0000000..07e1359
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestECSchemas.java
@@ -0,0 +1,57 @@
+/**
+ * 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 static org.junit.Assert.*;
+
+import java.io.IOException;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.hdfs.server.namenode.ECSchemaManager;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.junit.After;
+import org.junit.Before;
+import org.junit.Test;
+
+public class TestECSchemas {
+  private MiniDFSCluster cluster;
+
+  @Before
+  public void before() throws IOException {
+    cluster = new MiniDFSCluster.Builder(new Configuration()).numDataNodes(0)
+        .build();
+    cluster.waitActive();
+  }
+
+  @After
+  public void after() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testGetECSchemas() throws Exception {
+    ECSchema[] ecSchemas = cluster.getFileSystem().getClient().getECSchemas();
+    // TODO update assertion after HDFS-7866
+    assertNotNull(ecSchemas);
+    assertEquals("Should have only one ecSchema", 1, ecSchemas.length);
+    assertEquals("Returned schemas should have only default schema",
+        ECSchemaManager.getSystemDefaultSchema(), ecSchemas[0]);
+  }
+}


[19/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435 and HDFS-7930 (this commit is for HDFS-7930 only)

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435 and HDFS-7930 (this commit is for HDFS-7930 only)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/43df926c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/43df926c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/43df926c

Branch: refs/heads/HDFS-7285
Commit: 43df926ce8db5d0df7632356ea5fa85ede67edf9
Parents: 9c6e1f1
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Mar 23 11:25:40 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:51 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/server/blockmanagement/BlockManager.java      | 7 ++++---
 .../org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java  | 7 ++++---
 .../org/apache/hadoop/hdfs/server/namenode/INodeFile.java     | 2 +-
 3 files changed, 9 insertions(+), 7 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df926c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 8ea783f..4f81e3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -2117,17 +2117,18 @@ public class BlockManager {
    * Mark block replicas as corrupt except those on the storages in 
    * newStorages list.
    */
-  public void markBlockReplicasAsCorrupt(BlockInfoContiguous block, 
+  public void markBlockReplicasAsCorrupt(Block oldBlock,
+      BlockInfo block,
       long oldGenerationStamp, long oldNumBytes, 
       DatanodeStorageInfo[] newStorages) throws IOException {
     assert namesystem.hasWriteLock();
     BlockToMarkCorrupt b = null;
     if (block.getGenerationStamp() != oldGenerationStamp) {
-      b = new BlockToMarkCorrupt(block, oldGenerationStamp,
+      b = new BlockToMarkCorrupt(oldBlock, block, oldGenerationStamp,
           "genstamp does not match " + oldGenerationStamp
           + " : " + block.getGenerationStamp(), Reason.GENSTAMP_MISMATCH);
     } else if (block.getNumBytes() != oldNumBytes) {
-      b = new BlockToMarkCorrupt(block,
+      b = new BlockToMarkCorrupt(oldBlock, block,
           "length does not match " + oldNumBytes
           + " : " + block.getNumBytes(), Reason.SIZE_MISMATCH);
     } else {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df926c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index f1ccb1a..8398ea7 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -2794,7 +2794,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
   /** Compute quota change for converting a complete block to a UC block */
   private QuotaCounts computeQuotaDeltaForUCBlock(INodeFile file) {
     final QuotaCounts delta = new QuotaCounts.Builder().build();
-    final BlockInfoContiguous lastBlock = file.getLastBlock();
+    final BlockInfo lastBlock = file.getLastBlock();
     if (lastBlock != null) {
       final long diff = file.getPreferredBlockSize() - lastBlock.getNumBytes();
       final short repl = file.getBlockReplication();
@@ -4383,8 +4383,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         } else {
           iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos);
           if (closeFile) {
-            blockManager.markBlockReplicasAsCorrupt(storedBlock,
-                oldGenerationStamp, oldNumBytes, trimmedStorageInfos);
+            blockManager.markBlockReplicasAsCorrupt(oldBlock.getLocalBlock(),
+                storedBlock, oldGenerationStamp, oldNumBytes,
+                trimmedStorageInfos);
           }
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/43df926c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 97bbb9d..b40022e8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -985,7 +985,7 @@ public class INodeFile extends INodeWithAdditionalFields
    * @return the quota usage delta (not considering replication factor)
    */
   long computeQuotaDeltaForTruncate(final long newLength) {
-    final BlockInfoContiguous[] blocks = getBlocks();
+    final BlockInfo[] blocks = getBlocks();
     if (blocks == null || blocks.length == 0) {
       return 0;
     }


[08/50] [abbrv] hadoop git commit: HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from striped files. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from striped files. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/94a1ea4c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/94a1ea4c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/94a1ea4c

Branch: refs/heads/HDFS-7285
Commit: 94a1ea4c4eecd35b98485a3b2d5d72ae5161987c
Parents: d542a3d
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 9 14:59:58 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:41:52 2015 -0700

----------------------------------------------------------------------
 .../hadoop/hdfs/protocol/LocatedBlock.java      |   5 +-
 .../hdfs/protocol/LocatedStripedBlock.java      |  68 +++++++++
 ...tNamenodeProtocolServerSideTranslatorPB.java |  14 +-
 .../ClientNamenodeProtocolTranslatorPB.java     |  13 +-
 .../DatanodeProtocolClientSideTranslatorPB.java |   2 +-
 .../DatanodeProtocolServerSideTranslatorPB.java |   2 +-
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java |  80 +++++++----
 .../blockmanagement/BlockInfoStriped.java       |   5 +
 .../BlockInfoStripedUnderConstruction.java      |  99 +++++++------
 .../server/blockmanagement/BlockManager.java    |  51 ++++---
 .../blockmanagement/DatanodeDescriptor.java     |   4 +-
 .../blockmanagement/DatanodeStorageInfo.java    |   3 +-
 .../server/namenode/FSImageFormatPBINode.java   |  21 +--
 .../hdfs/server/namenode/FSNamesystem.java      |  35 +++--
 .../hadoop-hdfs/src/main/proto/hdfs.proto       |   1 +
 .../org/apache/hadoop/hdfs/DFSTestUtil.java     |  12 ++
 .../hadoop/hdfs/protocolPB/TestPBHelper.java    |  16 +--
 .../datanode/TestIncrementalBrVariations.java   |  14 +-
 .../server/namenode/TestAddStripedBlocks.java   | 141 +++++++++++++++++++
 .../hdfs/server/namenode/TestFSImage.java       |   5 +-
 20 files changed, 444 insertions(+), 147 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
index e729869..a38e8f2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java
@@ -25,7 +25,6 @@ import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
-import org.apache.hadoop.hdfs.protocol.DatanodeInfoWithStorage;
 import org.apache.hadoop.security.token.Token;
 
 import com.google.common.collect.Lists;
@@ -51,14 +50,14 @@ public class LocatedBlock {
   // else false. If block has few corrupt replicas, they are filtered and 
   // their locations are not part of this object
   private boolean corrupt;
-  private Token<BlockTokenIdentifier> blockToken = new Token<BlockTokenIdentifier>();
+  private Token<BlockTokenIdentifier> blockToken = new Token<>();
   /**
    * List of cached datanode locations
    */
   private DatanodeInfo[] cachedLocs;
 
   // Used when there are no locations
-  private static final DatanodeInfoWithStorage[] EMPTY_LOCS =
+  static final DatanodeInfoWithStorage[] EMPTY_LOCS =
       new DatanodeInfoWithStorage[0];
 
   public LocatedBlock(ExtendedBlock b, DatanodeInfo[] locs) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
new file mode 100644
index 0000000..97e3a69
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java
@@ -0,0 +1,68 @@
+/**
+ * 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.protocol;
+
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.fs.StorageType;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+
+import java.util.Arrays;
+
+/**
+ * {@link LocatedBlock} with striped block support. For a striped block, each
+ * datanode storage is associated with a block in the block group. We need to
+ * record the index (in the striped block group) for each of them.
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class LocatedStripedBlock extends LocatedBlock {
+  private int[] blockIndices;
+
+  public LocatedStripedBlock(ExtendedBlock b, DatanodeInfo[] locs,
+      String[] storageIDs, StorageType[] storageTypes, int[] indices,
+      long startOffset, boolean corrupt, DatanodeInfo[] cachedLocs) {
+    super(b, locs, storageIDs, storageTypes, startOffset, corrupt, cachedLocs);
+    assert indices != null && indices.length == locs.length;
+    this.blockIndices = new int[indices.length];
+    System.arraycopy(indices, 0, blockIndices, 0, indices.length);
+  }
+
+  public LocatedStripedBlock(ExtendedBlock b, DatanodeStorageInfo[] storages,
+      int[] indices, long startOffset, boolean corrupt) {
+    this(b, DatanodeStorageInfo.toDatanodeInfos(storages),
+        DatanodeStorageInfo.toStorageIDs(storages),
+        DatanodeStorageInfo.toStorageTypes(storages), indices,
+        startOffset, corrupt, EMPTY_LOCS);
+  }
+
+  @Override
+  public String toString() {
+    return getClass().getSimpleName() + "{" + getBlock()
+        + "; getBlockSize()=" + getBlockSize()
+        + "; corrupt=" + isCorrupt()
+        + "; offset=" + getStartOffset()
+        + "; locs=" + Arrays.asList(getLocations())
+        + "; indices=" + Arrays.asList(blockIndices)
+        + "}";
+  }
+
+  public int[] getBlockIndices() {
+    return this.blockIndices;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index e26158b..cf51e3a 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -419,7 +419,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           req.getClientName(), flags);
       AppendResponseProto.Builder builder = AppendResponseProto.newBuilder();
       if (result.getLastBlock() != null) {
-        builder.setBlock(PBHelper.convert(result.getLastBlock()));
+        builder.setBlock(PBHelper.convertLocatedBlock(result.getLastBlock()));
       }
       if (result.getFileStatus() != null) {
         builder.setStat(PBHelper.convert(result.getFileStatus()));
@@ -495,7 +495,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
           (favor == null || favor.size() == 0) ? null : favor
               .toArray(new String[favor.size()]));
       return AddBlockResponseProto.newBuilder()
-          .setBlock(PBHelper.convert(result)).build();
+          .setBlock(PBHelper.convertLocatedBlock(result)).build();
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -519,7 +519,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
               new DatanodeInfoProto[excludesList.size()])), 
           req.getNumAdditionalNodes(), req.getClientName());
       return GetAdditionalDatanodeResponseProto.newBuilder().setBlock(
-          PBHelper.convert(result))
+          PBHelper.convertLocatedBlock(result))
           .build();
     } catch (IOException e) {
       throw new ServiceException(e);
@@ -545,8 +545,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       ReportBadBlocksRequestProto req) throws ServiceException {
     try {
       List<LocatedBlockProto> bl = req.getBlocksList();
-      server.reportBadBlocks(PBHelper.convertLocatedBlock(
-              bl.toArray(new LocatedBlockProto[bl.size()])));
+      server.reportBadBlocks(PBHelper.convertLocatedBlocks(
+          bl.toArray(new LocatedBlockProto[bl.size()])));
     } catch (IOException e) {
       throw new ServiceException(e);
     }
@@ -950,8 +950,8 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       RpcController controller, UpdateBlockForPipelineRequestProto req)
       throws ServiceException {
     try {
-      LocatedBlockProto result = PBHelper.convert(server
-          .updateBlockForPipeline(PBHelper.convert(req.getBlock()),
+      LocatedBlockProto result = PBHelper.convertLocatedBlock(
+          server.updateBlockForPipeline(PBHelper.convert(req.getBlock()),
               req.getClientName()));
       return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result)
           .build();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 4ec6f9e..22da083 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -327,7 +327,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
     try {
       AppendResponseProto res = rpcProxy.append(null, req);
       LocatedBlock lastBlock = res.hasBlock() ? PBHelper
-          .convert(res.getBlock()) : null;
+          .convertLocatedBlockProto(res.getBlock()) : null;
       HdfsFileStatus stat = (res.hasStat()) ? PBHelper.convert(res.getStat())
           : null;
       return new LastBlockWithStatus(lastBlock, stat);
@@ -415,7 +415,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
       req.addAllFavoredNodes(Arrays.asList(favoredNodes));
     }
     try {
-      return PBHelper.convert(rpcProxy.addBlock(null, req.build()).getBlock());
+      return PBHelper.convertLocatedBlockProto(
+          rpcProxy.addBlock(null, req.build()).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -440,8 +441,8 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setClientName(clientName)
         .build();
     try {
-      return PBHelper.convert(rpcProxy.getAdditionalDatanode(null, req)
-          .getBlock());
+      return PBHelper.convertLocatedBlockProto(
+          rpcProxy.getAdditionalDatanode(null, req).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);
     }
@@ -468,7 +469,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
   @Override
   public void reportBadBlocks(LocatedBlock[] blocks) throws IOException {
     ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder()
-        .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlock(blocks)))
+        .addAllBlocks(Arrays.asList(PBHelper.convertLocatedBlocks(blocks)))
         .build();
     try {
       rpcProxy.reportBadBlocks(null, req);
@@ -900,7 +901,7 @@ public class ClientNamenodeProtocolTranslatorPB implements
         .setClientName(clientName)
         .build();
     try {
-      return PBHelper.convert(
+      return PBHelper.convertLocatedBlockProto(
           rpcProxy.updateBlockForPipeline(null, req).getBlock());
     } catch (ServiceException e) {
       throw ProtobufHelper.getRemoteException(e);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
index 825e835..2db70d1 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolClientSideTranslatorPB.java
@@ -279,7 +279,7 @@ public class DatanodeProtocolClientSideTranslatorPB implements
     ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto
         .newBuilder();
     for (int i = 0; i < blocks.length; i++) {
-      builder.addBlocks(i, PBHelper.convert(blocks[i]));
+      builder.addBlocks(i, PBHelper.convertLocatedBlock(blocks[i]));
     }
     ReportBadBlocksRequestProto req = builder.build();
     try {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
index 873eb6d..19b2f83 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/DatanodeProtocolServerSideTranslatorPB.java
@@ -258,7 +258,7 @@ public class DatanodeProtocolServerSideTranslatorPB implements
     List<LocatedBlockProto> lbps = request.getBlocksList();
     LocatedBlock [] blocks = new LocatedBlock [lbps.size()];
     for(int i=0; i<lbps.size(); i++) {
-      blocks[i] = PBHelper.convert(lbps.get(i));
+      blocks[i] = PBHelper.convertLocatedBlockProto(lbps.get(i));
     }
     try {
       impl.reportBadBlocks(blocks);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index 14410e6..cda708f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -80,6 +80,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus;
 import org.apache.hadoop.hdfs.protocol.SnapshotDiffReport;
@@ -628,7 +629,7 @@ public class PBHelper {
     if (b == null) {
       return null;
     }
-    LocatedBlockProto lb = PBHelper.convert((LocatedBlock)b);
+    LocatedBlockProto lb = PBHelper.convertLocatedBlock(b);
     RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder();
     builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp());
     if(b.getNewBlock() != null)
@@ -778,7 +779,7 @@ public class PBHelper {
     }
   }
   
-  public static LocatedBlockProto convert(LocatedBlock b) {
+  public static LocatedBlockProto convertLocatedBlock(LocatedBlock b) {
     if (b == null) return null;
     Builder builder = LocatedBlockProto.newBuilder();
     DatanodeInfo[] locs = b.getLocations();
@@ -799,21 +800,27 @@ public class PBHelper {
 
     StorageType[] storageTypes = b.getStorageTypes();
     if (storageTypes != null) {
-      for (int i = 0; i < storageTypes.length; ++i) {
-        builder.addStorageTypes(PBHelper.convertStorageType(storageTypes[i]));
+      for (StorageType storageType : storageTypes) {
+        builder.addStorageTypes(PBHelper.convertStorageType(storageType));
       }
     }
     final String[] storageIDs = b.getStorageIDs();
     if (storageIDs != null) {
       builder.addAllStorageIDs(Arrays.asList(storageIDs));
     }
+    if (b instanceof LocatedStripedBlock) {
+      int[] indices = ((LocatedStripedBlock) b).getBlockIndices();
+      for (int index : indices) {
+        builder.addBlockIndex(index);
+      }
+    }
 
     return builder.setB(PBHelper.convert(b.getBlock()))
         .setBlockToken(PBHelper.convert(b.getBlockToken()))
         .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build();
   }
   
-  public static LocatedBlock convert(LocatedBlockProto proto) {
+  public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) {
     if (proto == null) return null;
     List<DatanodeInfoProto> locs = proto.getLocsList();
     DatanodeInfo[] targets = new DatanodeInfo[locs.size()];
@@ -833,6 +840,15 @@ public class PBHelper {
       storageIDs = proto.getStorageIDsList().toArray(new String[storageIDsCount]);
     }
 
+    int[] indices = null;
+    final int indexCount = proto.getBlockIndexCount();
+    if (indexCount > 0) {
+      indices = new int[indexCount];
+      for (int i = 0; i < indexCount; i++) {
+        indices[i] = proto.getBlockIndex(i);
+      }
+    }
+
     // Set values from the isCached list, re-using references from loc
     List<DatanodeInfo> cachedLocs = new ArrayList<DatanodeInfo>(locs.size());
     List<Boolean> isCachedList = proto.getIsCachedList();
@@ -842,9 +858,17 @@ public class PBHelper {
       }
     }
 
-    LocatedBlock lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets,
-        storageIDs, storageTypes, proto.getOffset(), proto.getCorrupt(),
-        cachedLocs.toArray(new DatanodeInfo[0]));
+    final LocatedBlock lb;
+    if (indices == null) {
+      lb = new LocatedBlock(PBHelper.convert(proto.getB()), targets, storageIDs,
+          storageTypes, proto.getOffset(), proto.getCorrupt(),
+          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
+    } else {
+      lb = new LocatedStripedBlock(PBHelper.convert(proto.getB()), targets,
+          storageIDs, storageTypes, indices, proto.getOffset(),
+          proto.getCorrupt(),
+          cachedLocs.toArray(new DatanodeInfo[cachedLocs.size()]));
+    }
     lb.setBlockToken(PBHelper.convert(proto.getBlockToken()));
 
     return lb;
@@ -1260,36 +1284,36 @@ public class PBHelper {
   }
   
   // Located Block Arrays and Lists
-  public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) {
+  public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) {
     if (lb == null) return null;
-    return convertLocatedBlock2(Arrays.asList(lb)).toArray(
-        new LocatedBlockProto[lb.length]);
+    return convertLocatedBlocks2(Arrays.asList(lb))
+        .toArray(new LocatedBlockProto[lb.length]);
   }
   
-  public static LocatedBlock[] convertLocatedBlock(LocatedBlockProto[] lb) {
+  public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) {
     if (lb == null) return null;
-    return convertLocatedBlock(Arrays.asList(lb)).toArray(
-        new LocatedBlock[lb.length]);
+    return convertLocatedBlocks(Arrays.asList(lb))
+        .toArray(new LocatedBlock[lb.length]);
   }
   
-  public static List<LocatedBlock> convertLocatedBlock(
+  public static List<LocatedBlock> convertLocatedBlocks(
       List<LocatedBlockProto> lb) {
     if (lb == null) return null;
     final int len = lb.size();
-    List<LocatedBlock> result = 
-        new ArrayList<LocatedBlock>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(PBHelper.convert(lb.get(i)));
+    List<LocatedBlock> result = new ArrayList<>(len);
+    for (LocatedBlockProto aLb : lb) {
+      result.add(PBHelper.convertLocatedBlockProto(aLb));
     }
     return result;
   }
   
-  public static List<LocatedBlockProto> convertLocatedBlock2(List<LocatedBlock> lb) {
+  public static List<LocatedBlockProto> convertLocatedBlocks2(
+      List<LocatedBlock> lb) {
     if (lb == null) return null;
     final int len = lb.size();
-    List<LocatedBlockProto> result = new ArrayList<LocatedBlockProto>(len);
-    for (int i = 0; i < len; ++i) {
-      result.add(PBHelper.convert(lb.get(i)));
+    List<LocatedBlockProto> result = new ArrayList<>(len);
+    for (LocatedBlock aLb : lb) {
+      result.add(PBHelper.convertLocatedBlock(aLb));
     }
     return result;
   }
@@ -1299,8 +1323,9 @@ public class PBHelper {
   public static LocatedBlocks convert(LocatedBlocksProto lb) {
     return new LocatedBlocks(
         lb.getFileLength(), lb.getUnderConstruction(),
-        PBHelper.convertLocatedBlock(lb.getBlocksList()),
-        lb.hasLastBlock() ? PBHelper.convert(lb.getLastBlock()) : null,
+        PBHelper.convertLocatedBlocks(lb.getBlocksList()),
+        lb.hasLastBlock() ?
+            PBHelper.convertLocatedBlockProto(lb.getLastBlock()) : null,
         lb.getIsLastBlockComplete(),
         lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) :
             null);
@@ -1313,14 +1338,15 @@ public class PBHelper {
     LocatedBlocksProto.Builder builder = 
         LocatedBlocksProto.newBuilder();
     if (lb.getLastLocatedBlock() != null) {
-      builder.setLastBlock(PBHelper.convert(lb.getLastLocatedBlock()));
+      builder.setLastBlock(
+          PBHelper.convertLocatedBlock(lb.getLastLocatedBlock()));
     }
     if (lb.getFileEncryptionInfo() != null) {
       builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo()));
     }
     return builder.setFileLength(lb.getFileLength())
         .setUnderConstruction(lb.isUnderConstruction())
-        .addAllBlocks(PBHelper.convertLocatedBlock2(lb.getLocatedBlocks()))
+        .addAllBlocks(PBHelper.convertLocatedBlocks2(lb.getLocatedBlocks()))
         .setIsLastBlockComplete(lb.isLastBlockComplete()).build();
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 8b458df..84c3be6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -123,6 +123,11 @@ public class BlockInfoStriped extends BlockInfo {
     return -1;
   }
 
+  int getStorageBlockIndex(DatanodeStorageInfo storage) {
+    int i = this.findStorageInfo(storage);
+    return i == -1 ? -1 : indices[i];
+  }
+
   @Override
   boolean removeStorage(DatanodeStorageInfo storage) {
     int dnIndex = findStorageInfoFromEnd(storage);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
index 151241b2..b1857bb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
@@ -23,9 +23,6 @@ import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.ReplicaState;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
 
 import java.io.IOException;
-import java.util.ArrayList;
-import java.util.Iterator;
-import java.util.List;
 
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.COMPLETE;
 import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION;
@@ -39,12 +36,8 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
 
   /**
    * Block replicas as assigned when the block was allocated.
-   *
-   * TODO: we need to update this attribute, along with the return type of
-   * getExpectedStorageLocations and LocatedBlock. For striped blocks, clients
-   * need to understand the index of each striped block in the block group.
    */
-  private List<ReplicaUnderConstruction> replicas;
+  private ReplicaUnderConstruction[] replicas;
 
   /**
    * The new generation stamp, which this block will have
@@ -75,12 +68,12 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
 
   /**
    * Convert an under construction striped block to a complete striped block.
-   * 
+   *
    * @return BlockInfoStriped - a complete block.
-   * @throws IOException if the state of the block 
-   * (the generation stamp and the length) has not been committed by 
-   * the client or it does not have at least a minimal number of replicas 
-   * reported from data-nodes. 
+   * @throws IOException if the state of the block
+   * (the generation stamp and the length) has not been committed by
+   * the client or it does not have at least a minimal number of replicas
+   * reported from data-nodes.
    */
   BlockInfoStriped convertToCompleteBlock() throws IOException {
     assert getBlockUCState() != COMPLETE :
@@ -91,10 +84,13 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   /** Set expected locations */
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
-    this.replicas = new ArrayList<>(numLocations);
+    this.replicas = new ReplicaUnderConstruction[numLocations];
     for(int i = 0; i < numLocations; i++) {
-      replicas.add(new ReplicaUnderConstruction(this, targets[i],
-          ReplicaState.RBW));
+      // when creating a new block we simply sequentially assign block index to
+      // each storage
+      Block blk = new Block(this.getBlockId() + i, this.getGenerationStamp(), 0);
+      replicas[i] = new ReplicaUnderConstruction(blk, targets[i],
+          ReplicaState.RBW);
     }
   }
 
@@ -106,14 +102,24 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     int numLocations = getNumExpectedLocations();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
     for (int i = 0; i < numLocations; i++) {
-      storages[i] = replicas.get(i).getExpectedStorageLocation();
+      storages[i] = replicas[i].getExpectedStorageLocation();
     }
     return storages;
   }
 
+  /** @return the index array indicating the block index in each storage */
+  public int[] getBlockIndices() {
+    int numLocations = getNumExpectedLocations();
+    int[] indices = new int[numLocations];
+    for (int i = 0; i < numLocations; i++) {
+      indices[i] = BlockIdManager.getBlockIndex(replicas[i]);
+    }
+    return indices;
+  }
+
   /** Get the number of expected locations */
   public int getNumExpectedLocations() {
-    return replicas == null ? 0 : replicas.size();
+    return replicas == null ? 0 : replicas.length;
   }
 
   /**
@@ -178,7 +184,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
-    if (replicas == null || replicas.size() == 0) {
+    if (replicas == null || replicas.length == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*" +
           " BlockInfoUnderConstruction.initLeaseRecovery:" +
           " No blocks found, lease removed.");
@@ -186,28 +192,36 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     // TODO we need to implement different recovery logic here
   }
 
-  void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block block,
+  void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock,
       ReplicaState rState) {
-    Iterator<ReplicaUnderConstruction> it = replicas.iterator();
-    while (it.hasNext()) {
-      ReplicaUnderConstruction r = it.next();
-      DatanodeStorageInfo expectedLocation = r.getExpectedStorageLocation();
-      if (expectedLocation == storage) {
-        // Record the gen stamp from the report
-        r.setGenerationStamp(block.getGenerationStamp());
-        return;
-      } else if (expectedLocation != null &&
-          expectedLocation.getDatanodeDescriptor() ==
-              storage.getDatanodeDescriptor()) {
-        // The Datanode reported that the block is on a different storage
-        // than the one chosen by BlockPlacementPolicy. This can occur as
-        // we allow Datanodes to choose the target storage. Update our
-        // state by removing the stale entry and adding a new one.
-        it.remove();
-        break;
+    if (replicas == null) {
+      replicas = new ReplicaUnderConstruction[1];
+      replicas[0] = new ReplicaUnderConstruction(reportedBlock, storage, rState);
+    } else {
+      for (int i = 0; i < replicas.length; i++) {
+        DatanodeStorageInfo expected = replicas[i].getExpectedStorageLocation();
+        if (expected == storage) {
+          replicas[i].setBlockId(reportedBlock.getBlockId());
+          replicas[i].setGenerationStamp(reportedBlock.getGenerationStamp());
+          return;
+        } else if (expected != null && expected.getDatanodeDescriptor() ==
+            storage.getDatanodeDescriptor()) {
+          // The Datanode reported that the block is on a different storage
+          // than the one chosen by BlockPlacementPolicy. This can occur as
+          // we allow Datanodes to choose the target storage. Update our
+          // state by removing the stale entry and adding a new one.
+          replicas[i] = new ReplicaUnderConstruction(reportedBlock, storage,
+              rState);
+          return;
+        }
       }
+      ReplicaUnderConstruction[] newReplicas =
+          new ReplicaUnderConstruction[replicas.length + 1];
+      System.arraycopy(replicas, 0, newReplicas, 0, replicas.length);
+      newReplicas[newReplicas.length - 1] = new ReplicaUnderConstruction(
+          reportedBlock, storage, rState);
+      replicas = newReplicas;
     }
-    replicas.add(new ReplicaUnderConstruction(block, storage, rState));
   }
 
   @Override
@@ -226,12 +240,11 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   private void appendUCParts(StringBuilder sb) {
     sb.append("{UCState=").append(blockUCState).append(", replicas=[");
     if (replicas != null) {
-      Iterator<ReplicaUnderConstruction> iter = replicas.iterator();
-      if (iter.hasNext()) {
-        iter.next().appendStringTo(sb);
-        while (iter.hasNext()) {
+      int i = 0;
+      for (ReplicaUnderConstruction r : replicas) {
+        r.appendStringTo(sb);
+        if (++i < replicas.length) {
           sb.append(", ");
-          iter.next().appendStringTo(sb);
         }
       }
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index aa27f3b..b0ac1c3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -54,6 +54,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.fs.FileEncryptionInfo;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.UnregisteredNodeException;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier;
 import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager;
@@ -839,21 +840,26 @@ public class BlockManager {
   }
 
   /** @return a LocatedBlock for the given block */
-  private LocatedBlock createLocatedBlock(final BlockInfo blk,
-      final long pos) throws IOException {
-    if (blk instanceof BlockInfoContiguousUnderConstruction) {
-      if (blk.isComplete()) {
-        throw new IOException(
-            "blk instanceof BlockInfoUnderConstruction && blk.isComplete()"
-            + ", blk=" + blk);
+  private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) {
+    if (!blk.isComplete()) {
+      if (blk.isStriped()) {
+        final BlockInfoStripedUnderConstruction uc =
+            (BlockInfoStripedUnderConstruction) blk;
+        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+            blk);
+        return new LocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos,
+            false);
+      } else {
+        assert blk instanceof BlockInfoContiguousUnderConstruction;
+        final BlockInfoContiguousUnderConstruction uc =
+            (BlockInfoContiguousUnderConstruction) blk;
+        final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
+        final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(),
+            blk);
+        return new LocatedBlock(eb, storages, pos, false);
       }
-      final BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction) blk;
-      final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations();
-      final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-      return new LocatedBlock(eb, storages, pos, false);
     }
-    // TODO support BlockInfoStripedUC
 
     // get block locations
     final int numCorruptNodes = countNodes(blk).corruptReplicas();
@@ -868,13 +874,21 @@ public class BlockManager {
     final boolean isCorrupt = numCorruptNodes == numNodes;
     final int numMachines = isCorrupt ? numNodes: numNodes - numCorruptNodes;
     final DatanodeStorageInfo[] machines = new DatanodeStorageInfo[numMachines];
-    int j = 0;
+    final int[] blockIndices = blk.isStriped() ? new int[numMachines] : null;
+    int j = 0, i = 0;
     if (numMachines > 0) {
       for(DatanodeStorageInfo storage : blocksMap.getStorages(blk)) {
         final DatanodeDescriptor d = storage.getDatanodeDescriptor();
         final boolean replicaCorrupt = corruptReplicas.isReplicaCorrupt(blk, d);
-        if (isCorrupt || (!replicaCorrupt))
+        if (isCorrupt || (!replicaCorrupt)) {
           machines[j++] = storage;
+          // TODO this can be more efficient
+          if (blockIndices != null) {
+            int index = ((BlockInfoStriped) blk).getStorageBlockIndex(storage);
+            assert index >= 0;
+            blockIndices[i++] = index;
+          }
+        }
       }
     }
     assert j == machines.length :
@@ -884,7 +898,9 @@ public class BlockManager {
       " numCorrupt: " + numCorruptNodes +
       " numCorruptRepls: " + numCorruptReplicas;
     final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk);
-    return new LocatedBlock(eb, machines, pos, isCorrupt);
+    return blockIndices == null ?
+        new LocatedBlock(eb, machines, pos, isCorrupt) :
+        new LocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt);
   }
 
   /** Create a LocatedBlocks. */
@@ -2476,7 +2492,8 @@ public class BlockManager {
   void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock,
       DatanodeStorageInfo storageInfo) throws IOException {
     BlockInfo block = ucBlock.storedBlock;
-    BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock, ucBlock.reportedState);
+    BlockInfo.addReplica(block, storageInfo, ucBlock.reportedBlock,
+        ucBlock.reportedState);
 
     if (ucBlock.reportedState == ReplicaState.FINALIZED &&
         !block.findDatanode(storageInfo.getDatanodeDescriptor())) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 415646a..b7a3489 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -276,7 +276,9 @@ public class DatanodeDescriptor extends DatanodeInfo {
       return storageMap.get(storageID);
     }
   }
-  DatanodeStorageInfo[] getStorageInfos() {
+
+  @VisibleForTesting
+  public DatanodeStorageInfo[] getStorageInfos() {
     synchronized (storageMap) {
       final Collection<DatanodeStorageInfo> storages = storageMap.values();
       return storages.toArray(new DatanodeStorageInfo[storages.size()]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
index fa7f263..cf1f896 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeStorageInfo.java
@@ -203,7 +203,8 @@ public class DatanodeStorageInfo {
     return getState() == State.FAILED && numBlocks != 0;
   }
 
-  String getStorageID() {
+  @VisibleForTesting
+  public String getStorageID() {
     return storageID;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index 5627788..f293481 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -364,16 +364,19 @@ public final class FSImageFormatPBINode {
         INodeSection.FileUnderConstructionFeature uc = f.getFileUC();
         file.toUnderConstruction(uc.getClientName(), uc.getClientMachine());
         BlockInfo lastBlk = file.getLastBlock();
-        // replace the last block of file
-        final BlockInfo ucBlk;
-        if (stripeFeature != null) {
-          BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
-          ucBlk = new BlockInfoStripedUnderConstruction(striped,
-              striped.getDataBlockNum(), striped.getParityBlockNum());
-        } else {
-          ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk, replication);
+        if (lastBlk != null) {
+          // replace the last block of file
+          final BlockInfo ucBlk;
+          if (stripeFeature != null) {
+            BlockInfoStriped striped = (BlockInfoStriped) lastBlk;
+            ucBlk = new BlockInfoStripedUnderConstruction(striped,
+                striped.getDataBlockNum(), striped.getParityBlockNum());
+          } else {
+            ucBlk = new BlockInfoContiguousUnderConstruction(lastBlk,
+                replication);
+          }
+          file.setBlock(file.numBlocks() - 1, ucBlk);
         }
-        file.setBlock(file.numBlocks() - 1, ucBlk);
       }
       return file;
     }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 47ae7de..2b23cd3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -189,6 +189,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
 import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.protocol.QuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.RecoveryInProgressException;
 import org.apache.hadoop.hdfs.protocol.RollingUpgradeException;
@@ -206,6 +207,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -1750,8 +1752,14 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
     LocatedBlocks blocks = res.blocks;
     if (blocks != null) {
+      List<LocatedBlock> blkList = blocks.getLocatedBlocks();
+      if (blkList == null || blkList.size() == 0 ||
+          blkList.get(0) instanceof LocatedStripedBlock) {
+        // no need to sort locations for striped blocks
+        return blocks;
+      }
       blockManager.getDatanodeManager().sortLocatedBlocks(
-          clientMachine, blocks.getLocatedBlocks());
+          clientMachine, blkList);
 
       // lastBlock is not part of getLocatedBlocks(), might need to sort it too
       LocatedBlock lastBlock = blocks.getLastLocatedBlock();
@@ -3115,7 +3123,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         src, numTargets, clientNode, excludedNodes, blockSize, favoredNodes,
         storagePolicyID);
   }
-
   /**
    * Part II of getAdditionalBlock().
    * Should repeat the same analysis of the file state as in Part 1,
@@ -3125,7 +3132,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
    */
   LocatedBlock storeAllocatedBlock(String src, long fileId, String clientName,
       ExtendedBlock previous, DatanodeStorageInfo[] targets) throws IOException {
-    Block newBlock = null;
+    BlockInfo newBlockInfo = null;
     long offset;
     checkOperation(OperationCategory.WRITE);
     waitForLoadingFSImage();
@@ -3158,8 +3165,9 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
                                 ExtendedBlock.getLocalBlock(previous));
 
       // allocate new block, record block locations in INode.
-      newBlock = createNewBlock(isStriped);
-      saveAllocatedBlock(src, fileState.iip, newBlock, targets, isStriped);
+      Block newBlock = createNewBlock(isStriped);
+      newBlockInfo = saveAllocatedBlock(src, fileState.iip, newBlock, targets,
+          isStriped);
 
       persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
@@ -3169,7 +3177,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     getEditLog().logSync();
 
     // Return located block
-    return makeLocatedBlock(newBlock, targets, offset);
+    return makeLocatedBlock(newBlockInfo, targets, offset);
   }
 
   /*
@@ -3303,10 +3311,17 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     return new FileState(pendingFile, src, iip);
   }
 
-  LocatedBlock makeLocatedBlock(Block blk, DatanodeStorageInfo[] locs,
-                                        long offset) throws IOException {
-    LocatedBlock lBlk = new LocatedBlock(
-        getExtendedBlock(blk), locs, offset, false);
+  LocatedBlock makeLocatedBlock(BlockInfo blk, DatanodeStorageInfo[] locs,
+      long offset) throws IOException {
+    final LocatedBlock lBlk;
+    if (blk.isStriped()) {
+      assert blk instanceof BlockInfoStripedUnderConstruction;
+      lBlk = new LocatedStripedBlock(getExtendedBlock(blk), locs,
+          ((BlockInfoStripedUnderConstruction) blk).getBlockIndices(),
+          offset, false);
+    } else {
+      lBlk = new LocatedBlock(getExtendedBlock(blk), locs, offset, false);
+    }
     getBlockManager().setBlockToken(
         lBlk, BlockTokenIdentifier.AccessMode.WRITE);
     return lBlk;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index d281a64..67e2058 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -220,6 +220,7 @@ message LocatedBlockProto {
   repeated bool isCached = 6 [packed=true]; // if a location in locs is cached
   repeated StorageTypeProto storageTypes = 7;
   repeated string storageIDs = 8;
+  repeated uint32 blockIndex = 9; // used for striped block to indicate block index for each storage
 }
 
 message DataEncryptionKeyProto {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
index c3b5b44..4f37fbb 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/DFSTestUtil.java
@@ -127,6 +127,10 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
 import org.apache.hadoop.hdfs.server.namenode.ha.ConfiguredFailoverProxyProvider;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.hdfs.tools.DFSAdmin;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.nativeio.NativeIO;
@@ -1770,4 +1774,12 @@ public class DFSTestUtil {
     dn.setLastUpdateMonotonic(Time.monotonicNow() + offset);
   }
 
+  public static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock(
+      Block block, BlockStatus blockStatus, DatanodeStorage storage) {
+    ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1];
+    receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, blockStatus, null);
+    StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1];
+    reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
+    return reports;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
index c7233bd..4b42f4c 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocolPB/TestPBHelper.java
@@ -489,16 +489,16 @@ public class TestPBHelper {
   @Test
   public void testConvertLocatedBlock() {
     LocatedBlock lb = createLocatedBlock();
-    LocatedBlockProto lbProto = PBHelper.convert(lb);
-    LocatedBlock lb2 = PBHelper.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
+    LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
     compare(lb,lb2);
   }
 
   @Test
   public void testConvertLocatedBlockNoStorageMedia() {
     LocatedBlock lb = createLocatedBlockNoStorageMedia();
-    LocatedBlockProto lbProto = PBHelper.convert(lb);
-    LocatedBlock lb2 = PBHelper.convert(lbProto);
+    LocatedBlockProto lbProto = PBHelper.convertLocatedBlock(lb);
+    LocatedBlock lb2 = PBHelper.convertLocatedBlockProto(lbProto);
     compare(lb,lb2);
   }
 
@@ -508,8 +508,8 @@ public class TestPBHelper {
     for (int i=0;i<3;i++) {
       lbl.add(createLocatedBlock());
     }
-    List<LocatedBlockProto> lbpl = PBHelper.convertLocatedBlock2(lbl);
-    List<LocatedBlock> lbl2 = PBHelper.convertLocatedBlock(lbpl);
+    List<LocatedBlockProto> lbpl = PBHelper.convertLocatedBlocks2(lbl);
+    List<LocatedBlock> lbl2 = PBHelper.convertLocatedBlocks(lbpl);
     assertEquals(lbl.size(), lbl2.size());
     for (int i=0;i<lbl.size();i++) {
       compare(lbl.get(i), lbl2.get(2));
@@ -522,8 +522,8 @@ public class TestPBHelper {
     for (int i=0;i<3;i++) {
       lbl[i] = createLocatedBlock();
     }
-    LocatedBlockProto [] lbpl = PBHelper.convertLocatedBlock(lbl);
-    LocatedBlock [] lbl2 = PBHelper.convertLocatedBlock(lbpl);
+    LocatedBlockProto [] lbpl = PBHelper.convertLocatedBlocks(lbl);
+    LocatedBlock [] lbl2 = PBHelper.convertLocatedBlocks(lbpl);
     assertEquals(lbl.length, lbl2.length);
     for (int i=0;i<lbl.length;i++) {
       compare(lbl[i], lbl2[i]);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
index 4e73e6e..5d8d307 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/TestIncrementalBrVariations.java
@@ -220,15 +220,6 @@ public class TestIncrementalBrVariations {
     return new Block(10000000L, 100L, 1048576L);
   }
 
-  private static StorageReceivedDeletedBlocks[] makeReportForReceivedBlock(
-      Block block, DatanodeStorage storage) {
-    ReceivedDeletedBlockInfo[] receivedBlocks = new ReceivedDeletedBlockInfo[1];
-    receivedBlocks[0] = new ReceivedDeletedBlockInfo(block, BlockStatus.RECEIVED_BLOCK, null);
-    StorageReceivedDeletedBlocks[] reports = new StorageReceivedDeletedBlocks[1];
-    reports[0] = new StorageReceivedDeletedBlocks(storage, receivedBlocks);
-    return reports;
-  }
-
   /**
    * Verify that the NameNode can learn about new storages from incremental
    * block reports.
@@ -244,8 +235,9 @@ public class TestIncrementalBrVariations {
     // Generate a report for a fake block on a fake storage.
     final String newStorageUuid = UUID.randomUUID().toString();
     final DatanodeStorage newStorage = new DatanodeStorage(newStorageUuid);
-    StorageReceivedDeletedBlocks[] reports = makeReportForReceivedBlock(
-        getDummyBlock(), newStorage);
+    StorageReceivedDeletedBlocks[] reports = DFSTestUtil.
+        makeReportForReceivedBlock(getDummyBlock(), BlockStatus.RECEIVED_BLOCK,
+            newStorage);
 
     // Send the report to the NN.
     cluster.getNameNodeRpc().blockReceivedAndDeleted(dn0Reg, poolId, reports);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 7226f51..8b51309 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -19,18 +19,29 @@ package org.apache.hadoop.hdfs.server.namenode;
 
 import org.apache.hadoop.fs.FSDataOutputStream;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.Block;
+import org.apache.hadoop.hdfs.protocol.BlockListAsLongs;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.protocol.LocatedBlocks;
+import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.datanode.ReplicaBeingWritten;
+import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
+import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus;
+import org.apache.hadoop.hdfs.server.protocol.StorageBlockReport;
+import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks;
 import org.apache.hadoop.io.IOUtils;
 import org.junit.After;
 import org.junit.Assert;
@@ -38,6 +49,9 @@ import org.junit.Before;
 import org.junit.Test;
 
 import java.io.IOException;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.UUID;
 
 import static org.junit.Assert.assertEquals;
 
@@ -143,4 +157,131 @@ public class TestAddStripedBlocks {
     }
     return false;
   }
+
+  @Test
+  public void testGetLocatedStripedBlocks() throws Exception {
+    final Path file = new Path("/file1");
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      cluster.getNamesystem().getAdditionalBlock(file.toString(),
+          fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
+      BlockInfoStripedUnderConstruction lastBlk =
+          (BlockInfoStripedUnderConstruction) fileNode.getLastBlock();
+      DatanodeInfo[] expectedDNs = DatanodeStorageInfo
+          .toDatanodeInfos(lastBlk.getExpectedStorageLocations());
+      int[] indices = lastBlk.getBlockIndices();
+
+      LocatedBlocks blks = dfs.getClient().getLocatedBlocks(file.toString(), 0L);
+      Assert.assertEquals(1, blks.locatedBlockCount());
+      LocatedBlock lblk = blks.get(0);
+
+      Assert.assertTrue(lblk instanceof LocatedStripedBlock);
+      DatanodeInfo[] datanodes = lblk.getLocations();
+      int[] blockIndices = ((LocatedStripedBlock) lblk).getBlockIndices();
+      Assert.assertEquals(GROUP_SIZE, datanodes.length);
+      Assert.assertEquals(GROUP_SIZE, blockIndices.length);
+      Assert.assertArrayEquals(indices, blockIndices);
+      Assert.assertArrayEquals(expectedDNs, datanodes);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+  }
+
+  /**
+   * Test BlockInfoStripedUnderConstruction#addReplicaIfNotPresent in different
+   * scenarios.
+   */
+  @Test
+  public void testAddUCReplica() throws Exception {
+    final Path file = new Path("/file1");
+    final List<String> storageIDs = new ArrayList<>();
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+
+      // 1. create the UC striped block
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      cluster.getNamesystem().getAdditionalBlock(file.toString(),
+          fileNode.getId(), dfs.getClient().getClientName(), null, null, null);
+      BlockInfo lastBlock = fileNode.getLastBlock();
+      BlockInfoStripedUnderConstruction ucBlock =
+          (BlockInfoStripedUnderConstruction) lastBlock;
+
+      DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
+      int[] indices = ucBlock.getBlockIndices();
+      Assert.assertEquals(GROUP_SIZE, locs.length);
+      Assert.assertEquals(GROUP_SIZE, indices.length);
+
+      // 2. mimic incremental block reports and make sure the uc-replica list in
+      // the BlockStripedUC is correct
+      int i = 0;
+      for (DataNode dn : cluster.getDataNodes()) {
+        final Block block = new Block(lastBlock.getBlockId() + i++,
+            lastBlock.getGenerationStamp(), 0);
+        DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString());
+        storageIDs.add(storage.getStorageID());
+        StorageReceivedDeletedBlocks[] reports = DFSTestUtil
+            .makeReportForReceivedBlock(block, BlockStatus.RECEIVING_BLOCK,
+                storage);
+        for (StorageReceivedDeletedBlocks report : reports) {
+          cluster.getNamesystem().processIncrementalBlockReport(
+              dn.getDatanodeId(), report);
+        }
+      }
+
+      // make sure lastBlock is correct and the storages have been updated
+      locs = ucBlock.getExpectedStorageLocations();
+      indices = ucBlock.getBlockIndices();
+      Assert.assertEquals(GROUP_SIZE, locs.length);
+      Assert.assertEquals(GROUP_SIZE, indices.length);
+      for (DatanodeStorageInfo newstorage : locs) {
+        Assert.assertTrue(storageIDs.contains(newstorage.getStorageID()));
+      }
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+
+    // 3. restart the namenode. mimic the full block reports and check the
+    // uc-replica list again
+    cluster.restartNameNode(true);
+    final String bpId = cluster.getNamesystem().getBlockPoolId();
+    INodeFile fileNode = cluster.getNamesystem().getFSDirectory()
+        .getINode4Write(file.toString()).asFile();
+    BlockInfo lastBlock = fileNode.getLastBlock();
+    int i = GROUP_SIZE - 1;
+    for (DataNode dn : cluster.getDataNodes()) {
+      String storageID = storageIDs.get(i);
+      final Block block = new Block(lastBlock.getBlockId() + i--,
+          lastBlock.getGenerationStamp(), 0);
+      DatanodeStorage storage = new DatanodeStorage(storageID);
+      List<ReplicaBeingWritten> blocks = new ArrayList<>();
+      ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null,
+          null);
+      blocks.add(replica);
+      BlockListAsLongs bll = new BlockListAsLongs(null, blocks);
+      StorageBlockReport[] reports = {new StorageBlockReport(storage,
+          bll.getBlockListAsLongs())};
+      cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
+          bpId, reports);
+    }
+
+    BlockInfoStripedUnderConstruction ucBlock =
+        (BlockInfoStripedUnderConstruction) lastBlock;
+    DatanodeStorageInfo[] locs = ucBlock.getExpectedStorageLocations();
+    int[] indices = ucBlock.getBlockIndices();
+    Assert.assertEquals(GROUP_SIZE, locs.length);
+    Assert.assertEquals(GROUP_SIZE, indices.length);
+    for (i = 0; i < GROUP_SIZE; i++) {
+      Assert.assertEquals(storageIDs.get(i),
+          locs[GROUP_SIZE - 1 - i].getStorageID());
+      Assert.assertEquals(GROUP_SIZE - i - 1, indices[i]);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/94a1ea4c/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
index 913e0a7..4d42911 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSImage.java
@@ -25,6 +25,8 @@ import java.io.IOException;
 import java.util.EnumSet;
 
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption;
 import org.junit.Assert;
 
 import org.apache.hadoop.conf.Configuration;
@@ -219,8 +221,7 @@ public class TestFSImage {
         .format(false)
         .manageDataDfsDirs(false)
         .manageNameDfsDirs(false)
-        .waitSafeMode(false)
-        .startupOption(StartupOption.UPGRADE)
+        .waitSafeMode(false).startupOption(StartupOption.UPGRADE)
         .build();
     try {
       FileSystem fs = cluster.getFileSystem();


[05/50] [abbrv] hadoop git commit: HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print striped blocks. Contributed by Takuya Fukudome.

Posted by zh...@apache.org.
HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print striped blocks. Contributed by Takuya Fukudome.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/53b9a1b6
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/53b9a1b6
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/53b9a1b6

Branch: refs/heads/HDFS-7285
Commit: 53b9a1b6724ec4af687b91c478fec8a19cd70a77
Parents: 564860c
Author: Jing Zhao <ji...@apache.org>
Authored: Thu Mar 5 16:44:38 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:36:07 2015 -0700

----------------------------------------------------------------------
 .../java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java   | 4 ++--
 1 file changed, 2 insertions(+), 2 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/53b9a1b6/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 093ba25..22d61bc 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -877,8 +877,8 @@ public class INodeFile extends INodeWithAdditionalFields
     out.print(", fileSize=" + computeFileSize(snapshotId));
     // only compare the first block
     out.print(", blocks=");
-    out.print(blocks == null || blocks.length == 0? null: blocks[0]);
-    // TODO print striped blocks
+    BlockInfo[] blks = getBlocks();
+    out.print(blks == null || blks.length == 0? null: blks[0]);
     out.println();
   }
 


[07/50] [abbrv] hadoop git commit: HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( Contributed by Kai Zheng )

Posted by zh...@apache.org.
HADOOP-11646. Erasure Coder API for encoding and decoding of block group ( Contributed by Kai Zheng )


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d542a3d1
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d542a3d1
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d542a3d1

Branch: refs/heads/HDFS-7285
Commit: d542a3d1299e20268b2529c10dff7e3e9b377158
Parents: 53b9a1b
Author: Vinayakumar B <vi...@apache.org>
Authored: Mon Mar 9 12:32:26 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:36:08 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   2 +
 .../apache/hadoop/io/erasurecode/ECBlock.java   |  80 ++++++
 .../hadoop/io/erasurecode/ECBlockGroup.java     |  82 ++++++
 .../erasurecode/coder/AbstractErasureCoder.java |  63 +++++
 .../coder/AbstractErasureCodingStep.java        |  59 ++++
 .../coder/AbstractErasureDecoder.java           | 152 +++++++++++
 .../coder/AbstractErasureEncoder.java           |  50 ++++
 .../io/erasurecode/coder/ErasureCoder.java      |  77 ++++++
 .../io/erasurecode/coder/ErasureCodingStep.java |  55 ++++
 .../io/erasurecode/coder/ErasureDecoder.java    |  41 +++
 .../erasurecode/coder/ErasureDecodingStep.java  |  52 ++++
 .../io/erasurecode/coder/ErasureEncoder.java    |  39 +++
 .../erasurecode/coder/ErasureEncodingStep.java  |  49 ++++
 .../io/erasurecode/coder/XorErasureDecoder.java |  78 ++++++
 .../io/erasurecode/coder/XorErasureEncoder.java |  45 ++++
 .../erasurecode/rawcoder/RawErasureCoder.java   |   2 +-
 .../erasurecode/coder/TestErasureCoderBase.java | 266 +++++++++++++++++++
 .../io/erasurecode/coder/TestXorCoder.java      |  50 ++++
 18 files changed, 1241 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index ee42c84..c17a1bd 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -15,4 +15,6 @@
     HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng
     ( Kai Zheng )
 
+    HADOOP-11646. Erasure Coder API for encoding and decoding of block group
+    ( Kai Zheng via vinayakumarb )
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
new file mode 100644
index 0000000..956954a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlock.java
@@ -0,0 +1,80 @@
+/**
+ * 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.io.erasurecode;
+
+/**
+ * A wrapper of block level data source/output that {@link ECChunk}s can be
+ * extracted from. For HDFS, it can be an HDFS block (250MB). Note it only cares
+ * about erasure coding specific logic thus avoids coupling with any HDFS block
+ * details. We can have something like HdfsBlock extend it.
+ */
+public class ECBlock {
+
+  private boolean isParity;
+  private boolean isErased;
+
+  /**
+   * A default constructor. isParity and isErased are false by default.
+   */
+  public ECBlock() {
+    this(false, false);
+  }
+
+  /**
+   * A constructor specifying isParity and isErased.
+   * @param isParity
+   * @param isErased
+   */
+  public ECBlock(boolean isParity, boolean isErased) {
+    this.isParity = isParity;
+    this.isErased = isErased;
+  }
+
+  /**
+   * Set true if it's for a parity block.
+   * @param isParity
+   */
+  public void setParity(boolean isParity) {
+    this.isParity = isParity;
+  }
+
+  /**
+   * Set true if the block is missing.
+   * @param isMissing
+   */
+  public void setErased(boolean isMissing) {
+    this.isErased = isMissing;
+  }
+
+  /**
+   *
+   * @return true if it's parity block, otherwise false
+   */
+  public boolean isParity() {
+    return isParity;
+  }
+
+  /**
+   *
+   * @return true if it's missing or corrupt due to erasure, otherwise false
+   */
+  public boolean isErased() {
+    return isErased;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
new file mode 100644
index 0000000..2c851a5
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
@@ -0,0 +1,82 @@
+/**
+ * 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.io.erasurecode;
+
+/**
+ * A group of blocks or {@link ECBlock} incurred in an erasure coding task.
+ */
+public class ECBlockGroup {
+
+  private ECBlock[] dataBlocks;
+  private ECBlock[] parityBlocks;
+
+  /**
+   * A constructor specifying data blocks and parity blocks.
+   * @param dataBlocks
+   * @param parityBlocks
+   */
+  public ECBlockGroup(ECBlock[] dataBlocks, ECBlock[] parityBlocks) {
+    this.dataBlocks = dataBlocks;
+    this.parityBlocks = parityBlocks;
+  }
+
+  /**
+   * Get data blocks
+   * @return data blocks
+   */
+  public ECBlock[] getDataBlocks() {
+    return dataBlocks;
+  }
+
+  /**
+   * Get parity blocks
+   * @return parity blocks
+   */
+  public ECBlock[] getParityBlocks() {
+    return parityBlocks;
+  }
+
+  /**
+   * Any erased data block?
+   * @return true if any erased data block, false otherwise
+   */
+  public boolean anyErasedDataBlock() {
+    for (int i = 0; i < dataBlocks.length; ++i) {
+      if (dataBlocks[i].isErased()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+  /**
+   * Any erased parity block?
+   * @return true if any erased parity block, false otherwise
+   */
+  public boolean anyErasedParityBlock() {
+    for (int i = 0; i < parityBlocks.length; ++i) {
+      if (parityBlocks[i].isErased()) {
+        return true;
+      }
+    }
+
+    return false;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
new file mode 100644
index 0000000..f2cc041
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -0,0 +1,63 @@
+/**
+ * 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.io.erasurecode.coder;
+
+/**
+ * A common class of basic facilities to be shared by encoder and decoder
+ *
+ * It implements the {@link ErasureCoder} interface.
+ */
+public abstract class AbstractErasureCoder implements ErasureCoder {
+
+  private int numDataUnits;
+  private int numParityUnits;
+  private int chunkSize;
+
+  @Override
+  public void initialize(int numDataUnits, int numParityUnits,
+                         int chunkSize) {
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+    this.chunkSize = chunkSize;
+  }
+
+  @Override
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  @Override
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  @Override
+  public int getChunkSize() {
+    return chunkSize;
+  }
+
+  @Override
+  public boolean preferNativeBuffer() {
+    return false;
+  }
+
+  @Override
+  public void release() {
+    // Nothing to do by default
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
new file mode 100644
index 0000000..c429d49
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCodingStep.java
@@ -0,0 +1,59 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+
+/**
+ * Abstract class for common facilities shared by {@link ErasureEncodingStep}
+ * and {@link ErasureDecodingStep}.
+ *
+ * It implements {@link ErasureEncodingStep}.
+ */
+public abstract class AbstractErasureCodingStep implements ErasureCodingStep {
+
+  private ECBlock[] inputBlocks;
+  private ECBlock[] outputBlocks;
+
+  /**
+   * Constructor given input blocks and output blocks.
+   * @param inputBlocks
+   * @param outputBlocks
+   */
+  public AbstractErasureCodingStep(ECBlock[] inputBlocks,
+                                   ECBlock[] outputBlocks) {
+    this.inputBlocks = inputBlocks;
+    this.outputBlocks = outputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getInputBlocks() {
+    return inputBlocks;
+  }
+
+  @Override
+  public ECBlock[] getOutputBlocks() {
+    return outputBlocks;
+  }
+
+  @Override
+  public void finish() {
+    // NOOP by default
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
new file mode 100644
index 0000000..df71528
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java
@@ -0,0 +1,152 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+
+/**
+ * An abstract erasure decoder that's to be inherited by new decoders.
+ *
+ * It implements the {@link ErasureDecoder} interface.
+ */
+public abstract class AbstractErasureDecoder extends AbstractErasureCoder
+    implements ErasureDecoder {
+
+  @Override
+  public ErasureCodingStep decode(ECBlockGroup blockGroup) {
+    return performDecoding(blockGroup);
+  }
+
+  /**
+   * Perform decoding against a block blockGroup.
+   * @param blockGroup
+   * @return decoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep performDecoding(ECBlockGroup blockGroup);
+
+  /**
+   * We have all the data blocks and parity blocks as input blocks for
+   * recovering by default. It's codec specific
+   * @param blockGroup
+   * @return
+   */
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] inputBlocks = new ECBlock[getNumParityUnits()
+        + getNumDataUnits()];
+
+    System.arraycopy(blockGroup.getParityBlocks(), 0, inputBlocks, 0,
+        getNumParityUnits());
+    System.arraycopy(blockGroup.getDataBlocks(), 0, inputBlocks,
+        getNumParityUnits(), getNumDataUnits());
+
+    return inputBlocks;
+  }
+
+  /**
+   * Which blocks were erased ? We only care data blocks here. Sub-classes can
+   * override this behavior.
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    ECBlock[] outputBlocks = new ECBlock[
+        getNumErasedBlocks(blockGroup.getDataBlocks())];
+
+    int idx = 0;
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      if (blockGroup.getDataBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
+      }
+    }
+
+    return outputBlocks;
+  }
+
+  /**
+   * Get the number of erased blocks in the block group.
+   * @param blockGroup
+   * @return number of erased blocks
+   */
+  protected int getNumErasedBlocks(ECBlockGroup blockGroup) {
+    int num = getNumErasedBlocks(blockGroup.getParityBlocks());
+    num += getNumErasedBlocks(blockGroup.getDataBlocks());
+    return num;
+  }
+
+  /**
+   * Find out how many blocks are erased.
+   * @param inputBlocks all the input blocks
+   * @return number of erased blocks
+   */
+  protected static int getNumErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = 0;
+    for (int i = 0; i < inputBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        numErased ++;
+      }
+    }
+
+    return numErased;
+  }
+
+  /**
+   * Get indexes of erased blocks from inputBlocks
+   * @param inputBlocks
+   * @return indexes of erased blocks from inputBlocks
+   */
+  protected int[] getErasedIndexes(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new int[0];
+    }
+
+    int[] erasedIndexes = new int[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedIndexes.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedIndexes[j++] = i;
+      }
+    }
+
+    return erasedIndexes;
+  }
+
+  /**
+   * Get erased input blocks from inputBlocks
+   * @param inputBlocks
+   * @return an array of erased blocks from inputBlocks
+   */
+  protected ECBlock[] getErasedBlocks(ECBlock[] inputBlocks) {
+    int numErased = getNumErasedBlocks(inputBlocks);
+    if (numErased == 0) {
+      return new ECBlock[0];
+    }
+
+    ECBlock[] erasedBlocks = new ECBlock[numErased];
+    int i = 0, j = 0;
+    for (; i < inputBlocks.length && j < erasedBlocks.length; i++) {
+      if (inputBlocks[i].isErased()) {
+        erasedBlocks[j++] = inputBlocks[i];
+      }
+    }
+
+    return erasedBlocks;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
new file mode 100644
index 0000000..80b9875
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java
@@ -0,0 +1,50 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+
+/**
+ * An abstract erasure encoder that's to be inherited by new encoders.
+ *
+ * It implements the {@link ErasureEncoder} interface.
+ */
+public abstract class AbstractErasureEncoder extends AbstractErasureCoder
+    implements ErasureEncoder {
+
+  @Override
+  public ErasureCodingStep encode(ECBlockGroup blockGroup) {
+    return performEncoding(blockGroup);
+  }
+
+  /**
+   * Perform encoding against a block group.
+   * @param blockGroup
+   * @return encoding step for caller to do the real work
+   */
+  protected abstract ErasureCodingStep performEncoding(ECBlockGroup blockGroup);
+
+  protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getDataBlocks();
+  }
+
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    return blockGroup.getParityBlocks();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
new file mode 100644
index 0000000..68875c0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -0,0 +1,77 @@
+/**
+ * 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.io.erasurecode.coder;
+
+/**
+ * An erasure coder to perform encoding or decoding given a group. Generally it
+ * involves calculating necessary internal steps according to codec logic. For
+ * each step,it calculates necessary input blocks to read chunks from and output
+ * parity blocks to write parity chunks into from the group. It also takes care
+ * of appropriate raw coder to use for the step. And encapsulates all the
+ * necessary info (input blocks, output blocks and raw coder) into a step
+ * represented by {@link ErasureCodingStep}. ErasureCoder callers can use the
+ * step to do the real work with retrieved input and output chunks.
+ *
+ * Note, currently only one coding step is supported. Will support complex cases
+ * of multiple coding steps.
+ *
+ */
+public interface ErasureCoder {
+
+  /**
+   * Initialize with the important parameters for the code.
+   * @param numDataUnits how many data inputs for the coding
+   * @param numParityUnits how many parity outputs the coding generates
+   * @param chunkSize the size of the input/output buffer
+   */
+  public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
+
+  /**
+   * The number of data input units for the coding. A unit can be a byte,
+   * chunk or buffer or even a block.
+   * @return count of data input units
+   */
+  public int getNumDataUnits();
+
+  /**
+   * The number of parity output units for the coding. A unit can be a byte,
+   * chunk, buffer or even a block.
+   * @return count of parity output units
+   */
+  public int getNumParityUnits();
+
+  /**
+   * Chunk buffer size for the input/output
+   * @return chunk buffer size
+   */
+  public int getChunkSize();
+
+  /**
+   * Tell if native or off-heap buffer is preferred or not. It's for callers to
+   * decide how to allocate coding chunk buffers, either on heap or off heap.
+   * It will return false by default.
+   * @return true if native buffer is preferred for performance consideration,
+   * otherwise false.
+   */
+  public boolean preferNativeBuffer();
+
+  /**
+   * Release the resources if any. Good chance to invoke RawErasureCoder#release.
+   */
+  public void release();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
new file mode 100644
index 0000000..a3b177f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCodingStep.java
@@ -0,0 +1,55 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+
+/**
+ * Erasure coding step that's involved in encoding/decoding of a block group.
+ */
+public interface ErasureCodingStep {
+
+  /**
+   * Input blocks of readable data involved in this step, may be data blocks
+   * or parity blocks.
+   * @return input blocks
+   */
+  public ECBlock[] getInputBlocks();
+
+  /**
+   * Output blocks of writable buffers involved in this step, may be data
+   * blocks or parity blocks.
+   * @return output blocks
+   */
+  public ECBlock[] getOutputBlocks();
+
+  /**
+   * Perform encoding or decoding given the input chunks, and generated results
+   * will be written to the output chunks.
+   * @param inputChunks
+   * @param outputChunks
+   */
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks);
+
+  /**
+   * Notify erasure coder that all the chunks of input blocks are processed so
+   * the coder can be able to update internal states, considering next step.
+   */
+  public void finish();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
new file mode 100644
index 0000000..dfd9e54
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecoder.java
@@ -0,0 +1,41 @@
+/**
+ * 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.io.erasurecode.coder;
+
+
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+
+/**
+ * Erasure decoder interface to perform decoding given a block group.
+ *
+ * It extends {@link ErasureCoder}.
+ */
+public interface ErasureDecoder extends ErasureCoder {
+
+  /**
+   * Perform the decoding given a blockGroup. By default it will try the best to
+   * attempt to recover all the missing blocks according to the codec logic.
+   *
+   * Note, currently only one coding step is supported. Will support complex
+   * cases of multiple coding steps.
+   *
+   * @param blockGroup
+   */
+  public ErasureCodingStep decode(ECBlockGroup blockGroup);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
new file mode 100644
index 0000000..980c580
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureDecodingStep.java
@@ -0,0 +1,52 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+
+/**
+ * Erasure decoding step, a wrapper of all the necessary information to perform
+ * a decoding step involved in the whole process of decoding a block group.
+ */
+public class ErasureDecodingStep extends AbstractErasureCodingStep {
+  private int[] erasedIndexes;
+  private RawErasureDecoder rawDecoder;
+
+  /**
+   * The constructor with all the necessary info.
+   * @param inputBlocks
+   * @param erasedIndexes the indexes of erased blocks in inputBlocks array
+   * @param outputBlocks
+   * @param rawDecoder
+   */
+  public ErasureDecodingStep(ECBlock[] inputBlocks, int[] erasedIndexes,
+                             ECBlock[] outputBlocks,
+                             RawErasureDecoder rawDecoder) {
+    super(inputBlocks, outputBlocks);
+    this.erasedIndexes = erasedIndexes;
+    this.rawDecoder = rawDecoder;
+  }
+
+  @Override
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
+    rawDecoder.decode(inputChunks, erasedIndexes, outputChunks);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
new file mode 100644
index 0000000..e837d22
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncoder.java
@@ -0,0 +1,39 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+
+/**
+ * Erasure encoder interface to perform encoding given a block group.
+ *
+ * It extends {@link ErasureCoder}.
+ */
+public interface ErasureEncoder extends ErasureCoder {
+
+  /**
+   * Calculate the encoding steps given a block blockGroup.
+   *
+   * Note, currently only one coding step is supported. Will support complex
+   * cases of multiple coding steps.
+   *
+   * @param blockGroup
+   */
+  public ErasureCodingStep encode(ECBlockGroup blockGroup);
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
new file mode 100644
index 0000000..bd7587f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureEncodingStep.java
@@ -0,0 +1,49 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+
+/**
+ * Erasure encoding step, a wrapper of all the necessary information to perform
+ * an encoding step involved in the whole process of encoding a block group.
+ */
+public class ErasureEncodingStep extends AbstractErasureCodingStep {
+
+  private RawErasureEncoder rawEncoder;
+
+  /**
+   * The constructor with all the necessary info.
+   * @param inputBlocks
+   * @param outputBlocks
+   * @param rawEncoder
+   */
+  public ErasureEncodingStep(ECBlock[] inputBlocks, ECBlock[] outputBlocks,
+                             RawErasureEncoder rawEncoder) {
+    super(inputBlocks, outputBlocks);
+    this.rawEncoder = rawEncoder;
+  }
+
+  @Override
+  public void performCoding(ECChunk[] inputChunks, ECChunk[] outputChunks) {
+    rawEncoder.encode(inputChunks, outputChunks);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.java
new file mode 100644
index 0000000..9a6f6e2
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureDecoder.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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XorRawDecoder;
+
+/**
+ * Xor erasure decoder that decodes a block group.
+ *
+ * It implements {@link ErasureDecoder}.
+ */
+public class XorErasureDecoder extends AbstractErasureDecoder {
+
+  @Override
+  protected ErasureCodingStep performDecoding(final ECBlockGroup blockGroup) {
+    // May be configured
+    RawErasureDecoder rawDecoder = new XorRawDecoder();
+    rawDecoder.initialize(getNumDataUnits(),
+        getNumParityUnits(), getChunkSize());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureDecodingStep(inputBlocks,
+        getErasedIndexes(inputBlocks),
+        getOutputBlocks(blockGroup), rawDecoder);
+  }
+
+  /**
+   * Which blocks were erased ? For XOR it's simple we only allow and return one
+   * erased block, either data or parity.
+   * @param blockGroup
+   * @return output blocks to recover
+   */
+  @Override
+  protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) {
+    /**
+     * If more than one blocks (either data or parity) erased, then it's not
+     * edible to recover. We don't have the check here since it will be done
+     * by upper level: ErasreCoder call can be avoid if not possible to recover
+     * at all.
+     */
+    int erasedNum = getNumErasedBlocks(blockGroup);
+    ECBlock[] outputBlocks = new ECBlock[erasedNum];
+
+    int idx = 0;
+    for (int i = 0; i < getNumParityUnits(); i++) {
+      if (blockGroup.getParityBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getParityBlocks()[i];
+      }
+    }
+
+    for (int i = 0; i < getNumDataUnits(); i++) {
+      if (blockGroup.getDataBlocks()[i].isErased()) {
+        outputBlocks[idx++] = blockGroup.getDataBlocks()[i];
+      }
+    }
+
+    return outputBlocks;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
new file mode 100644
index 0000000..db6e3db
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XorErasureEncoder.java
@@ -0,0 +1,45 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder;
+import org.apache.hadoop.io.erasurecode.rawcoder.XorRawEncoder;
+
+/**
+ * Xor erasure encoder that encodes a block group.
+ *
+ * It implements {@link ErasureEncoder}.
+ */
+public class XorErasureEncoder extends AbstractErasureEncoder {
+
+  @Override
+  protected ErasureCodingStep performEncoding(final ECBlockGroup blockGroup) {
+    // May be configured
+    RawErasureEncoder rawEncoder = new XorRawEncoder();
+    rawEncoder.initialize(getNumDataUnits(),
+        getNumParityUnits(), getChunkSize());
+
+    ECBlock[] inputBlocks = getInputBlocks(blockGroup);
+
+    return new ErasureEncodingStep(inputBlocks,
+        getOutputBlocks(blockGroup), rawEncoder);
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 6e07cf1..91a9abf 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -71,7 +71,7 @@ public interface RawErasureCoder {
   public boolean preferNativeBuffer();
 
   /**
-   * Should be called when release this blockcoder. Good chance to release encoding
+   * Should be called when release this coder. Good chance to release encoding
    * or decoding buffers
    */
   public void release();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
new file mode 100644
index 0000000..ca5c1c9
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -0,0 +1,266 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECChunk;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.TestCoderBase;
+
+/**
+ * Erasure coder test base with utilities.
+ */
+public abstract class TestErasureCoderBase extends TestCoderBase {
+  protected Class<? extends ErasureEncoder> encoderClass;
+  protected Class<? extends ErasureDecoder> decoderClass;
+
+  protected int numChunksInBlock = 16;
+
+  /**
+   * It's just a block for this test purpose. We don't use HDFS block here
+   * at all for simple.
+   */
+  protected static class TestBlock extends ECBlock {
+    private ECChunk[] chunks;
+
+    // For simple, just assume the block have the chunks already ready.
+    // In practice we need to read/write chunks from/to the block via file IO.
+    public TestBlock(ECChunk[] chunks) {
+      this.chunks = chunks;
+    }
+  }
+
+  /**
+   * Generating source data, encoding, recovering and then verifying.
+   * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
+   * it supports two kinds of ByteBuffers, one is array backed, the other is
+   * direct ByteBuffer. Have usingDirectBuffer to indicate which case to test.
+   * @param usingDirectBuffer
+   */
+  protected void testCoding(boolean usingDirectBuffer) {
+    this.usingDirectBuffer = usingDirectBuffer;
+
+    ErasureEncoder encoder = createEncoder();
+    // Generate data and encode
+    ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
+    // Backup all the source chunks for later recovering because some coders
+    // may affect the source data.
+    TestBlock[] clonedDataBlocks = cloneBlocksWithData((TestBlock[])
+        blockGroup.getDataBlocks());
+    // Make a copy of a strip for later comparing
+    TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks);
+
+    ErasureCodingStep codingStep = encoder.encode(blockGroup);
+    performCodingStep(codingStep);
+    // Erase the copied sources
+    eraseSomeDataBlocks(clonedDataBlocks);
+
+    //Decode
+    blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
+    ErasureDecoder decoder = createDecoder();
+    codingStep = decoder.decode(blockGroup);
+    performCodingStep(codingStep);
+
+    //Compare
+    compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks());
+  }
+
+  /**
+   * This is typically how a coding step should be performed.
+   * @param codingStep
+   */
+  private void performCodingStep(ErasureCodingStep codingStep) {
+    // Pretend that we're opening these input blocks and output blocks.
+    ECBlock[] inputBlocks = codingStep.getInputBlocks();
+    ECBlock[] outputBlocks = codingStep.getOutputBlocks();
+    // We allocate input and output chunks accordingly.
+    ECChunk[] inputChunks = new ECChunk[inputBlocks.length];
+    ECChunk[] outputChunks = new ECChunk[outputBlocks.length];
+
+    for (int i = 0; i < numChunksInBlock; ++i) {
+      // Pretend that we're reading input chunks from input blocks.
+      for (int j = 0; j < inputBlocks.length; ++j) {
+        inputChunks[j] = ((TestBlock) inputBlocks[j]).chunks[i];
+      }
+
+      // Pretend that we allocate and will write output results to the blocks.
+      for (int j = 0; j < outputBlocks.length; ++j) {
+        outputChunks[j] = allocateOutputChunk();
+        ((TestBlock) outputBlocks[j]).chunks[i] = outputChunks[j];
+      }
+
+      // Given the input chunks and output chunk buffers, just call it !
+      codingStep.performCoding(inputChunks, outputChunks);
+    }
+
+    codingStep.finish();
+  }
+
+  /**
+   * Compare and verify if recovered blocks data are the same with the erased
+   * blocks data.
+   * @param erasedBlocks
+   * @param recoveredBlocks
+   */
+  protected void compareAndVerify(ECBlock[] erasedBlocks,
+                                  ECBlock[] recoveredBlocks) {
+    for (int i = 0; i < erasedBlocks.length; ++i) {
+      compareAndVerify(((TestBlock) erasedBlocks[i]).chunks,
+          ((TestBlock) recoveredBlocks[i]).chunks);
+    }
+  }
+
+  /**
+   * Create erasure encoder for test.
+   * @return
+   */
+  private ErasureEncoder createEncoder() {
+    ErasureEncoder encoder;
+    try {
+      encoder = encoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create encoder", e);
+    }
+
+    encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return encoder;
+  }
+
+  /**
+   * Create the erasure decoder for the test.
+   * @return
+   */
+  private ErasureDecoder createDecoder() {
+    ErasureDecoder decoder;
+    try {
+      decoder = decoderClass.newInstance();
+    } catch (Exception e) {
+      throw new RuntimeException("Failed to create decoder", e);
+    }
+
+    decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    return decoder;
+  }
+
+  /**
+   * Prepare a block group for encoding.
+   * @return
+   */
+  protected ECBlockGroup prepareBlockGroupForEncoding() {
+    ECBlock[] dataBlocks = new TestBlock[numDataUnits];
+    ECBlock[] parityBlocks = new TestBlock[numParityUnits];
+
+    for (int i = 0; i < numDataUnits; i++) {
+      dataBlocks[i] = generateDataBlock();
+    }
+
+    for (int i = 0; i < numParityUnits; i++) {
+      parityBlocks[i] = allocateOutputBlock();
+    }
+
+    return new ECBlockGroup(dataBlocks, parityBlocks);
+  }
+
+  /**
+   * Generate random data and return a data block.
+   * @return
+   */
+  protected ECBlock generateDataBlock() {
+    ECChunk[] chunks = new ECChunk[numChunksInBlock];
+
+    for (int i = 0; i < numChunksInBlock; ++i) {
+      chunks[i] = generateDataChunk();
+    }
+
+    return new TestBlock(chunks);
+  }
+
+  /**
+   * Copy those data blocks that's to be erased for later comparing and
+   * verifying.
+   * @param dataBlocks
+   * @return
+   */
+  protected TestBlock[] copyDataBlocksToErase(TestBlock[] dataBlocks) {
+    TestBlock[] copiedBlocks = new TestBlock[erasedDataIndexes.length];
+
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      copiedBlocks[i] = cloneBlockWithData(dataBlocks[erasedDataIndexes[i]]);
+    }
+
+    return copiedBlocks;
+  }
+
+  /**
+   * Allocate an output block. Note the chunk buffer will be allocated by the
+   * up caller when performing the coding step.
+   * @return
+   */
+  protected TestBlock allocateOutputBlock() {
+    ECChunk[] chunks = new ECChunk[numChunksInBlock];
+
+    return new TestBlock(chunks);
+  }
+
+  /**
+   * Clone blocks with data copied along with, avoiding affecting the original
+   * blocks.
+   * @param blocks
+   * @return
+   */
+  protected static TestBlock[] cloneBlocksWithData(TestBlock[] blocks) {
+    TestBlock[] results = new TestBlock[blocks.length];
+    for (int i = 0; i < blocks.length; ++i) {
+      results[i] = cloneBlockWithData(blocks[i]);
+    }
+
+    return results;
+  }
+
+  /**
+   * Clone exactly a block, avoiding affecting the original block.
+   * @param block
+   * @return a new block
+   */
+  protected static TestBlock cloneBlockWithData(TestBlock block) {
+    ECChunk[] newChunks = cloneChunksWithData(block.chunks);
+
+    return new TestBlock(newChunks);
+  }
+
+  /**
+   * Erase some data blocks specified by the indexes from the data blocks.
+   * @param dataBlocks
+   */
+  protected void eraseSomeDataBlocks(TestBlock[] dataBlocks) {
+    for (int i = 0; i < erasedDataIndexes.length; ++i) {
+      eraseDataFromBlock(dataBlocks, erasedDataIndexes[i]);
+    }
+  }
+
+  /**
+   * Erase data from a block specified by erased index.
+   * @param blocks
+   * @param erasedIndex
+   */
+  protected void eraseDataFromBlock(TestBlock[] blocks, int erasedIndex) {
+    TestBlock theBlock = blocks[erasedIndex];
+    eraseDataFromChunks(theBlock.chunks);
+    theBlock.setErased(true);
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d542a3d1/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java
new file mode 100644
index 0000000..d46fe06
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXorCoder.java
@@ -0,0 +1,50 @@
+/**
+ * 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.io.erasurecode.coder;
+
+import org.junit.Before;
+import org.junit.Test;
+
+/**
+ * Test XOR encoding and decoding.
+ */
+public class TestXorCoder extends TestErasureCoderBase {
+
+  @Before
+  public void setup() {
+    this.encoderClass = XorErasureEncoder.class;
+    this.decoderClass = XorErasureDecoder.class;
+
+    this.numDataUnits = 10;
+    this.numParityUnits = 1;
+    this.erasedDataIndexes = new int[] {0};
+
+    this.numChunksInBlock = 10;
+  }
+
+  @Test
+  public void testCodingNoDirectBuffer() {
+    testCoding(false);
+  }
+
+  @Test
+  public void testCodingDirectBuffer() {
+    testCoding(true);
+  }
+
+}


[17/50] [abbrv] hadoop git commit: HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11707. Add factory to create raw erasure coder.  Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/04011acf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/04011acf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/04011acf

Branch: refs/heads/HDFS-7285
Commit: 04011acf8b50e16d050f99f1cb4a4360838b3c18
Parents: 2c549c4
Author: Kai Zheng <ka...@intel.com>
Authored: Fri Mar 20 15:07:00 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:50 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  3 +-
 .../rawcoder/JRSRawErasureCoderFactory.java     | 34 ++++++++++++++++++
 .../rawcoder/RawErasureCoderFactory.java        | 38 ++++++++++++++++++++
 .../rawcoder/XorRawErasureCoderFactory.java     | 34 ++++++++++++++++++
 4 files changed, 108 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/04011acf/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index e27ff5c..f566f0e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -24,4 +24,5 @@
     HADOOP-11706. Refine a little bit erasure coder API. Contributed by Kai Zheng
     ( Kai Zheng )
 
-
+    HADOOP-11707. Add factory to create raw erasure coder. Contributed by Kai Zheng
+    ( Kai Zheng )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04011acf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
new file mode 100644
index 0000000..d6b40aa
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/JRSRawErasureCoderFactory.java
@@ -0,0 +1,34 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+/**
+ * A raw coder factory for raw Reed-Solomon coder in Java.
+ */
+public class JRSRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder() {
+    return new JRSRawEncoder();
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder() {
+    return new JRSRawDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04011acf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
new file mode 100644
index 0000000..95a1cfe
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java
@@ -0,0 +1,38 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+/**
+ * Raw erasure coder factory that can be used to create raw encoder and decoder.
+ * It helps in configuration since only one factory class is needed to be
+ * configured.
+ */
+public interface RawErasureCoderFactory {
+
+  /**
+   * Create raw erasure encoder.
+   * @return raw erasure encoder
+   */
+  public RawErasureEncoder createEncoder();
+
+  /**
+   * Create raw erasure decoder.
+   * @return raw erasure decoder
+   */
+  public RawErasureDecoder createDecoder();
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/04011acf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java
new file mode 100644
index 0000000..751d16f
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XorRawErasureCoderFactory.java
@@ -0,0 +1,34 @@
+/**
+ * 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.io.erasurecode.rawcoder;
+
+/**
+ * A raw coder factory for raw XOR coder.
+ */
+public class XorRawErasureCoderFactory implements RawErasureCoderFactory {
+
+  @Override
+  public RawErasureEncoder createEncoder() {
+    return new XorRawEncoder();
+  }
+
+  @Override
+  public RawErasureDecoder createDecoder() {
+    return new XorRawDecoder();
+  }
+}


[30/50] [abbrv] hadoop git commit: HDFS-7617. Add unit tests for editlog transactions for EC. Contributed by Hui Zheng.

Posted by zh...@apache.org.
HDFS-7617. Add unit tests for editlog transactions for EC. Contributed by Hui Zheng.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/ade2d75f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/ade2d75f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/ade2d75f

Branch: refs/heads/HDFS-7285
Commit: ade2d75f8864f21df5ab7368da38476efb7a3560
Parents: c69f784
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Mar 31 10:46:04 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:48:56 2015 -0700

----------------------------------------------------------------------
 .../server/namenode/TestFSEditLogLoader.java    | 157 +++++++++++++++++++
 1 file changed, 157 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/ade2d75f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
index 833ef95..d3cb749 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestFSEditLogLoader.java
@@ -39,14 +39,18 @@ import org.apache.commons.logging.impl.Log4JLogger;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSTestUtil;
 import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.server.common.Storage.StorageDirectory;
 import org.apache.hadoop.hdfs.server.namenode.FSEditLogLoader.EditLogValidation;
 import org.apache.hadoop.hdfs.server.namenode.NNStorage.NameNodeDirType;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.test.PathUtils;
 import org.apache.log4j.Level;
@@ -414,4 +418,157 @@ public class TestFSEditLogLoader {
           fromByte(code), FSEditLogOpCodes.fromByte(code));
     }
   }
+
+  @Test
+  public void testAddNewStripedBlock() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/ec";
+      String testFile = "testfile_001";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser1";
+      String clientMachine = "testMachine1";
+      long blkId = 1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+      short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
+      short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
+
+      //set the storage policy of the directory
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      fs.setStoragePolicy(new Path(testDir),
+          HdfsConstants.EC_STORAGE_POLICY_NAME);
+
+      // Create a file with striped block
+      Path p = new Path(testFilePath);
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      fns.leaveSafeMode();
+
+      // Add a striped block to the file
+      BlockInfoStriped stripedBlk = new BlockInfoStriped(
+          new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.getStripedBlocksFeature().addBlock(stripedBlk);
+      fns.getEditLog().logAddBlock(testFilePath, file);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      //If the block by loaded is the same as above it means that
+      //we have successfully applied the edit log to the fsimage.
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+
+      INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
+          .getINode(testFilePath);
+
+      assertTrue(inodeLoaded.isWithStripedBlocks());
+
+      BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
+      assertEquals(1, blks.length);
+      assertEquals(blkId, blks[0].getBlockId());
+      assertEquals(blkNumBytes, blks[0].getNumBytes());
+      assertEquals(timestamp, blks[0].getGenerationStamp());
+      assertEquals(blockNum, blks[0].getDataBlockNum());
+      assertEquals(parityNum, blks[0].getParityBlockNum());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
+
+  @Test
+  public void testUpdateStripedBlocks() throws IOException{
+    // start a cluster
+    Configuration conf = new HdfsConfiguration();
+    MiniDFSCluster cluster = null;
+    try {
+      cluster = new MiniDFSCluster.Builder(conf).numDataNodes(9)
+          .build();
+      cluster.waitActive();
+      DistributedFileSystem fs = cluster.getFileSystem();
+      FSNamesystem fns = cluster.getNamesystem();
+
+      String testDir = "/ec";
+      String testFile = "testfile_002";
+      String testFilePath = testDir + "/" + testFile;
+      String clientName = "testUser2";
+      String clientMachine = "testMachine2";
+      long blkId = 1;
+      long blkNumBytes = 1024;
+      long timestamp = 1426222918;
+      short blockNum = HdfsConstants.NUM_DATA_BLOCKS;
+      short parityNum = HdfsConstants.NUM_PARITY_BLOCKS;
+
+      //set the storage policy of the directory
+      fs.mkdir(new Path(testDir), new FsPermission("755"));
+      fs.setStoragePolicy(new Path(testDir),
+          HdfsConstants.EC_STORAGE_POLICY_NAME);
+
+      //create a file with striped blocks
+      Path p = new Path(testFilePath);
+      DFSTestUtil.createFile(fs, p, 0, (short) 1, 1);
+      BlockInfoStriped stripedBlk = new BlockInfoStriped(
+          new Block(blkId, blkNumBytes, timestamp), blockNum, parityNum);
+      INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath);
+      file.toUnderConstruction(clientName, clientMachine);
+      file.getStripedBlocksFeature().addBlock(stripedBlk);
+      fns.getEditLog().logAddBlock(testFilePath, file);
+      file.toCompleteFile(System.currentTimeMillis());
+      fns.enterSafeMode(false);
+      fns.saveNamespace(0, 0);
+      fns.leaveSafeMode();
+
+      //update the last block
+      long newBlkNumBytes = 1024*8;
+      long newTimestamp = 1426222918+3600;
+      file.toUnderConstruction(clientName, clientMachine);
+      file.getLastBlock().setNumBytes(newBlkNumBytes);
+      file.getLastBlock().setGenerationStamp(newTimestamp);
+      fns.getEditLog().logUpdateBlocks(testFilePath, file, true);
+      file.toCompleteFile(System.currentTimeMillis());
+
+      //After the namenode restarts if the block by loaded is the same as above
+      //(new block size and timestamp) it means that we have successfully
+      //applied the edit log to the fsimage.
+      cluster.restartNameNodes();
+      cluster.waitActive();
+      fns = cluster.getNamesystem();
+
+      INodeFile inodeLoaded = (INodeFile)fns.getFSDirectory()
+          .getINode(testFilePath);
+
+      assertTrue(inodeLoaded.isWithStripedBlocks());
+
+      BlockInfoStriped[] blks = (BlockInfoStriped[])inodeLoaded.getBlocks();
+      assertEquals(1, blks.length);
+      assertEquals(blkId, blks[0].getBlockId());
+      assertEquals(newBlkNumBytes, blks[0].getNumBytes());
+      assertEquals(newTimestamp, blks[0].getGenerationStamp());
+      assertEquals(blockNum, blks[0].getDataBlockNum());
+      assertEquals(parityNum, blks[0].getParityBlockNum());
+
+      cluster.shutdown();
+      cluster = null;
+    } finally {
+      if (cluster != null) {
+        cluster.shutdown();
+      }
+    }
+  }
 }


[43/50] [abbrv] hadoop git commit: HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B)

Posted by zh...@apache.org.
HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from NameNode (Contributed by Vinayakumar B)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bc69d45a
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bc69d45a
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bc69d45a

Branch: refs/heads/HDFS-7285
Commit: bc69d45af3a81f4c1e61ae7829265391d2781e36
Parents: e67086d
Author: Vinayakumar B <vi...@apache.org>
Authored: Wed Apr 8 12:48:59 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:56:56 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        |  5 ++-
 .../java/org/apache/hadoop/hdfs/DFSClient.java  | 14 ++++++
 .../hadoop/hdfs/protocol/ClientProtocol.java    | 10 +++++
 ...tNamenodeProtocolServerSideTranslatorPB.java | 19 ++++++++
 .../ClientNamenodeProtocolTranslatorPB.java     | 18 ++++++++
 .../apache/hadoop/hdfs/protocolPB/PBHelper.java | 47 +++++++++++++++++++-
 .../hdfs/server/namenode/FSNamesystem.java      | 31 +++++++++++++
 .../hdfs/server/namenode/NameNodeRpcServer.java |  7 +++
 .../src/main/proto/ClientNamenodeProtocol.proto | 10 +++++
 .../hadoop-hdfs/src/main/proto/hdfs.proto       | 28 ++++++++++++
 .../hadoop/hdfs/TestErasureCodingZones.java     | 38 +++++++++++++++-
 11 files changed, 223 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 9927ccf..7423033 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -49,4 +49,7 @@
     (Hui Zheng via Zhe Zhang)
 
     HDFS-7839. Erasure coding: implement facilities in NameNode to create and
-    manage EC zones (Zhe Zhang)
\ No newline at end of file
+    manage EC zones (Zhe Zhang)
+
+    HDFS-8023. Erasure Coding: retrieve eraure coding schema for a file from
+    NameNode (vinayakumarb)
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
index 2ef1d36..b2a69dd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSClient.java
@@ -117,6 +117,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.EncryptionZoneIterator;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
@@ -3091,6 +3092,19 @@ public class DFSClient implements java.io.Closeable, RemotePeerFactory,
     }
   }
 
+  public ECInfo getErasureCodingInfo(String src) throws IOException {
+    checkOpen();
+    TraceScope scope = getPathTraceScope("getErasureCodingInfo", src);
+    try {
+      return namenode.getErasureCodingInfo(src);
+    } catch (RemoteException re) {
+      throw re.unwrapRemoteException(AccessControlException.class,
+          FileNotFoundException.class, UnresolvedPathException.class);
+    } finally {
+      scope.close();
+    }
+  }
+
   public DFSInotifyEventInputStream getInotifyEventStream() throws IOException {
     return new DFSInotifyEventInputStream(traceSampler, namenode);
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
index 8efe344..45d92f3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java
@@ -1464,4 +1464,14 @@ public interface ClientProtocol {
    */
   @Idempotent
   public EventBatchList getEditsFromTxid(long txid) throws IOException;
+
+  /**
+   * Gets the ECInfo for the specified file/directory
+   * 
+   * @param src
+   * @return Returns the ECInfo if the file/directory is erasure coded, null otherwise
+   * @throws IOException
+   */
+  @Idempotent
+  public ECInfo getErasureCodingInfo(String src) throws IOException;
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
index a209539..1493e52 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolServerSideTranslatorPB.java
@@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.CachePoolEntry;
 import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.CorruptFileBlocks;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus;
@@ -107,6 +108,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidResponseProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
@@ -1511,4 +1514,20 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
       throw new ServiceException(e);
     }
   }
+
+  @Override
+  public GetErasureCodingInfoResponseProto getErasureCodingInfo(RpcController controller,
+      GetErasureCodingInfoRequestProto request) throws ServiceException {
+    try {
+      ECInfo ecInfo = server.getErasureCodingInfo(request.getSrc());
+      GetErasureCodingInfoResponseProto.Builder resBuilder = GetErasureCodingInfoResponseProto
+          .newBuilder();
+      if (ecInfo != null) {
+        resBuilder.setECInfo(PBHelper.convertECInfo(ecInfo));
+      }
+      return resBuilder.build();
+    } catch (IOException e) {
+      throw new ServiceException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
index 43a0322..568da68 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java
@@ -58,6 +58,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
@@ -107,6 +108,8 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDat
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetDatanodeStorageReportRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetEditsFromTxidRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoRequestProto;
+import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetErasureCodingInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoRequestProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileInfoResponseProto;
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.GetFileLinkInfoRequestProto;
@@ -1532,4 +1535,19 @@ public class ClientNamenodeProtocolTranslatorPB implements
       throw ProtobufHelper.getRemoteException(e);
     }
   }
+
+  @Override
+  public ECInfo getErasureCodingInfo(String src) throws IOException {
+    GetErasureCodingInfoRequestProto req = GetErasureCodingInfoRequestProto.newBuilder()
+        .setSrc(src).build();
+    try {
+      GetErasureCodingInfoResponseProto res = rpcProxy.getErasureCodingInfo(null, req);
+      if (res.hasECInfo()) {
+        return PBHelper.convertECInfo(res.getECInfo());
+      }
+      return null;
+    } catch (ServiceException e) {
+      throw ProtobufHelper.getRemoteException(e);
+    }
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
index cda708f..afdb302 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelper.java
@@ -29,7 +29,11 @@ import java.io.InputStream;
 import java.util.ArrayList;
 import java.util.Arrays;
 import java.util.EnumSet;
+import java.util.HashMap;
 import java.util.List;
+import java.util.Map;
+import java.util.Map.Entry;
+import java.util.Set;
 
 import org.apache.hadoop.fs.CacheFlag;
 import org.apache.hadoop.fs.ContentSummary;
@@ -76,6 +80,7 @@ import org.apache.hadoop.hdfs.protocol.FsPermissionExtension;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.RollingUpgradeAction;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.HdfsFileStatus;
 import org.apache.hadoop.hdfs.protocol.HdfsLocatedFileStatus;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
@@ -112,7 +117,6 @@ import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.Rollin
 import org.apache.hadoop.hdfs.protocol.proto.ClientNamenodeProtocolProtos.SafeModeActionProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmIdProto;
 import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.ShortCircuitShmSlotProto;
-import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BalancerBandwidthCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto;
 import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockIdCommandProto;
@@ -146,6 +150,9 @@ import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeLocalInfoProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeStorageProto.StorageState;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DirectoryListingProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECInfoProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaOptionEntryProto;
+import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ECSchemaProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExportedBlockKeysProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ExtendedBlockProto;
 import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto;
@@ -228,6 +235,7 @@ import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.SlotId;
 import org.apache.hadoop.hdfs.util.ExactSizeInputStream;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.security.proto.SecurityProtos.TokenProto;
 import org.apache.hadoop.security.token.Token;
 import org.apache.hadoop.util.DataChecksum;
@@ -3097,4 +3105,41 @@ public class PBHelper {
         setId(context.getReportId()).
         build();
   }
+
+  public static ECInfo convertECInfo(ECInfoProto ecInfoProto) {
+    return new ECInfo(ecInfoProto.getSrc(),
+        convertECSchema(ecInfoProto.getSchema()));
+  }
+
+  public static ECInfoProto convertECInfo(ECInfo ecInfo) {
+    return ECInfoProto.newBuilder().setSrc(ecInfo.getSrc())
+        .setSchema(convertECSchema(ecInfo.getSchema())).build();
+  }
+
+  public static ECSchema convertECSchema(ECSchemaProto schema) {
+    List<ECSchemaOptionEntryProto> optionsList = schema.getOptionsList();
+    Map<String, String> options = new HashMap<>(optionsList.size());
+    for (ECSchemaOptionEntryProto option : optionsList) {
+      options.put(option.getKey(), option.getValue());
+    }
+    // include chunksize in options.
+    options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(schema.getChunkSize()));
+    return new ECSchema(schema.getSchemaName(), schema.getCodecName(),
+        schema.getDataUnits(), schema.getParityUnits(), options);
+  }
+
+  public static ECSchemaProto convertECSchema(ECSchema schema) {
+    ECSchemaProto.Builder builder = ECSchemaProto.newBuilder()
+        .setSchemaName(schema.getSchemaName())
+        .setCodecName(schema.getCodecName())
+        .setDataUnits(schema.getNumDataUnits())
+        .setParityUnits(schema.getNumParityUnits())
+        .setChunkSize(schema.getChunkSize());
+    Set<Entry<String, String>> entrySet = schema.getOptions().entrySet();
+    for (Entry<String, String> entry : entrySet) {
+      builder.addOptions(ECSchemaOptionEntryProto.newBuilder()
+          .setKey(entry.getKey()).setValue(entry.getValue()).build());
+    }
+    return builder.build();
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 79d6f21..4c86bd3 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -180,6 +180,7 @@ import org.apache.hadoop.hdfs.protocol.ClientProtocol;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants;
@@ -261,6 +262,7 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary;
 import org.apache.hadoop.io.EnumSetWritable;
 import org.apache.hadoop.io.IOUtils;
 import org.apache.hadoop.io.Text;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.ipc.RetriableException;
 import org.apache.hadoop.ipc.RetryCache;
 import org.apache.hadoop.ipc.Server;
@@ -8138,6 +8140,35 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     logAuditEvent(true, "createErasureCodingZone", srcArg, null, resultingStat);
   }
 
+  /**
+   * Get the erasure coding information for specified src
+   */
+  ECInfo getErasureCodingInfo(String src) throws AccessControlException,
+      UnresolvedLinkException, IOException {
+    checkOperation(OperationCategory.READ);
+    final byte[][] pathComponents = FSDirectory
+        .getPathComponentsForReservedPath(src);
+    final FSPermissionChecker pc = getPermissionChecker();
+    readLock();
+    try {
+      checkOperation(OperationCategory.READ);
+      src = dir.resolvePath(pc, src, pathComponents);
+      final INodesInPath iip = dir.getINodesInPath(src, true);
+      if (isPermissionEnabled) {
+        dir.checkPathAccess(pc, iip, FsAction.READ);
+      }
+      if (dir.getECPolicy(iip)) {
+        // TODO HDFS-8074 and HDFS-7859 : To get from loaded schemas
+        Map<String, String> options = new HashMap<String, String>();
+        ECSchema defaultSchema = new ECSchema("RS-6-3", "rs", 6, 3, options);
+        return new ECInfo(src, defaultSchema);
+      }
+    } finally {
+      readUnlock();
+    }
+    return null;
+  }
+
   void setXAttr(String src, XAttr xAttr, EnumSet<XAttrSetFlag> flag,
                 boolean logRetryCache)
       throws IOException {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
index ce24662..fe9b901 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeRpcServer.java
@@ -84,6 +84,7 @@ import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
 import org.apache.hadoop.hdfs.protocol.DirectoryListing;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.protocol.EncryptionZone;
 import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.protocol.FSLimitException;
@@ -2042,4 +2043,10 @@ class NameNodeRpcServer implements NamenodeProtocols {
     namesystem.checkSuperuserPrivilege();
     nn.spanReceiverHost.removeSpanReceiver(id);
   }
+
+  @Override // ClientNameNodeProtocol
+  public ECInfo getErasureCodingInfo(String src) throws IOException {
+    checkNNStartup();
+    return namesystem.getErasureCodingInfo(src);
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
index 183aff8..9488aed 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/ClientNamenodeProtocol.proto
@@ -721,6 +721,14 @@ message CreateErasureCodingZoneRequestProto {
 message CreateErasureCodingZoneResponseProto {
 }
 
+message GetErasureCodingInfoRequestProto {
+  required string src = 1;
+}
+
+message GetErasureCodingInfoResponseProto {
+  optional ECInfoProto ECInfo = 1;
+}
+
 service ClientNamenodeProtocol {
   rpc getBlockLocations(GetBlockLocationsRequestProto)
       returns(GetBlockLocationsResponseProto);
@@ -869,4 +877,6 @@ service ClientNamenodeProtocol {
       returns(GetCurrentEditLogTxidResponseProto);
   rpc getEditsFromTxid(GetEditsFromTxidRequestProto)
       returns(GetEditsFromTxidResponseProto);
+  rpc getErasureCodingInfo(GetErasureCodingInfoRequestProto)
+      returns(GetErasureCodingInfoResponseProto);
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
index 67e2058..1314ea0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/hdfs.proto
@@ -620,3 +620,31 @@ message RollingUpgradeStatusProto {
   required string blockPoolId = 1;
   optional bool finalized = 2 [default = false];
 }
+
+/**
+ * ECSchema options entry
+ */
+message ECSchemaOptionEntryProto {
+  required string key = 1;
+  required string value = 2;
+}
+
+/**
+ * ECShema for erasurecoding
+ */
+message ECSchemaProto {
+  required string schemaName = 1;
+  required string codecName = 2;
+  required uint32 dataUnits = 3;
+  required uint32 parityUnits = 4;
+  required uint32 chunkSize = 5;
+  repeated ECSchemaOptionEntryProto options = 6;
+}
+
+/**
+ * ECInfo
+ */
+message ECInfoProto {
+ required string src = 1;
+ required ECSchemaProto schema = 2;
+}
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bc69d45a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
index 49f08eef..bdca915 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingZones.java
@@ -20,8 +20,10 @@ package org.apache.hadoop.hdfs;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.Path;
 import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.hdfs.protocol.ECInfo;
 import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
 import org.apache.hadoop.hdfs.server.namenode.INode;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.junit.After;
 import org.junit.Before;
 import org.junit.Test;
@@ -29,8 +31,7 @@ import org.junit.Test;
 import java.io.IOException;
 
 import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains;
-import static org.junit.Assert.assertTrue;
-import static org.junit.Assert.fail;
+import static org.junit.Assert.*;
 
 public class TestErasureCodingZones {
   private final int NUM_OF_DATANODES = 3;
@@ -148,4 +149,37 @@ public class TestErasureCodingZones {
           "destination have different erasure coding policies", e);
     }
   }
+
+  @Test
+  public void testGetErasureCodingInfo() throws Exception {
+    String src = "/ec";
+    final Path ecDir = new Path(src);
+    fs.mkdir(ecDir, FsPermission.getDirDefault());
+    // dir ECInfo before creating ec zone
+    assertNull(fs.getClient().getErasureCodingInfo(src));
+    // dir ECInfo after creating ec zone
+    fs.getClient().createErasureCodingZone(src);
+    verifyErasureCodingInfo(src);
+    fs.create(new Path(ecDir, "/child1")).close();
+    // verify for the files in ec zone
+    verifyErasureCodingInfo(src + "/child1");
+  }
+
+  private void verifyErasureCodingInfo(String src) throws IOException {
+    ECInfo ecInfo = fs.getClient().getErasureCodingInfo(src);
+    assertNotNull("ECInfo should have been non-null", ecInfo);
+    assertEquals(src, ecInfo.getSrc());
+    ECSchema schema = ecInfo.getSchema();
+    assertNotNull(schema);
+    assertEquals("Default schema should be returned", "RS-6-3",
+        schema.getSchemaName());
+    assertEquals("Default codec(rs) should be returned", "rs",
+        schema.getCodecName());
+    assertEquals("Default numDataUnits should be used", 6,
+        schema.getNumDataUnits());
+    assertEquals("Default numParityUnits should be used", 3,
+        schema.getNumParityUnits());
+    assertEquals("Default chunkSize should be used",
+        ECSchema.DEFAULT_CHUNK_SIZE, schema.getChunkSize());
+  }
 }
\ No newline at end of file


[29/50] [abbrv] hadoop git commit: HDFS-8027. Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits (Vinayakumar B)

Posted by zh...@apache.org.
HDFS-8027. Erasure Coding: Update CHANGES-HDFS-7285.txt with branch commits (Vinayakumar B)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/c69f7847
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/c69f7847
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/c69f7847

Branch: refs/heads/HDFS-7285
Commit: c69f78472b2f531caa5e199b6ef38d82672b86ee
Parents: 2c14bbe
Author: Vinayakumar B <vi...@intel.com>
Authored: Tue Mar 31 15:12:09 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:48:56 2015 -0700

----------------------------------------------------------------------
 .../hadoop-hdfs/CHANGES-HDFS-EC-7285.txt        | 40 +++++++++++++++++++-
 1 file changed, 39 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/c69f7847/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
index 21e4c03..a686315 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt
@@ -3,6 +3,44 @@
     HDFS-7347. Configurable erasure coding policy for individual files and
     directories ( Zhe Zhang via vinayakumarb )
 
-    HDFS-7716. Add a test for BlockGroup support in FSImage.
+    HDFS-7339. Representing striped block groups in NameNode with hierarchical
+    naming protocol ( Zhe Zhang )
+
+    HDFS-7652. Process block reports for erasure coded blocks (Zhe Zhang)
+
+    HDFS-7716. Erasure Coding: extend BlockInfo to handle EC info (Jing Zhao)
+
+    HDFS-7749. Erasure Coding: Add striped block support in INodeFile (Jing Zhao)
+
+    HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode
+    (Jing Zhao via Zhe Zhang)
+
+    HDFS-7872. Erasure Coding: INodeFile.dumpTreeRecursively() supports to print
+    striped blocks (Takuya Fukudome via jing9)
+
+    HDFS-7853. Erasure coding: extend LocatedBlocks to support reading from
+    striped files (Jing Zhao)
+
+    HDFS-7826. Erasure Coding: Update INodeFile quota computation for striped
+    blocks ( Kai Sasaki via jing9 )
+
+    HDFS-7912. Erasure Coding: track BlockInfo instead of Block in
+    UnderReplicatedBlocks and PendingReplicationBlocks (Jing Zhao)
+
+    HDFS-7369. Erasure coding: distribute recovery work for striped blocks to
+    DataNode (Zhe Zhang)
+
+    HDFS-7864. Erasure Coding: Update safemode calculation for striped blocks
+    (GAO Rui via jing9)
+
+    HDFS-7827. Erasure Coding: support striped blocks in non-protobuf fsimage
+    ( Hui Zheng via jing9 )
+
+    HDFS-7616. Add a test for BlockGroup support in FSImage.
     (Takuya Fukudome via szetszwo)
 
+    HDFS-7907. Erasure Coding: track invalid, corrupt, and under-recovery striped
+    blocks in NameNode (Jing Zhao)
+
+    HDFS-8005. Erasure Coding: simplify striped block recovery work computation
+    and add tests (Jing Zhao)
\ No newline at end of file


[40/50] [abbrv] hadoop git commit: HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code ( Contributed by Kai Zheng)

Posted by zh...@apache.org.
HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code ( Contributed by Kai Zheng)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/42cb9d8c
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/42cb9d8c
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/42cb9d8c

Branch: refs/heads/HDFS-7285
Commit: 42cb9d8c1d5b4074b30591044420f4bed74e815a
Parents: b2306c4
Author: Vinayakumar B <vi...@apache.org>
Authored: Tue Apr 7 16:05:22 2015 +0530
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:15:36 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  3 +
 .../hadoop/io/erasurecode/ECBlockGroup.java     | 18 ++++
 .../erasurecode/codec/AbstractErasureCodec.java | 88 +++++++++++++++++++
 .../io/erasurecode/codec/ErasureCodec.java      | 56 ++++++++++++
 .../io/erasurecode/codec/RSErasureCodec.java    | 38 +++++++++
 .../io/erasurecode/codec/XORErasureCodec.java   | 45 ++++++++++
 .../erasurecode/coder/AbstractErasureCoder.java |  7 ++
 .../io/erasurecode/coder/ErasureCoder.java      |  7 ++
 .../io/erasurecode/grouper/BlockGrouper.java    | 90 ++++++++++++++++++++
 9 files changed, 352 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 7716728..c72394e 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -37,3 +37,6 @@
 
     HADOOP-11805 Better to rename some raw erasure coders. Contributed by Kai Zheng
     ( Kai Zheng )
+
+    HADOOP-11645. Erasure Codec API covering the essential aspects for an erasure code
+    ( Kai Zheng via vinayakumarb )

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
index 2c851a5..0a86907 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java
@@ -79,4 +79,22 @@ public class ECBlockGroup {
     return false;
   }
 
+  /**
+   * Get erased blocks count
+   * @return
+   */
+  public int getErasedCount() {
+    int erasedCount = 0;
+
+    for (ECBlock dataBlock : dataBlocks) {
+      if (dataBlock.isErased()) erasedCount++;
+    }
+
+    for (ECBlock parityBlock : parityBlocks) {
+      if (parityBlock.isErased()) erasedCount++;
+    }
+
+    return erasedCount;
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
new file mode 100644
index 0000000..9993786
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java
@@ -0,0 +1,88 @@
+/**
+ * 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.io.erasurecode.codec;
+
+import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.coder.*;
+import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
+
+/**
+ * Abstract Erasure Codec that implements {@link ErasureCodec}.
+ */
+public abstract class AbstractErasureCodec extends Configured
+    implements ErasureCodec {
+
+  private ECSchema schema;
+
+  @Override
+  public void setSchema(ECSchema schema) {
+    this.schema = schema;
+  }
+
+  public String getName() {
+    return schema.getCodecName();
+  }
+
+  protected ECSchema getSchema() {
+    return schema;
+  }
+
+  @Override
+  public BlockGrouper createBlockGrouper() {
+    BlockGrouper blockGrouper = new BlockGrouper();
+    blockGrouper.setSchema(getSchema());
+
+    return blockGrouper;
+  }
+
+  @Override
+  public ErasureCoder createEncoder() {
+    ErasureCoder encoder = doCreateEncoder();
+    prepareErasureCoder(encoder);
+    return encoder;
+  }
+
+  /**
+   * Create a new encoder instance to be initialized afterwards.
+   * @return encoder
+   */
+  protected abstract ErasureCoder doCreateEncoder();
+
+  @Override
+  public ErasureCoder createDecoder() {
+    ErasureCoder decoder = doCreateDecoder();
+    prepareErasureCoder(decoder);
+    return decoder;
+  }
+
+  /**
+   * Create a new decoder instance to be initialized afterwards.
+   * @return decoder
+   */
+  protected abstract ErasureCoder doCreateDecoder();
+
+  private void prepareErasureCoder(ErasureCoder erasureCoder) {
+    if (getSchema() == null) {
+      throw new RuntimeException("No schema been set yet");
+    }
+
+    erasureCoder.setConf(getConf());
+    erasureCoder.initialize(getSchema());
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
new file mode 100644
index 0000000..e639484
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.java
@@ -0,0 +1,56 @@
+/**
+ * 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.io.erasurecode.codec;
+
+import org.apache.hadoop.conf.Configurable;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.io.erasurecode.grouper.BlockGrouper;
+
+/**
+ * Erasure Codec API that's to cover the essential specific aspects of a code.
+ * Currently it cares only block grouper and erasure coder. In future we may
+ * add more aspects here to make the behaviors customizable.
+ */
+public interface ErasureCodec extends Configurable {
+
+  /**
+   * Set EC schema to be used by this codec.
+   * @param schema
+   */
+  public void setSchema(ECSchema schema);
+
+  /**
+   * Create block grouper
+   * @return block grouper
+   */
+  public BlockGrouper createBlockGrouper();
+
+  /**
+   * Create Erasure Encoder
+   * @return erasure encoder
+   */
+  public ErasureCoder createEncoder();
+
+  /**
+   * Create Erasure Decoder
+   * @return erasure decoder
+   */
+  public ErasureCoder createDecoder();
+
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
new file mode 100644
index 0000000..9e91b60
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java
@@ -0,0 +1,38 @@
+/**
+ * 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.io.erasurecode.codec;
+
+import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.io.erasurecode.coder.RSErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder;
+
+/**
+ * A Reed-Solomon erasure codec.
+ */
+public class RSErasureCodec extends AbstractErasureCodec {
+
+  @Override
+  protected ErasureCoder doCreateEncoder() {
+    return new RSErasureEncoder();
+  }
+
+  @Override
+  protected ErasureCoder doCreateDecoder() {
+    return new RSErasureDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
new file mode 100644
index 0000000..0f726d7
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java
@@ -0,0 +1,45 @@
+/**
+ * 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.io.erasurecode.codec;
+
+import org.apache.hadoop.io.erasurecode.ECSchema;
+import org.apache.hadoop.io.erasurecode.coder.ErasureCoder;
+import org.apache.hadoop.io.erasurecode.coder.XORErasureDecoder;
+import org.apache.hadoop.io.erasurecode.coder.XORErasureEncoder;
+
+/**
+ * A XOR erasure codec.
+ */
+public class XORErasureCodec extends AbstractErasureCodec {
+
+  @Override
+  public void setSchema(ECSchema schema) {
+    super.setSchema(schema);
+    assert(schema.getNumParityUnits() == 1);
+  }
+
+  @Override
+  protected ErasureCoder doCreateEncoder() {
+    return new XORErasureEncoder();
+  }
+
+  @Override
+  protected ErasureCoder doCreateDecoder() {
+    return new XORErasureDecoder();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
index 0e4de89..e5bf11a 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.conf.Configured;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoder;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureCoderFactory;
 import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder;
@@ -105,6 +106,12 @@ public abstract class AbstractErasureCoder
   }
 
   @Override
+  public void initialize(ECSchema schema) {
+      initialize(schema.getNumDataUnits(), schema.getNumParityUnits(),
+          schema.getChunkSize());
+  }
+
+  @Override
   public int getNumDataUnits() {
     return numDataUnits;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
index fb90156..64a82ea 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -19,6 +19,7 @@ package org.apache.hadoop.io.erasurecode.coder;
 
 import org.apache.hadoop.conf.Configurable;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
 
 /**
  * An erasure coder to perform encoding or decoding given a group. Generally it
@@ -45,6 +46,12 @@ public interface ErasureCoder extends Configurable {
   public void initialize(int numDataUnits, int numParityUnits, int chunkSize);
 
   /**
+   * Initialize with an EC schema.
+   * @param schema
+   */
+  public void initialize(ECSchema schema);
+
+  /**
    * The number of data input units for the coding. A unit can be a byte,
    * chunk or buffer or even a block.
    * @return count of data input units

http://git-wip-us.apache.org/repos/asf/hadoop/blob/42cb9d8c/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
new file mode 100644
index 0000000..bdc1624
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/grouper/BlockGrouper.java
@@ -0,0 +1,90 @@
+/**
+ * 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.io.erasurecode.grouper;
+
+import org.apache.hadoop.io.erasurecode.ECBlock;
+import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECSchema;
+
+/**
+ * As part of a codec, to handle how to form a block group for encoding
+ * and provide instructions on how to recover erased blocks from a block group
+ */
+public class BlockGrouper {
+
+  private ECSchema schema;
+
+  /**
+   * Set EC schema.
+   * @param schema
+   */
+  public void setSchema(ECSchema schema) {
+    this.schema = schema;
+  }
+
+  /**
+   * Get EC schema.
+   * @return
+   */
+  protected ECSchema getSchema() {
+    return schema;
+  }
+
+  /**
+   * Get required data blocks count in a BlockGroup.
+   * @return count of required data blocks
+   */
+  public int getRequiredNumDataBlocks() {
+    return schema.getNumDataUnits();
+  }
+
+  /**
+   * Get required parity blocks count in a BlockGroup.
+   * @return count of required parity blocks
+   */
+  public int getRequiredNumParityBlocks() {
+    return schema.getNumParityUnits();
+  }
+
+  /**
+   * Calculating and organizing BlockGroup, to be called by ECManager
+   * @param dataBlocks Data blocks to compute parity blocks against
+   * @param parityBlocks To be computed parity blocks
+   * @return
+   */
+  public ECBlockGroup makeBlockGroup(ECBlock[] dataBlocks,
+                                     ECBlock[] parityBlocks) {
+
+    ECBlockGroup blockGroup = new ECBlockGroup(dataBlocks, parityBlocks);
+    return blockGroup;
+  }
+
+  /**
+   * Given a BlockGroup, tell if any of the missing blocks can be recovered,
+   * to be called by ECManager
+   * @param blockGroup a blockGroup that may contain erased blocks but not sure
+   *                   recoverable or not
+   * @return true if any erased block recoverable, false otherwise
+   */
+  public boolean anyRecoverable(ECBlockGroup blockGroup) {
+    int erasedCount = blockGroup.getErasedCount();
+
+    return erasedCount > 0 && erasedCount <= getRequiredNumParityBlocks();
+  }
+
+}


[15/50] [abbrv] hadoop git commit: HADOOP-11706 Refine a little bit erasure coder API

Posted by zh...@apache.org.
HADOOP-11706 Refine a little bit erasure coder API


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/068a5d5b
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/068a5d5b
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/068a5d5b

Branch: refs/heads/HDFS-7285
Commit: 068a5d5b29d34d3a101531e820a43860952b92ee
Parents: c3b0338
Author: Kai Zheng <ka...@intel.com>
Authored: Wed Mar 18 19:21:37 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:42:00 2015 -0700

----------------------------------------------------------------------
 .../io/erasurecode/coder/ErasureCoder.java      |  4 +++-
 .../erasurecode/rawcoder/RawErasureCoder.java   |  4 +++-
 .../hadoop/io/erasurecode/TestCoderBase.java    | 17 +++++++++++++---
 .../erasurecode/coder/TestErasureCoderBase.java | 21 +++-----------------
 .../erasurecode/rawcoder/TestJRSRawCoder.java   | 12 +++++------
 .../erasurecode/rawcoder/TestRawCoderBase.java  |  2 ++
 6 files changed, 31 insertions(+), 29 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/068a5d5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
index 68875c0..c5922f3 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/ErasureCoder.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configurable;
+
 /**
  * An erasure coder to perform encoding or decoding given a group. Generally it
  * involves calculating necessary internal steps according to codec logic. For
@@ -31,7 +33,7 @@ package org.apache.hadoop.io.erasurecode.coder;
  * of multiple coding steps.
  *
  */
-public interface ErasureCoder {
+public interface ErasureCoder extends Configurable {
 
   /**
    * Initialize with the important parameters for the code.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/068a5d5b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
index 91a9abf..9af5b6c 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java
@@ -17,6 +17,8 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.conf.Configurable;
+
 /**
  * RawErasureCoder is a common interface for {@link RawErasureEncoder} and
  * {@link RawErasureDecoder} as both encoder and decoder share some properties.
@@ -31,7 +33,7 @@ package org.apache.hadoop.io.erasurecode.rawcoder;
  * low level constructs, since it only takes care of the math calculation with
  * a group of byte buffers.
  */
-public interface RawErasureCoder {
+public interface RawErasureCoder extends Configurable {
 
   /**
    * Initialize with the important parameters for the code.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/068a5d5b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 194413a..22fd98d 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -17,11 +17,12 @@
  */
 package org.apache.hadoop.io.erasurecode;
 
+import org.apache.hadoop.conf.Configuration;
+
 import java.nio.ByteBuffer;
 import java.util.Arrays;
 import java.util.Random;
 
-import static org.junit.Assert.assertArrayEquals;
 import static org.junit.Assert.assertTrue;
 
 /**
@@ -31,6 +32,7 @@ import static org.junit.Assert.assertTrue;
 public abstract class TestCoderBase {
   protected static Random RAND = new Random();
 
+  private Configuration conf;
   protected int numDataUnits;
   protected int numParityUnits;
   protected int chunkSize = 16 * 1024;
@@ -49,8 +51,9 @@ public abstract class TestCoderBase {
    * @param numParityUnits
    * @param erasedIndexes
    */
-  protected void prepare(int numDataUnits, int numParityUnits,
-                         int[] erasedIndexes) {
+  protected void prepare(Configuration conf, int numDataUnits,
+                         int numParityUnits, int[] erasedIndexes) {
+    this.conf = conf;
     this.numDataUnits = numDataUnits;
     this.numParityUnits = numParityUnits;
     this.erasedDataIndexes = erasedIndexes != null ?
@@ -58,6 +61,14 @@ public abstract class TestCoderBase {
   }
 
   /**
+   * Get the conf the test.
+   * @return configuration
+   */
+  protected Configuration getConf() {
+    return this.conf;
+  }
+
+  /**
    * Compare and verify if erased chunks are equal to recovered chunks
    * @param erasedChunks
    * @param recoveredChunks

http://git-wip-us.apache.org/repos/asf/hadoop/blob/068a5d5b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
index d911db9..b963a59 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -17,10 +17,9 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
-import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.erasurecode.ECBlock;
-import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
+import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.TestCoderBase;
 
 /**
@@ -30,7 +29,6 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected Class<? extends ErasureEncoder> encoderClass;
   protected Class<? extends ErasureDecoder> decoderClass;
 
-  private Configuration conf;
   protected int numChunksInBlock = 16;
 
   /**
@@ -48,19 +46,6 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   }
 
   /**
-   * Prepare before running the case.
-   * @param conf
-   * @param numDataUnits
-   * @param numParityUnits
-   * @param erasedIndexes
-   */
-  protected void prepare(Configuration conf, int numDataUnits,
-                         int numParityUnits, int[] erasedIndexes) {
-    this.conf = conf;
-    super.prepare(numDataUnits, numParityUnits, erasedIndexes);
-  }
-
-  /**
    * Generating source data, encoding, recovering and then verifying.
    * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
    * it supports two kinds of ByteBuffers, one is array backed, the other is
@@ -162,7 +147,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    ((AbstractErasureCoder)encoder).setConf(conf);
+    encoder.setConf(getConf());
     return encoder;
   }
 
@@ -179,7 +164,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
-    ((AbstractErasureCoder)decoder).setConf(conf);
+    decoder.setConf(getConf());
     return decoder;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/068a5d5b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
index e54f647..39e5deb 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestJRSRawCoder.java
@@ -46,37 +46,37 @@ public class TestJRSRawCoder extends TestRawCoderBase {
 
   @Test
   public void testCodingNoDirectBuffer_10x4() {
-    prepare(10, 4, null);
+    prepare(null, 10, 4, null);
     testCoding(false);
   }
 
   @Test
   public void testCodingDirectBuffer_10x4() {
-    prepare(10, 4, null);
+    prepare(null, 10, 4, null);
     testCoding(true);
   }
 
   @Test
   public void testCodingDirectBuffer_10x4_erasure_of_2_4() {
-    prepare(10, 4, new int[] {2, 4});
+    prepare(null, 10, 4, new int[] {2, 4});
     testCoding(true);
   }
 
   @Test
   public void testCodingDirectBuffer_10x4_erasing_all() {
-    prepare(10, 4, new int[] {0, 1, 2, 3});
+    prepare(null, 10, 4, new int[] {0, 1, 2, 3});
     testCoding(true);
   }
 
   @Test
   public void testCodingNoDirectBuffer_3x3() {
-    prepare(3, 3, null);
+    prepare(null, 3, 3, null);
     testCoding(false);
   }
 
   @Test
   public void testCodingDirectBuffer_3x3() {
-    prepare(3, 3, null);
+    prepare(null, 3, 3, null);
     testCoding(true);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/068a5d5b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
index 890f632..b036eed 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -86,6 +86,7 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    encoder.setConf(getConf());
     return encoder;
   }
 
@@ -102,6 +103,7 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    decoder.setConf(getConf());
     return decoder;
   }
 


[24/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for HDFS-7742)

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts in the branch when merging (this commit is for HDFS-7742)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/f6975aeb
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/f6975aeb
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/f6975aeb

Branch: refs/heads/HDFS-7285
Commit: f6975aeb0f4d02a5a6e621c548aec0d6a1e3d9be
Parents: dcb987a
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Mar 30 10:23:09 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:53 2015 -0700

----------------------------------------------------------------------
 .../hdfs/server/blockmanagement/TestBlockManager.java   | 12 ++++++------
 1 file changed, 6 insertions(+), 6 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/f6975aeb/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index cbea3d8..43f4607 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -552,11 +552,11 @@ public class TestBlockManager {
     assertNotNull("Chooses decommissioning source node for a normal replication"
         + " if all available source nodes have reached their replication"
         + " limits below the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
-            new NumberReplicas(),
+            new NumberReplicas(), new LinkedList<Short>(), 1,
             UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
 
 
@@ -566,11 +566,11 @@ public class TestBlockManager {
 
     assertNull("Does not choose a source decommissioning node for a normal"
         + " replication when all available nodes exceed the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
-            new NumberReplicas(),
+            new NumberReplicas(), new LinkedList<Short>(), 1,
             UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED));
   }
 


[04/50] [abbrv] hadoop git commit: HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode. Contributed by Jing Zhao.

Posted by zh...@apache.org.
HDFS-7837. Erasure Coding: allocate and persist striped blocks in NameNode. Contributed by Jing Zhao.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/cae78517
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/cae78517
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/cae78517

Branch: refs/heads/HDFS-7285
Commit: cae78517ef9fc7ed53e68c5b7def879acfd8e534
Parents: 5608b43
Author: Jing Zhao <ji...@apache.org>
Authored: Mon Mar 2 13:44:33 2015 -0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:36:00 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockIdManager.java  |  31 +++-
 .../hdfs/server/blockmanagement/BlockInfo.java  |   4 +-
 .../blockmanagement/BlockInfoContiguous.java    |   5 +
 .../blockmanagement/BlockInfoStriped.java       |   8 +-
 .../server/blockmanagement/BlockManager.java    |  44 ++++--
 .../hdfs/server/blockmanagement/BlocksMap.java  |  20 ++-
 .../blockmanagement/DecommissionManager.java    |   9 +-
 .../hdfs/server/namenode/FSDirectory.java       |  27 +++-
 .../hdfs/server/namenode/FSEditLogLoader.java   |  69 ++++++---
 .../hdfs/server/namenode/FSImageFormat.java     |  12 +-
 .../server/namenode/FSImageFormatPBINode.java   |   5 +-
 .../server/namenode/FSImageFormatProtobuf.java  |   9 +-
 .../hdfs/server/namenode/FSNamesystem.java      |  28 ++--
 .../hadoop/hdfs/server/namenode/INodeFile.java  |  25 +++-
 .../server/namenode/NameNodeLayoutVersion.java  |   3 +-
 .../hadoop-hdfs/src/main/proto/fsimage.proto    |   1 +
 .../hdfs/server/namenode/TestAddBlockgroup.java |  85 -----------
 .../server/namenode/TestAddStripedBlocks.java   | 146 +++++++++++++++++++
 18 files changed, 351 insertions(+), 180 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
index 3ae54ce..1d69d74 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockIdManager.java
@@ -103,21 +103,38 @@ public class BlockIdManager {
   }
 
   /**
-   * Sets the maximum allocated block ID for this filesystem. This is
+   * Sets the maximum allocated contiguous block ID for this filesystem. This is
    * the basis for allocating new block IDs.
    */
-  public void setLastAllocatedBlockId(long blockId) {
+  public void setLastAllocatedContiguousBlockId(long blockId) {
     blockIdGenerator.skipTo(blockId);
   }
 
   /**
-   * Gets the maximum sequentially allocated block ID for this filesystem
+   * Gets the maximum sequentially allocated contiguous block ID for this
+   * filesystem
    */
-  public long getLastAllocatedBlockId() {
+  public long getLastAllocatedContiguousBlockId() {
     return blockIdGenerator.getCurrentValue();
   }
 
   /**
+   * Sets the maximum allocated striped block ID for this filesystem. This is
+   * the basis for allocating new block IDs.
+   */
+  public void setLastAllocatedStripedBlockId(long blockId) {
+    blockGroupIdGenerator.skipTo(blockId);
+  }
+
+  /**
+   * Gets the maximum sequentially allocated striped block ID for this
+   * filesystem
+   */
+  public long getLastAllocatedStripedBlockId() {
+    return blockGroupIdGenerator.getCurrentValue();
+  }
+
+  /**
    * Sets the current generation stamp for legacy blocks
    */
   public void setGenerationStampV1(long stamp) {
@@ -188,11 +205,11 @@ public class BlockIdManager {
   /**
    * Increments, logs and then returns the block ID
    */
-  public long nextBlockId() {
+  public long nextContiguousBlockId() {
     return blockIdGenerator.nextValue();
   }
 
-  public long nextBlockGroupId() {
+  public long nextStripedBlockId() {
     return blockGroupIdGenerator.nextValue();
   }
 
@@ -216,7 +233,7 @@ public class BlockIdManager {
     return id < 0;
   }
 
-  public static long convertToGroupID(long id) {
+  public static long convertToStripedID(long id) {
     return id & (~HdfsConstants.BLOCK_GROUP_INDEX_MASK);
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
index d15cbec..1d8afbd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfo.java
@@ -169,6 +169,8 @@ public abstract class BlockInfo extends Block
    */
   abstract void replaceBlock(BlockInfo newBlock);
 
+  public abstract boolean isStriped();
+
   /**
    * Find specified DatanodeDescriptor.
    * @return index or -1 if not found.
@@ -332,7 +334,7 @@ public abstract class BlockInfo extends Block
   }
 
   static BlockInfo copyOf(BlockInfo b) {
-    if (b instanceof BlockInfoContiguous) {
+    if (!b.isStriped()) {
       return new BlockInfoContiguous((BlockInfoContiguous) b);
     } else {
       return new BlockInfoStriped((BlockInfoStriped) b);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
index e54cba3..b5d3d59 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguous.java
@@ -144,4 +144,9 @@ public class BlockInfoContiguous extends BlockInfo {
     ucBlock.setBlockCollection(getBlockCollection());
     return ucBlock;
   }
+
+  @Override
+  public final boolean isStriped() {
+    return false;
+  }
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
index 57de772..8b458df 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java
@@ -18,7 +18,6 @@
 package org.apache.hadoop.hdfs.server.blockmanagement;
 
 import org.apache.hadoop.hdfs.protocol.Block;
-import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
 import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState;
 
 /**
@@ -57,7 +56,7 @@ public class BlockInfoStriped extends BlockInfo {
     this.setBlockCollection(b.getBlockCollection());
   }
 
-  private short getTotalBlockNum() {
+  short getTotalBlockNum() {
     return (short) (dataBlockNum + parityBlockNum);
   }
 
@@ -175,6 +174,11 @@ public class BlockInfoStriped extends BlockInfo {
   }
 
   @Override
+  public final boolean isStriped() {
+    return true;
+  }
+
+  @Override
   public int numNodes() {
     assert this.triplets != null : "BlockInfo is not initialized";
     assert triplets.length % 3 == 0 : "Malformed BlockInfo";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 2e05854..aa27f3b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -580,11 +580,22 @@ public class BlockManager {
     return maxReplicationStreams;
   }
 
-  /**
-   * @return true if the block has minimum replicas
-   */
-  public boolean checkMinReplication(Block block) {
-    return (countNodes(block).liveReplicas() >= minReplication);
+  public int getDefaultStorageNum(BlockInfo block) {
+    return block.isStriped() ?
+        ((BlockInfoStriped) block).getTotalBlockNum() : defaultReplication;
+  }
+
+  public short getMinStorageNum(BlockInfo block) {
+    return block.isStriped() ?
+        ((BlockInfoStriped) block).getDataBlockNum() : minReplication;
+  }
+
+  public boolean checkMinStorage(BlockInfo block) {
+    return countNodes(block).liveReplicas() >= getMinStorageNum(block);
+  }
+
+  public boolean checkMinStorage(BlockInfo block, int liveNum) {
+    return liveNum >= getMinStorageNum(block);
   }
 
   /**
@@ -628,7 +639,7 @@ public class BlockManager {
       return false; // already completed (e.g. by syncBlock)
     
     final boolean b = commitBlock(lastBlock, commitBlock);
-    if (countNodes(lastBlock).liveReplicas() >= minReplication) {
+    if (checkMinStorage(lastBlock)) {
       completeBlock(bc, bc.numBlocks() - 1, false);
     }
     return b;
@@ -652,7 +663,7 @@ public class BlockManager {
     }
 
     int numNodes = curBlock.numNodes();
-    if (!force && numNodes < minReplication) {
+    if (!force && !checkMinStorage(curBlock, numNodes)) {
       throw new IOException("Cannot complete block: " +
           "block does not satisfy minimal replication requirement.");
     }
@@ -696,9 +707,8 @@ public class BlockManager {
    * when tailing edit logs as a Standby.
    */
   public BlockInfo forceCompleteBlock(final BlockCollection bc,
-      final BlockInfoContiguousUnderConstruction block) throws IOException {
-    // TODO: support BlockInfoStripedUC for editlog
-    block.commitBlock(block);
+      final BlockInfo block) throws IOException {
+    BlockInfo.commitBlock(block, block);
     return completeBlock(bc, block, true);
   }
 
@@ -749,7 +759,7 @@ public class BlockManager {
     // count in safe-mode.
     namesystem.adjustSafeModeBlockTotals(
         // decrement safe if we had enough
-        targets.length >= minReplication ? -1 : 0,
+        checkMinStorage(oldBlock, targets.length) ? -1 : 0,
         // always decrement total blocks
         -1);
 
@@ -1192,8 +1202,8 @@ public class BlockManager {
     NumberReplicas numberOfReplicas = countNodes(b.stored);
     boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= bc
         .getBlockReplication();
-    boolean minReplicationSatisfied =
-        numberOfReplicas.liveReplicas() >= minReplication;
+    boolean minReplicationSatisfied = checkMinStorage(b.stored,
+        numberOfReplicas.liveReplicas());
     boolean hasMoreCorruptReplicas = minReplicationSatisfied &&
         (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) >
         bc.getBlockReplication();
@@ -2501,7 +2511,7 @@ public class BlockManager {
     // Now check for completion of blocks and safe block count
     int numCurrentReplica = countLiveNodes(storedBlock);
     if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED
-        && numCurrentReplica >= minReplication) {
+        && checkMinStorage(storedBlock, numCurrentReplica)) {
       completeBlock(storedBlock.getBlockCollection(), storedBlock, false);
     } else if (storedBlock.isComplete()) {
       // check whether safe replication is reached for the block
@@ -2576,7 +2586,7 @@ public class BlockManager {
       + pendingReplications.getNumReplicas(storedBlock);
 
     if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED &&
-        numLiveReplicas >= minReplication) {
+        checkMinStorage(storedBlock, numLiveReplicas)) {
       storedBlock = completeBlock(bc, storedBlock, false);
     } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) {
       // check whether safe replication is reached for the block
@@ -3247,6 +3257,8 @@ public class BlockManager {
   /**
    * Return the number of nodes hosting a given block, grouped
    * by the state of those replicas.
+   * For a striped block, this includes nodes storing blocks belonging to the
+   * striped block group.
    */
   public NumberReplicas countNodes(Block b) {
     int decommissioned = 0;
@@ -3402,7 +3414,7 @@ public class BlockManager {
     BlockInfo info = null;
     if (BlockIdManager.isStripedBlockID(block.getBlockId())) {
       info = blocksMap.getStoredBlock(
-          new Block(BlockIdManager.convertToGroupID(block.getBlockId())));
+          new Block(BlockIdManager.convertToStripedID(block.getBlockId())));
     }
     if (info == null) {
       info = blocksMap.getStoredBlock(block);

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
index d383de8..0cf5fe6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlocksMap.java
@@ -43,8 +43,15 @@ class BlocksMap {
 
     @Override
     public boolean hasNext() {
-      return blockInfo != null && nextIdx < blockInfo.getCapacity()
-              && blockInfo.getDatanode(nextIdx) != null;
+      if (blockInfo == null) {
+        return false;
+      }
+      while (nextIdx < blockInfo.getCapacity() &&
+          blockInfo.getDatanode(nextIdx) == null) {
+        // note that for striped blocks there may be null in the triplets
+        nextIdx++;
+      }
+      return nextIdx < blockInfo.getCapacity();
     }
 
     @Override
@@ -123,10 +130,13 @@ class BlocksMap {
       return;
 
     blockInfo.setBlockCollection(null);
-    // TODO: fix this logic for block group
-    for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) {
+    final int size = blockInfo instanceof BlockInfoContiguous ?
+        blockInfo.numNodes() : blockInfo.getCapacity();
+    for(int idx = size - 1; idx >= 0; idx--) {
       DatanodeDescriptor dn = blockInfo.getDatanode(idx);
-      dn.removeBlock(blockInfo); // remove from the list and wipe the location
+      if (dn != null) {
+        dn.removeBlock(blockInfo); // remove from the list and wipe the location
+      }
     }
   }
   

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
index 5c9aec7..3ea5311 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DecommissionManager.java
@@ -544,7 +544,7 @@ public class DecommissionManager {
       int underReplicatedInOpenFiles = 0;
       while (it.hasNext()) {
         numBlocksChecked++;
-        final BlockInfoContiguous block = it.next();
+        final BlockInfo block = it.next();
         // Remove the block from the list if it's no longer in the block map,
         // e.g. the containing file has been deleted
         if (blockManager.blocksMap.getStoredBlock(block) == null) {
@@ -578,8 +578,9 @@ public class DecommissionManager {
         }
 
         // Even if the block is under-replicated, 
-        // it doesn't block decommission if it's sufficiently replicated 
-        if (isSufficientlyReplicated(block, bc, num)) {
+        // it doesn't block decommission if it's sufficiently replicated
+        BlockInfoContiguous blk = (BlockInfoContiguous) block;
+        if (isSufficientlyReplicated(blk, bc, num)) {
           if (pruneSufficientlyReplicated) {
             it.remove();
           }
@@ -588,7 +589,7 @@ public class DecommissionManager {
 
         // We've found an insufficiently replicated block.
         if (insufficientlyReplicated != null) {
-          insufficientlyReplicated.add(block);
+          insufficientlyReplicated.add(blk);
         }
         // Log if this is our first time through
         if (firstReplicationLog) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
index 0aad56d..fa3f6a6 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirectory.java
@@ -57,6 +57,7 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelper;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
@@ -425,6 +426,10 @@ public class FSDirectory implements Closeable {
     writeLock();
     try {
       newiip = addINode(existing, newNode);
+      // TODO: we will no longer use storage policy for "Erasure Coding Zone"
+      if (newiip != null && newNode.isStriped()) {
+        newNode.addStripedBlocksFeature();
+      }
     } finally {
       writeUnlock();
     }
@@ -460,6 +465,10 @@ public class FSDirectory implements Closeable {
     try {
       INodesInPath iip = addINode(existing, newNode);
       if (iip != null) {
+        // TODO: we will no longer use storage policy for "Erasure Coding Zone"
+        if (newNode.isStriped()) {
+          newNode.addStripedBlocksFeature();
+        }
         if (aclEntries != null) {
           AclStorage.updateINodeAcl(newNode, aclEntries, CURRENT_STATE_ID);
         }
@@ -481,7 +490,7 @@ public class FSDirectory implements Closeable {
   /**
    * Add a block to the file. Returns a reference to the added block.
    */
-  BlockInfoContiguous addBlock(String path, INodesInPath inodesInPath,
+  BlockInfo addBlock(String path, INodesInPath inodesInPath,
       Block block, DatanodeStorageInfo[] targets, boolean isStriped)
       throws IOException {
     writeLock();
@@ -493,16 +502,20 @@ public class FSDirectory implements Closeable {
       Preconditions.checkState(fileINode.isUnderConstruction());
 
       // check quota limits and updated space consumed
+      // TODO add quota usage for EC files
       updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(),
           fileINode.getBlockReplication(), true);
 
       // associate new last block for the file
-      BlockInfoContiguousUnderConstruction blockInfo =
-        new BlockInfoContiguousUnderConstruction(
-            block,
-            numLocations,
-            BlockUCState.UNDER_CONSTRUCTION,
-            targets);
+      final BlockInfo blockInfo;
+      if (isStriped) {
+        blockInfo = new BlockInfoStripedUnderConstruction(block,
+            HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS,
+            BlockUCState.UNDER_CONSTRUCTION, targets);
+      } else {
+        blockInfo = new BlockInfoContiguousUnderConstruction(block,
+            numLocations, BlockUCState.UNDER_CONSTRUCTION, targets);
+      }
       getBlockManager().addBlockCollection(blockInfo, fileINode);
       fileINode.addBlock(blockInfo);
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
index 946a37c..89cfe05 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSEditLogLoader.java
@@ -35,7 +35,9 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.classification.InterfaceStability;
 import org.apache.hadoop.fs.XAttrSetFlag;
 import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo;
@@ -414,7 +416,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       newFile.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       newFile.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      updateBlocks(fsDir, addCloseOp, iip, newFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, addCloseOp, iip, newFile, newFile.isStriped());
       break;
     }
     case OP_CLOSE: {
@@ -434,7 +437,8 @@ public class FSEditLogLoader {
       // Update the salient file attributes.
       file.setAccessTime(addCloseOp.atime, Snapshot.CURRENT_STATE_ID);
       file.setModificationTime(addCloseOp.mtime, Snapshot.CURRENT_STATE_ID);
-      updateBlocks(fsDir, addCloseOp, iip, file);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, addCloseOp, iip, file, file.isStriped());
 
       // Now close the file
       if (!file.isUnderConstruction() &&
@@ -492,7 +496,8 @@ public class FSEditLogLoader {
       INodesInPath iip = fsDir.getINodesInPath(path, true);
       INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path);
       // Update in-memory data structures
-      updateBlocks(fsDir, updateOp, iip, oldFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      updateBlocks(fsDir, updateOp, iip, oldFile, oldFile.isStriped());
       
       if (toAddRetryCache) {
         fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId);
@@ -508,7 +513,8 @@ public class FSEditLogLoader {
       }
       INodeFile oldFile = INodeFile.valueOf(fsDir.getINode(path), path);
       // add the new block to the INodeFile
-      addNewBlock(addBlockOp, oldFile);
+      // TODO whether the file is striped should later be retrieved from iip
+      addNewBlock(addBlockOp, oldFile, oldFile.isStriped());
       break;
     }
     case OP_SET_REPLICATION: {
@@ -784,8 +790,15 @@ public class FSEditLogLoader {
     }
     case OP_ALLOCATE_BLOCK_ID: {
       AllocateBlockIdOp allocateBlockIdOp = (AllocateBlockIdOp) op;
-      fsNamesys.getBlockIdManager().setLastAllocatedBlockId(
-          allocateBlockIdOp.blockId);
+      if (BlockIdManager.isStripedBlockID(allocateBlockIdOp.blockId)) {
+        // ALLOCATE_BLOCK_ID is added for sequential block id, thus if the id
+        // is negative, it must belong to striped blocks
+        fsNamesys.getBlockIdManager().setLastAllocatedStripedBlockId(
+            allocateBlockIdOp.blockId);
+      } else {
+        fsNamesys.getBlockIdManager().setLastAllocatedContiguousBlockId(
+            allocateBlockIdOp.blockId);
+      }
       break;
     }
     case OP_ROLLING_UPGRADE_START: {
@@ -937,9 +950,9 @@ public class FSEditLogLoader {
 
   /**
    * Add a new block into the given INodeFile
-   * TODO support adding striped block
    */
-  private void addNewBlock(AddBlockOp op, INodeFile file) throws IOException {
+  private void addNewBlock(AddBlockOp op, INodeFile file, boolean isStriped)
+      throws IOException {
     BlockInfo[] oldBlocks = file.getBlocks();
     Block pBlock = op.getPenultimateBlock();
     Block newBlock= op.getLastBlock();
@@ -947,7 +960,7 @@ public class FSEditLogLoader {
     if (pBlock != null) { // the penultimate block is not null
       assert oldBlocks != null && oldBlocks.length > 0;
       // compare pBlock with the last block of oldBlocks
-      Block oldLastBlock = oldBlocks[oldBlocks.length - 1];
+      BlockInfo oldLastBlock = oldBlocks[oldBlocks.length - 1];
       if (oldLastBlock.getBlockId() != pBlock.getBlockId()
           || oldLastBlock.getGenerationStamp() != pBlock.getGenerationStamp()) {
         throw new IOException(
@@ -957,29 +970,33 @@ public class FSEditLogLoader {
       }
       
       oldLastBlock.setNumBytes(pBlock.getNumBytes());
-      if (oldLastBlock instanceof BlockInfoContiguousUnderConstruction) {
-        fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldLastBlock);
+      if (!oldLastBlock.isComplete()) {
+        fsNamesys.getBlockManager().forceCompleteBlock(file, oldLastBlock);
         fsNamesys.getBlockManager().processQueuedMessagesForBlock(pBlock);
       }
     } else { // the penultimate block is null
       Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0);
     }
     // add the new block
-    BlockInfoContiguous newBI = new BlockInfoContiguousUnderConstruction(
-          newBlock, file.getBlockReplication());
-    fsNamesys.getBlockManager().addBlockCollection(newBI, file);
-    file.addBlock(newBI);
+    final BlockInfo newBlockInfo;
+    if (isStriped) {
+      newBlockInfo = new BlockInfoStripedUnderConstruction(newBlock,
+          HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+    } else {
+      newBlockInfo = new BlockInfoContiguousUnderConstruction(newBlock,
+          file.getBlockReplication());
+    }
+    fsNamesys.getBlockManager().addBlockCollection(newBlockInfo, file);
+    file.addBlock(newBlockInfo);
     fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock);
   }
   
   /**
    * Update in-memory data structures with new block information.
-   * TODO support adding striped block
    * @throws IOException
    */
   private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op,
-      INodesInPath iip, INodeFile file) throws IOException {
+      INodesInPath iip, INodeFile file, boolean isStriped) throws IOException {
     // Update its block list
     BlockInfo[] oldBlocks = file.getBlocks();
     Block[] newBlocks = op.getBlocks();
@@ -1008,11 +1025,10 @@ public class FSEditLogLoader {
         oldBlock.getGenerationStamp() != newBlock.getGenerationStamp();
       oldBlock.setGenerationStamp(newBlock.getGenerationStamp());
       
-      if (oldBlock instanceof BlockInfoContiguousUnderConstruction &&
+      if (!oldBlock.isComplete() &&
           (!isLastBlock || op.shouldCompleteLastBlock())) {
         changeMade = true;
-        fsNamesys.getBlockManager().forceCompleteBlock(file,
-            (BlockInfoContiguousUnderConstruction) oldBlock);
+        fsNamesys.getBlockManager().forceCompleteBlock(file, oldBlock);
       }
       if (changeMade) {
         // The state or gen-stamp of the block has changed. So, we may be
@@ -1041,13 +1057,18 @@ public class FSEditLogLoader {
       // We're adding blocks
       for (int i = oldBlocks.length; i < newBlocks.length; i++) {
         Block newBlock = newBlocks[i];
-        BlockInfoContiguous newBI;
+        final BlockInfo newBI;
         if (!op.shouldCompleteLastBlock()) {
           // TODO: shouldn't this only be true for the last block?
           // what about an old-version fsync() where fsync isn't called
           // until several blocks in?
-          newBI = new BlockInfoContiguousUnderConstruction(
-              newBlock, file.getBlockReplication());
+          if (isStriped) {
+            newBI = new BlockInfoStripedUnderConstruction(newBlock,
+                HdfsConstants.NUM_DATA_BLOCKS, HdfsConstants.NUM_PARITY_BLOCKS);
+          } else {
+            newBI = new BlockInfoContiguousUnderConstruction(newBlock,
+                file.getBlockReplication());
+          }
         } else {
           // OP_CLOSE should add finalized blocks. This code path
           // is only executed when loading edits written by prior

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
index d62b804..2e6e741 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormat.java
@@ -357,7 +357,14 @@ public class FSImageFormat {
 
           // read the max sequential block ID.
           long maxSequentialBlockId = in.readLong();
-          namesystem.getBlockIdManager().setLastAllocatedBlockId(maxSequentialBlockId);
+          namesystem.getBlockIdManager().setLastAllocatedContiguousBlockId(
+              maxSequentialBlockId);
+          if (NameNodeLayoutVersion.supports(
+              NameNodeLayoutVersion.Feature.ERASURE_CODING, imgVersion)) {
+            final long maxStripedBlockId = in.readLong();
+            namesystem.getBlockIdManager().setLastAllocatedStripedBlockId(
+                maxStripedBlockId);
+          }
         } else {
 
           long startingGenStamp = namesystem.getBlockIdManager()
@@ -1264,7 +1271,8 @@ public class FSImageFormat {
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV1());
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampV2());
         out.writeLong(sourceNamesystem.getBlockIdManager().getGenerationStampAtblockIdSwitch());
-        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedBlockId());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedContiguousBlockId());
+        out.writeLong(sourceNamesystem.getBlockIdManager().getLastAllocatedStripedBlockId());
         out.writeLong(context.getTxId());
         out.writeLong(sourceNamesystem.dir.getLastInodeId());
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
index a025bb0..5627788 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatPBINode.java
@@ -630,8 +630,9 @@ public final class FSImageFormatPBINode {
       INodeSection.INodeFile.Builder b = buildINodeFile(n,
           parent.getSaverContext());
 
-      if (n.getBlocks() != null) {
-        for (Block block : n.getBlocks()) {
+      BlockInfoContiguous[] cBlks = n.getContiguousBlocks();
+      if (cBlks != null) {
+        for (Block block : cBlks) {
           b.addBlocks(PBHelper.convert(block));
         }
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
index 3ee848a..62dc068 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSImageFormatProtobuf.java
@@ -296,7 +296,11 @@ public final class FSImageFormatProtobuf {
       blockIdManager.setGenerationStampV1(s.getGenstampV1());
       blockIdManager.setGenerationStampV2(s.getGenstampV2());
       blockIdManager.setGenerationStampV1Limit(s.getGenstampV1Limit());
-      blockIdManager.setLastAllocatedBlockId(s.getLastAllocatedBlockId());
+      blockIdManager.setLastAllocatedContiguousBlockId(s.getLastAllocatedBlockId());
+      if (s.hasLastAllocatedStripedBlockId()) {
+        blockIdManager.setLastAllocatedStripedBlockId(
+            s.getLastAllocatedStripedBlockId());
+      }
       imgTxId = s.getTransactionId();
       if (s.hasRollingUpgradeStartTime()
           && fsn.getFSImage().hasRollbackFSImage()) {
@@ -536,7 +540,8 @@ public final class FSImageFormatProtobuf {
           .setGenstampV1(blockIdManager.getGenerationStampV1())
           .setGenstampV1Limit(blockIdManager.getGenerationStampV1Limit())
           .setGenstampV2(blockIdManager.getGenerationStampV2())
-          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedBlockId())
+          .setLastAllocatedBlockId(blockIdManager.getLastAllocatedContiguousBlockId())
+          .setLastAllocatedStripedBlockId(blockIdManager.getLastAllocatedStripedBlockId())
           .setTransactionId(context.getTxId());
 
       // We use the non-locked version of getNamespaceInfo here since

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index e5a2749..47ae7de 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -206,7 +206,6 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -2070,7 +2069,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     boolean shouldRecoverNow = (newBlock == null);
 
     BlockInfo oldBlock = file.getLastBlock();
-    assert oldBlock instanceof BlockInfoContiguous;
+    assert !oldBlock.isStriped();
 
     boolean shouldCopyOnTruncate = shouldCopyOnTruncate(file,
         (BlockInfoContiguous) oldBlock);
@@ -3112,7 +3111,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // choose targets for the new block to be allocated.
-    return getBlockManager().chooseTarget4NewBlock( 
+    return getBlockManager().chooseTarget4NewBlock(
         src, numTargets, clientNode, excludedNodes, blockSize, favoredNodes,
         storagePolicyID);
   }
@@ -3160,8 +3159,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
       // allocate new block, record block locations in INode.
       newBlock = createNewBlock(isStriped);
-      INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile);
-      saveAllocatedBlock(src, inodesInPath, newBlock, targets, isStriped);
+      saveAllocatedBlock(src, fileState.iip, newBlock, targets, isStriped);
 
       persistNewBlock(src, pendingFile);
       offset = pendingFile.computeFileSize();
@@ -3583,8 +3581,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       Block newBlock, DatanodeStorageInfo[] targets, boolean isStriped)
       throws IOException {
     assert hasWriteLock();
-    BlockInfoContiguous b = dir.addBlock(src, inodesInPath, newBlock, targets,
-        isStriped);
+    BlockInfo b = dir.addBlock(src, inodesInPath, newBlock, targets, isStriped);
     NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src);
     DatanodeStorageInfo.incrementBlocksScheduled(targets);
   }
@@ -3791,7 +3788,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         BlockInfo bi = getStoredBlock(b);
         if (bi.isComplete()) {
           numRemovedComplete++;
-          if (bi.numNodes() >= blockManager.minReplication) {
+          if (blockManager.checkMinStorage(bi, bi.numNodes())) {
             numRemovedSafe++;
           }
         }
@@ -4020,7 +4017,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       curBlock = blocks[nrCompleteBlocks];
       if(!curBlock.isComplete())
         break;
-      assert blockManager.checkMinReplication(curBlock) :
+      assert blockManager.checkMinStorage(curBlock) :
               "A COMPLETE block is not minimally replicated in " + src;
     }
 
@@ -4055,8 +4052,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     BlockInfo penultimateBlock = pendingFile.getPenultimateBlock();
 
     // If penultimate block doesn't exist then its minReplication is met
-    boolean penultimateBlockMinReplication = penultimateBlock == null ||
-        blockManager.checkMinReplication(penultimateBlock);
+    boolean penultimateBlockMinStorage = penultimateBlock == null ||
+        blockManager.checkMinStorage(penultimateBlock);
 
     switch(lastBlockState) {
     case COMPLETE:
@@ -4064,8 +4061,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       break;
     case COMMITTED:
       // Close file if committed blocks are minimally replicated
-      if(penultimateBlockMinReplication &&
-          blockManager.checkMinReplication(lastBlock)) {
+      if(penultimateBlockMinStorage &&
+          blockManager.checkMinStorage(lastBlock)) {
         finalizeINodeFileUnderConstruction(src, pendingFile,
             iip.getLatestSnapshotId());
         NameNode.stateChangeLog.warn("BLOCK*"
@@ -4165,6 +4162,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     }
 
     // Adjust disk space consumption if required
+    // TODO: support EC files
     final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes();    
     if (diff > 0) {
       try {
@@ -5053,8 +5051,6 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
 
   /**
    * Persist the new block (the last block of the given file).
-   * @param path
-   * @param file
    */
   private void persistNewBlock(String path, INodeFile file) {
     Preconditions.checkArgument(file.isUnderConstruction());
@@ -6170,7 +6166,7 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
     assert hasWriteLock();
     checkNameNodeSafeMode("Cannot get next block ID");
     final long blockId = isStriped ?
-        blockIdManager.nextBlockGroupId() : blockIdManager.nextBlockId();
+        blockIdManager.nextStripedBlockId() : blockIdManager.nextContiguousBlockId();
     getEditLog().logAllocateBlockId(blockId);
     // NB: callers sync the log
     return blockId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index 5462da7..093ba25 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -86,7 +86,7 @@ public class INodeFile extends INodeWithAdditionalFields
    */
   static enum HeaderFormat {
     PREFERRED_BLOCK_SIZE(null, 48, 1),
-    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1),
+    REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 0),
     STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH,
         0);
 
@@ -261,10 +261,10 @@ public class INodeFile extends INodeWithAdditionalFields
   public void setBlock(int index, BlockInfo blk) {
     FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
     if (sb == null) {
-      assert blk instanceof BlockInfoContiguous;
+      assert !blk.isStriped();
       this.blocks[index] = (BlockInfoContiguous) blk;
     } else {
-      assert blk instanceof BlockInfoStriped;
+      assert blk.isStriped();
       assert hasNoContiguousBlock();
       sb.setBlock(index, (BlockInfoStriped) blk);
     }
@@ -282,12 +282,12 @@ public class INodeFile extends INodeWithAdditionalFields
     final BlockInfo ucBlock;
     FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
     if (sb == null) {
-      assert lastBlock instanceof BlockInfoContiguous;
+      assert !lastBlock.isStriped();
       ucBlock = ((BlockInfoContiguous) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     } else {
       assert hasNoContiguousBlock();
-      assert lastBlock instanceof BlockInfoStriped;
+      assert lastBlock.isStriped();
       ucBlock = ((BlockInfoStriped) lastBlock)
           .convertToBlockUnderConstruction(UNDER_CONSTRUCTION, locations);
     }
@@ -546,7 +546,7 @@ public class INodeFile extends INodeWithAdditionalFields
   /**
    * add a contiguous block to the block list
    */
-  void addBlock(BlockInfoContiguous newblock) {
+  private void addContiguousBlock(BlockInfoContiguous newblock) {
     if (this.blocks == null) {
       this.setContiguousBlocks(new BlockInfoContiguous[]{newblock});
     } else {
@@ -558,6 +558,19 @@ public class INodeFile extends INodeWithAdditionalFields
     }
   }
 
+  /** add a striped or contiguous block */
+  void addBlock(BlockInfo newblock) {
+    FileWithStripedBlocksFeature sb = getStripedBlocksFeature();
+    if (sb == null) {
+      assert !newblock.isStriped();
+      addContiguousBlock((BlockInfoContiguous) newblock);
+    } else {
+      assert newblock.isStriped();
+      assert hasNoContiguousBlock();
+      sb.addBlock((BlockInfoStriped) newblock);
+    }
+  }
+
   /** Set the blocks. */
   public void setContiguousBlocks(BlockInfoContiguous[] blocks) {
     this.blocks = blocks;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
index d235e2b..f93218f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/NameNodeLayoutVersion.java
@@ -72,7 +72,8 @@ public class NameNodeLayoutVersion {
     BLOCK_STORAGE_POLICY(-60, "Block Storage policy"),
     TRUNCATE(-61, "Truncate"),
     APPEND_NEW_BLOCK(-62, "Support appending to new block"),
-    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types");
+    QUOTA_BY_STORAGE_TYPE(-63, "Support quota for specific storage types"),
+    ERASURE_CODING(-64, "Support erasure coding");
 
     private final FeatureInfo info;
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
index b6fd033..3f3a71e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/fsimage.proto
@@ -73,6 +73,7 @@ message NameSystemSection {
   optional uint64 lastAllocatedBlockId = 5;
   optional uint64 transactionId = 6;
   optional uint64 rollingUpgradeStartTime = 7;
+  optional uint64 lastAllocatedStripedBlockId = 8;
 }
 
 /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
deleted file mode 100644
index a2ef7b2..0000000
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddBlockgroup.java
+++ /dev/null
@@ -1,85 +0,0 @@
-/**
- * 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.server.namenode;
-
-import org.apache.commons.logging.Log;
-import org.apache.commons.logging.LogFactory;
-import org.apache.hadoop.conf.Configuration;
-import org.apache.hadoop.fs.Path;
-import org.apache.hadoop.hdfs.DFSConfigKeys;
-import org.apache.hadoop.hdfs.DFSTestUtil;
-import org.apache.hadoop.hdfs.DistributedFileSystem;
-import org.apache.hadoop.hdfs.MiniDFSCluster;
-import org.apache.hadoop.hdfs.protocol.HdfsConstants;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
-import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
-import org.junit.After;
-import org.junit.Before;
-import org.junit.Test;
-
-import java.io.IOException;
-
-import static org.junit.Assert.assertEquals;
-
-public class TestAddBlockgroup {
-
-  public static final Log LOG = LogFactory.getLog(TestAddBlockgroup.class);
-
-  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
-      HdfsConstants.NUM_PARITY_BLOCKS;
-  private final short NUM_DATANODES = GROUP_SIZE;
-
-  private static final int BLOCKSIZE = 1024;
-  private static final short REPLICATION = 3;
-
-  private MiniDFSCluster cluster;
-  private Configuration conf;
-
-  @Before
-  public void setup() throws IOException {
-    conf = new Configuration();
-    conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCKSIZE);
-    cluster = new MiniDFSCluster.Builder(conf).numDataNodes(NUM_DATANODES)
-        .build();
-    cluster.waitActive();
-    cluster.getFileSystem().setStoragePolicy(new Path("/"),
-        HdfsConstants.EC_STORAGE_POLICY_NAME);
-  }
-
-  @After
-  public void tearDown() {
-    if (cluster != null) {
-      cluster.shutdown();
-    }
-  }
-
-  @Test
-  public void testAddBlockGroup() throws Exception {
-    DistributedFileSystem fs = cluster.getFileSystem();
-    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
-
-    final Path file1 = new Path("/file1");
-    DFSTestUtil.createFile(fs, file1, BLOCKSIZE * 2, REPLICATION, 0L);
-    INodeFile file1Node = fsdir.getINode4Write(file1.toString()).asFile();
-    BlockInfo[] file1Blocks = file1Node.getBlocks();
-    assertEquals(2, file1Blocks.length);
-    assertEquals(GROUP_SIZE, file1Blocks[0].numNodes());
-    assertEquals(HdfsConstants.MAX_BLOCKS_IN_GROUP,
-        file1Blocks[1].getBlockId() - file1Blocks[0].getBlockId());
-  }
-}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/cae78517/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
new file mode 100644
index 0000000..7226f51
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -0,0 +1,146 @@
+/**
+ * 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.server.namenode;
+
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.DatanodeID;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.protocol.LocatedBlock;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo;
+import org.apache.hadoop.hdfs.server.common.HdfsServerConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.io.IOUtils;
+import org.junit.After;
+import org.junit.Assert;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestAddStripedBlocks {
+  private final short GROUP_SIZE = HdfsConstants.NUM_DATA_BLOCKS +
+      HdfsConstants.NUM_PARITY_BLOCKS;
+
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem dfs;
+
+  @Before
+  public void setup() throws IOException {
+    cluster = new MiniDFSCluster.Builder(new HdfsConfiguration())
+        .numDataNodes(GROUP_SIZE).build();
+    cluster.waitActive();
+    dfs = cluster.getFileSystem();
+    dfs.setStoragePolicy(new Path("/"), HdfsConstants.EC_STORAGE_POLICY_NAME);
+  }
+
+  @After
+  public void tearDown() {
+    if (cluster != null) {
+      cluster.shutdown();
+    }
+  }
+
+  @Test
+  public void testAddStripedBlock() throws Exception {
+    final Path file = new Path("/file1");
+    // create an empty file
+    FSDataOutputStream out = null;
+    try {
+      out = dfs.create(file, (short) 1);
+
+      FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+      INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+      LocatedBlock newBlock = cluster.getNamesystem().getAdditionalBlock(
+          file.toString(), fileNode.getId(), dfs.getClient().getClientName(),
+          null, null, null);
+      assertEquals(GROUP_SIZE, newBlock.getLocations().length);
+      assertEquals(GROUP_SIZE, newBlock.getStorageIDs().length);
+
+      BlockInfo[] blocks = fileNode.getBlocks();
+      assertEquals(1, blocks.length);
+      Assert.assertTrue(blocks[0].isStriped());
+
+      checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), true);
+    } finally {
+      IOUtils.cleanup(null, out);
+    }
+
+    // restart NameNode to check editlog
+    cluster.restartNameNode(true);
+    FSDirectory fsdir = cluster.getNamesystem().getFSDirectory();
+    INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    BlockInfo[] blocks = fileNode.getBlocks();
+    assertEquals(1, blocks.length);
+    Assert.assertTrue(blocks[0].isStriped());
+    checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+
+    // save namespace, restart namenode, and check
+    dfs = cluster.getFileSystem();
+    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_ENTER);
+    dfs.saveNamespace();
+    dfs.setSafeMode(HdfsConstants.SafeModeAction.SAFEMODE_LEAVE);
+    cluster.restartNameNode(true);
+    fsdir = cluster.getNamesystem().getFSDirectory();
+    fileNode = fsdir.getINode4Write(file.toString()).asFile();
+    blocks = fileNode.getBlocks();
+    assertEquals(1, blocks.length);
+    Assert.assertTrue(blocks[0].isStriped());
+    checkStripedBlockUC((BlockInfoStriped) fileNode.getLastBlock(), false);
+  }
+
+  private void checkStripedBlockUC(BlockInfoStriped block,
+      boolean checkReplica) {
+    assertEquals(0, block.numNodes());
+    Assert.assertFalse(block.isComplete());
+    Assert.assertEquals(HdfsConstants.NUM_DATA_BLOCKS, block.getDataBlockNum());
+    Assert.assertEquals(HdfsConstants.NUM_PARITY_BLOCKS,
+        block.getParityBlockNum());
+    Assert.assertEquals(0,
+        block.getBlockId() & HdfsConstants.BLOCK_GROUP_INDEX_MASK);
+
+    final BlockInfoStripedUnderConstruction blockUC =
+        (BlockInfoStripedUnderConstruction) block;
+    Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION,
+        blockUC.getBlockUCState());
+    if (checkReplica) {
+      Assert.assertEquals(GROUP_SIZE, blockUC.getNumExpectedLocations());
+      DatanodeStorageInfo[] storages = blockUC.getExpectedStorageLocations();
+      for (DataNode dn : cluster.getDataNodes()) {
+        Assert.assertTrue(includeDataNode(dn.getDatanodeId(), storages));
+      }
+    }
+  }
+
+  private boolean includeDataNode(DatanodeID dn, DatanodeStorageInfo[] storages) {
+    for (DatanodeStorageInfo storage : storages) {
+      if (storage.getDatanodeDescriptor().equals(dn)) {
+        return true;
+      }
+    }
+    return false;
+  }
+}


[21/50] [abbrv] hadoop git commit: HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435, HDFS-7930, HDFS-7960 (this commit is for HDFS-7960)

Posted by zh...@apache.org.
HDFS-7936. Erasure coding: resolving conflicts when merging with HDFS-7903, HDFS-7435, HDFS-7930, HDFS-7960 (this commit is for HDFS-7960)


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/bd9bf78f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/bd9bf78f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/bd9bf78f

Branch: refs/heads/HDFS-7285
Commit: bd9bf78fd7438427582567bc7b547db3f0c9c01a
Parents: 6a8afd3
Author: Zhe Zhang <zh...@apache.org>
Authored: Tue Mar 24 11:39:36 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:52 2015 -0700

----------------------------------------------------------------------
 .../apache/hadoop/hdfs/server/blockmanagement/BlockManager.java | 4 ++--
 .../blockmanagement/TestNameNodePrunesMissingStorages.java      | 5 ++++-
 .../hadoop/hdfs/server/namenode/TestAddStripedBlocks.java       | 2 +-
 3 files changed, 7 insertions(+), 4 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd9bf78f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 7eb1861..a3c5512 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -1978,10 +1978,10 @@ public class BlockManager {
              "longer exists on the DataNode.",
               Long.toHexString(context.getReportId()), zombie.getStorageID());
     assert(namesystem.hasWriteLock());
-    Iterator<BlockInfoContiguous> iter = zombie.getBlockIterator();
+    Iterator<BlockInfo> iter = zombie.getBlockIterator();
     int prevBlocks = zombie.numBlocks();
     while (iter.hasNext()) {
-      BlockInfoContiguous block = iter.next();
+      BlockInfo block = iter.next();
       // We assume that a block can be on only one storage in a DataNode.
       // That's why we pass in the DatanodeDescriptor rather than the
       // DatanodeStorageInfo.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd9bf78f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
index 4b97d01..e9329cd 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestNameNodePrunesMissingStorages.java
@@ -171,9 +171,12 @@ public class TestNameNodePrunesMissingStorages {
       String datanodeUuid;
       // Find the first storage which this block is in.
       try {
+        BlockInfo storedBlock =
+            cluster.getNamesystem().getBlockManager().
+                getStoredBlock(block.getLocalBlock());
         Iterator<DatanodeStorageInfo> storageInfoIter =
             cluster.getNamesystem().getBlockManager().
-                getStorages(block.getLocalBlock()).iterator();
+                blocksMap.getStorages(storedBlock).iterator();
         assertTrue(storageInfoIter.hasNext());
         DatanodeStorageInfo info = storageInfoIter.next();
         storageIdToRemove = info.getStorageID();

http://git-wip-us.apache.org/repos/asf/hadoop/blob/bd9bf78f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
index 05aec4b..7d7c81e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java
@@ -269,7 +269,7 @@ public class TestAddStripedBlocks {
       StorageBlockReport[] reports = {new StorageBlockReport(storage,
           bll)};
       cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId),
-          bpId, reports);
+          bpId, reports, null);
     }
 
     BlockInfoStripedUnderConstruction ucBlock =


[25/50] [abbrv] hadoop git commit: HADOOP-11664. Loading predefined EC schemas from configuration. Contributed by Kai Zheng.

Posted by zh...@apache.org.
HADOOP-11664. Loading predefined EC schemas from configuration. Contributed by Kai Zheng.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/dcb987a7
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/dcb987a7
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/dcb987a7

Branch: refs/heads/HDFS-7285
Commit: dcb987a77ad0a3821a33eb38fe2c08a5ab72006a
Parents: d77a788
Author: Zhe Zhang <zh...@apache.org>
Authored: Fri Mar 27 14:52:50 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:53 2015 -0700

----------------------------------------------------------------------
 .../src/main/conf/ecschema-def.xml              |  40 +++++
 .../hadoop/fs/CommonConfigurationKeys.java      |   5 +
 .../hadoop/io/erasurecode/SchemaLoader.java     | 147 +++++++++++++++++++
 .../hadoop/io/erasurecode/TestSchemaLoader.java |  80 ++++++++++
 4 files changed, 272 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcb987a7/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
new file mode 100644
index 0000000..e619485
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/conf/ecschema-def.xml
@@ -0,0 +1,40 @@
+<?xml version="1.0"?>
+
+<!--
+ 
+ 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.
+
+-->
+
+<!--
+Please define your EC schemas here. Note, once these schemas are loaded
+and referenced by EC storage policies, any change to them will be ignored.
+You can modify and remove those not used yet, or add new ones.
+-->
+
+<schemas>
+  <schema name="RS-6-3">
+    <k>6</k>
+    <m>3</m>
+    <codec>RS</codec>
+  </schema>
+  <schema name="RS-10-4">
+    <k>10</k>
+    <m>4</m>
+    <codec>RS</codec>
+  </schema>
+</schemas>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcb987a7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
index 70fea01..af32674 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/CommonConfigurationKeys.java
@@ -141,6 +141,11 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic {
   /** Supported erasure codec classes */
   public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs";
 
+  public static final String IO_ERASURECODE_SCHEMA_FILE_KEY =
+      "io.erasurecode.schema.file";
+  public static final String IO_ERASURECODE_SCHEMA_FILE_DEFAULT =
+      "ecschema-def.xml";
+
   /** Use XOR raw coder when possible for the RS codec */
   public static final String IO_ERASURECODE_CODEC_RS_USEXOR_KEY =
       "io.erasurecode.codec.rs.usexor";

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcb987a7/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
new file mode 100644
index 0000000..c51ed37
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/SchemaLoader.java
@@ -0,0 +1,147 @@
+/**
+ * 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.io.erasurecode;
+
+import org.apache.commons.logging.Log;
+import org.apache.commons.logging.LogFactory;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.w3c.dom.*;
+import org.xml.sax.SAXException;
+
+import javax.xml.parsers.DocumentBuilder;
+import javax.xml.parsers.DocumentBuilderFactory;
+import javax.xml.parsers.ParserConfigurationException;
+import java.io.File;
+import java.io.IOException;
+import java.net.URL;
+import java.util.*;
+
+/**
+ * A EC schema loading utility that loads predefined EC schemas from XML file
+ */
+public class SchemaLoader {
+  private static final Log LOG = LogFactory.getLog(SchemaLoader.class.getName());
+
+  /**
+   * Load predefined ec schemas from configuration file. This file is
+   * expected to be in the XML format.
+   */
+  public List<ECSchema> loadSchema(Configuration conf) {
+    File confFile = getSchemaFile(conf);
+    if (confFile == null) {
+      LOG.warn("Not found any predefined EC schema file");
+      return Collections.emptyList();
+    }
+
+    try {
+      return loadSchema(confFile);
+    } catch (ParserConfigurationException e) {
+      throw new RuntimeException("Failed to load schema file: " + confFile);
+    } catch (IOException e) {
+      throw new RuntimeException("Failed to load schema file: " + confFile);
+    } catch (SAXException e) {
+      throw new RuntimeException("Failed to load schema file: " + confFile);
+    }
+  }
+
+  private List<ECSchema> loadSchema(File schemaFile)
+      throws ParserConfigurationException, IOException, SAXException {
+
+    LOG.info("Loading predefined EC schema file " + schemaFile);
+
+    // Read and parse the schema file.
+    DocumentBuilderFactory dbf = DocumentBuilderFactory.newInstance();
+    dbf.setIgnoringComments(true);
+    DocumentBuilder builder = dbf.newDocumentBuilder();
+    Document doc = builder.parse(schemaFile);
+    Element root = doc.getDocumentElement();
+
+    if (!"schemas".equals(root.getTagName())) {
+      throw new RuntimeException("Bad EC schema config file: " +
+          "top-level element not <schemas>");
+    }
+
+    NodeList elements = root.getChildNodes();
+    List<ECSchema> schemas = new ArrayList<ECSchema>();
+    for (int i = 0; i < elements.getLength(); i++) {
+      Node node = elements.item(i);
+      if (node instanceof Element) {
+        Element element = (Element) node;
+        if ("schema".equals(element.getTagName())) {
+          ECSchema schema = loadSchema(element);
+            schemas.add(schema);
+        } else {
+          LOG.warn("Bad element in EC schema configuration file: " +
+              element.getTagName());
+        }
+      }
+    }
+
+    return schemas;
+  }
+
+  /**
+   * Path to the XML file containing predefined ec schemas. If the path is
+   * relative, it is searched for in the classpath.
+   */
+  private File getSchemaFile(Configuration conf) {
+    String schemaFilePath = conf.get(
+        CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY,
+        CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_DEFAULT);
+    File schemaFile = new File(schemaFilePath);
+    if (! schemaFile.isAbsolute()) {
+      URL url = Thread.currentThread().getContextClassLoader()
+          .getResource(schemaFilePath);
+      if (url == null) {
+        LOG.warn(schemaFilePath + " not found on the classpath.");
+        schemaFile = null;
+      } else if (! url.getProtocol().equalsIgnoreCase("file")) {
+        throw new RuntimeException(
+            "EC predefined schema file " + url +
+                " found on the classpath is not on the local filesystem.");
+      } else {
+        schemaFile = new File(url.getPath());
+      }
+    }
+
+    return schemaFile;
+  }
+
+  /**
+   * Loads a schema from a schema element in the configuration file
+   */
+  private ECSchema loadSchema(Element element) {
+    String schemaName = element.getAttribute("name");
+    Map<String, String> ecOptions = new HashMap<String, String>();
+    NodeList fields = element.getChildNodes();
+
+    for (int i = 0; i < fields.getLength(); i++) {
+      Node fieldNode = fields.item(i);
+      if (fieldNode instanceof Element) {
+        Element field = (Element) fieldNode;
+        String tagName = field.getTagName();
+        String value = ((Text) field.getFirstChild()).getData().trim();
+        ecOptions.put(tagName, value);
+      }
+    }
+
+    ECSchema schema = new ECSchema(schemaName, ecOptions);
+    return schema;
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/dcb987a7/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
new file mode 100644
index 0000000..7bb0a9a
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestSchemaLoader.java
@@ -0,0 +1,80 @@
+/**
+ * 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.io.erasurecode;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.CommonConfigurationKeys;
+import org.junit.Test;
+
+import java.io.File;
+import java.io.FileWriter;
+import java.io.PrintWriter;
+import java.util.List;
+
+import static org.junit.Assert.assertEquals;
+
+public class TestSchemaLoader {
+
+  final static String TEST_DIR = new File(System.getProperty(
+      "test.build.data", "/tmp")).getAbsolutePath();
+
+  final static String SCHEMA_FILE = new File(TEST_DIR, "test-ecschema")
+      .getAbsolutePath();
+
+  @Test
+  public void testLoadSchema() throws Exception {
+    PrintWriter out = new PrintWriter(new FileWriter(SCHEMA_FILE));
+    out.println("<?xml version=\"1.0\"?>");
+    out.println("<schemas>");
+    out.println("  <schema name=\"RSk6m3\">");
+    out.println("    <k>6</k>");
+    out.println("    <m>3</m>");
+    out.println("    <codec>RS</codec>");
+    out.println("  </schema>");
+    out.println("  <schema name=\"RSk10m4\">");
+    out.println("    <k>10</k>");
+    out.println("    <m>4</m>");
+    out.println("    <codec>RS</codec>");
+    out.println("  </schema>");
+    out.println("</schemas>");
+    out.close();
+
+    Configuration conf = new Configuration();
+    conf.set(CommonConfigurationKeys.IO_ERASURECODE_SCHEMA_FILE_KEY,
+        SCHEMA_FILE);
+
+    SchemaLoader schemaLoader = new SchemaLoader();
+    List<ECSchema> schemas = schemaLoader.loadSchema(conf);
+
+    assertEquals(2, schemas.size());
+
+    ECSchema schema1 = schemas.get(0);
+    assertEquals("RSk6m3", schema1.getSchemaName());
+    assertEquals(3, schema1.getOptions().size());
+    assertEquals(6, schema1.getNumDataUnits());
+    assertEquals(3, schema1.getNumParityUnits());
+    assertEquals("RS", schema1.getCodecName());
+
+    ECSchema schema2 = schemas.get(1);
+    assertEquals("RSk10m4", schema2.getSchemaName());
+    assertEquals(3, schema2.getOptions().size());
+    assertEquals(10, schema2.getNumDataUnits());
+    assertEquals(4, schema2.getNumParityUnits());
+    assertEquals("RS", schema2.getCodecName());
+  }
+}
\ No newline at end of file


[06/50] [abbrv] hadoop git commit: HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/564860ca
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/564860ca
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/564860ca

Branch: refs/heads/HDFS-7285
Commit: 564860ca4d56219ef340b38bc749f6c67bb0373a
Parents: cae7851
Author: drankye <ka...@intel.com>
Authored: Thu Mar 5 22:51:52 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:36:07 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |   4 +
 .../apache/hadoop/io/erasurecode/ECSchema.java  | 203 +++++++++++++++++++
 .../hadoop/io/erasurecode/TestECSchema.java     |  54 +++++
 3 files changed, 261 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/564860ca/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index 7bbacf7..ee42c84 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -12,3 +12,7 @@
     HADOOP-11542. Raw Reed-Solomon coder in pure Java. Contributed by Kai Zheng
     ( Kai Zheng )
 
+    HADOOP-11643. Define EC schema API for ErasureCodec. Contributed by Kai Zheng
+    ( Kai Zheng )
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/564860ca/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
new file mode 100644
index 0000000..8dc3f45
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java
@@ -0,0 +1,203 @@
+/**
+ * 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.io.erasurecode;
+
+import java.util.Collections;
+import java.util.Map;
+
+/**
+ * Erasure coding schema to housekeeper relevant information.
+ */
+public class ECSchema {
+  public static final String NUM_DATA_UNITS_KEY = "k";
+  public static final String NUM_PARITY_UNITS_KEY = "m";
+  public static final String CODEC_NAME_KEY = "codec";
+  public static final String CHUNK_SIZE_KEY = "chunkSize";
+  public static final int DEFAULT_CHUNK_SIZE = 64 * 1024; // 64K
+
+  private String schemaName;
+  private String codecName;
+  private Map<String, String> options;
+  private int numDataUnits;
+  private int numParityUnits;
+  private int chunkSize;
+
+  /**
+   * Constructor with schema name and provided options. Note the options may
+   * contain additional information for the erasure codec to interpret further.
+   * @param schemaName schema name
+   * @param options schema options
+   */
+  public ECSchema(String schemaName, Map<String, String> options) {
+    assert (schemaName != null && ! schemaName.isEmpty());
+
+    this.schemaName = schemaName;
+
+    if (options == null || options.isEmpty()) {
+      throw new IllegalArgumentException("No schema options are provided");
+    }
+
+    String codecName = options.get(CODEC_NAME_KEY);
+    if (codecName == null || codecName.isEmpty()) {
+      throw new IllegalArgumentException("No codec option is provided");
+    }
+
+    int dataUnits = 0, parityUnits = 0;
+    try {
+      if (options.containsKey(NUM_DATA_UNITS_KEY)) {
+        dataUnits = Integer.parseInt(options.get(NUM_DATA_UNITS_KEY));
+      }
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException("Option value " +
+          options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          " is found. It should be an integer");
+    }
+
+    try {
+      if (options.containsKey(NUM_PARITY_UNITS_KEY)) {
+        parityUnits = Integer.parseInt(options.get(NUM_PARITY_UNITS_KEY));
+      }
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException("Option value " +
+          options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          " is found. It should be an integer");
+    }
+
+    initWith(codecName, dataUnits, parityUnits, options);
+  }
+
+  /**
+   * Constructor with key parameters provided. Note the options may contain
+   * additional information for the erasure codec to interpret further.
+   * @param schemaName
+   * @param codecName
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param options
+   */
+  public ECSchema(String schemaName, String codecName,
+                  int numDataUnits, int numParityUnits,
+                  Map<String, String> options) {
+    assert (schemaName != null && ! schemaName.isEmpty());
+    assert (codecName != null && ! codecName.isEmpty());
+
+    this.schemaName = schemaName;
+    initWith(codecName, numDataUnits, numParityUnits, options);
+  }
+
+  private void initWith(String codecName, int numDataUnits, int numParityUnits,
+                        Map<String, String> options) {
+    this.codecName = codecName;
+    this.numDataUnits = numDataUnits;
+    this.numParityUnits = numParityUnits;
+
+    this.options = options != null ? Collections.unmodifiableMap(options) :
+        Collections.EMPTY_MAP;
+
+    this.chunkSize = DEFAULT_CHUNK_SIZE;
+    try {
+      if (options.containsKey(CHUNK_SIZE_KEY)) {
+        this.chunkSize = Integer.parseInt(options.get(CHUNK_SIZE_KEY));
+      }
+    } catch (NumberFormatException e) {
+      throw new IllegalArgumentException("Option value " +
+          options.get(CHUNK_SIZE_KEY) + " for " + CHUNK_SIZE_KEY +
+          " is found. It should be an integer");
+    }
+
+    boolean isFine = numDataUnits > 0 && numParityUnits > 0 && chunkSize > 0;
+    if (! isFine) {
+      throw new IllegalArgumentException("Bad codec options are found");
+    }
+  }
+
+  /**
+   * Get the schema name
+   * @return schema name
+   */
+  public String getSchemaName() {
+    return schemaName;
+  }
+
+  /**
+   * Get the codec name
+   * @return codec name
+   */
+  public String getCodecName() {
+    return codecName;
+  }
+
+  /**
+   * Get erasure coding options
+   * @return encoding options
+   */
+  public Map<String, String> getOptions() {
+    return options;
+  }
+
+  /**
+   * Get required data units count in a coding group
+   * @return count of data units
+   */
+  public int getNumDataUnits() {
+    return numDataUnits;
+  }
+
+  /**
+   * Get required parity units count in a coding group
+   * @return count of parity units
+   */
+  public int getNumParityUnits() {
+    return numParityUnits;
+  }
+
+  /**
+   * Get chunk buffer size for the erasure encoding/decoding.
+   * @return chunk buffer size
+   */
+  public int getChunkSize() {
+    return chunkSize;
+  }
+
+  /**
+   * Make a meaningful string representation for log output.
+   * @return string representation
+   */
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder("ECSchema=[");
+
+    sb.append("Name=" + schemaName + ",");
+    sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ",");
+    sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits + ",");
+    sb.append(CHUNK_SIZE_KEY + "=" + chunkSize + ",");
+
+    for (String opt : options.keySet()) {
+      boolean skip = (opt.equals(NUM_DATA_UNITS_KEY) ||
+          opt.equals(NUM_PARITY_UNITS_KEY) ||
+          opt.equals(CHUNK_SIZE_KEY));
+      if (! skip) {
+        sb.append(opt + "=" + options.get(opt) + ",");
+      }
+    }
+
+    sb.append("]");
+
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/564860ca/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
new file mode 100644
index 0000000..4285ef0
--- /dev/null
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.java
@@ -0,0 +1,54 @@
+/**
+ * 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.io.erasurecode;
+
+import org.junit.Test;
+import static org.junit.Assert.assertEquals;
+import java.util.HashMap;
+import java.util.Map;
+
+public class TestECSchema {
+
+  @Test
+  public void testGoodSchema() {
+    String schemaName = "goodSchema";
+    int numDataUnits = 6;
+    int numParityUnits = 3;
+    int chunkSize = 64 * 1024 * 1024;
+    String codec = "rs";
+    String extraOption = "extraOption";
+    String extraOptionValue = "extraOptionValue";
+
+    Map<String, String> options = new HashMap<String, String>();
+    options.put(ECSchema.NUM_DATA_UNITS_KEY, String.valueOf(numDataUnits));
+    options.put(ECSchema.NUM_PARITY_UNITS_KEY, String.valueOf(numParityUnits));
+    options.put(ECSchema.CODEC_NAME_KEY, codec);
+    options.put(ECSchema.CHUNK_SIZE_KEY, String.valueOf(chunkSize));
+    options.put(extraOption, extraOptionValue);
+
+    ECSchema schema = new ECSchema(schemaName, options);
+    System.out.println(schema.toString());
+    
+    assertEquals(schemaName, schema.getSchemaName());
+    assertEquals(numDataUnits, schema.getNumDataUnits());
+    assertEquals(numParityUnits, schema.getNumParityUnits());
+    assertEquals(chunkSize, schema.getChunkSize());
+    assertEquals(codec, schema.getCodecName());
+    assertEquals(extraOptionValue, schema.getOptions().get(extraOption));
+  }
+}


[16/50] [abbrv] hadoop git commit: HDFS-7369. Erasure coding: distribute recovery work for striped blocks to DataNode. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7369. Erasure coding: distribute recovery work for striped blocks to DataNode. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/2c549c4e
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/2c549c4e
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/2c549c4e

Branch: refs/heads/HDFS-7285
Commit: 2c549c4e814c46f0aa367c08bd750ba337430ac3
Parents: e692325
Author: Zhe Zhang <zh...@apache.org>
Authored: Wed Mar 18 15:52:36 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:43:46 2015 -0700

----------------------------------------------------------------------
 .../server/blockmanagement/BlockCollection.java |   5 +
 .../server/blockmanagement/BlockManager.java    | 291 +++++++++++++------
 .../blockmanagement/DatanodeDescriptor.java     |  72 ++++-
 .../server/blockmanagement/DatanodeManager.java |  20 +-
 .../hadoop/hdfs/server/namenode/INodeFile.java  |   9 +-
 .../server/protocol/BlockECRecoveryCommand.java |  63 ++++
 .../hdfs/server/protocol/DatanodeProtocol.java  |   1 +
 .../blockmanagement/BlockManagerTestUtil.java   |   2 +-
 .../blockmanagement/TestBlockManager.java       |  22 +-
 .../TestRecoverStripedBlocks.java               | 107 +++++++
 10 files changed, 484 insertions(+), 108 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
index 440a081..50dd17b 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockCollection.java
@@ -86,4 +86,9 @@ public interface BlockCollection {
    * @return whether the block collection is under construction.
    */
   public boolean isUnderConstruction();
+
+  /**
+   * @return whether the block collection is in striping format
+   */
+  public boolean isStriped();
 }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
index 25a2446..8ea783f 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManager.java
@@ -42,6 +42,7 @@ import org.apache.hadoop.classification.InterfaceAudience;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
 import org.apache.hadoop.hdfs.DFSConfigKeys;
 import org.apache.hadoop.hdfs.DFSUtil;
 import org.apache.hadoop.hdfs.HAUtil;
@@ -532,9 +533,9 @@ public class BlockManager {
     
     NumberReplicas numReplicas = new NumberReplicas();
     // source node returned is not used
-    chooseSourceDatanode(block, containingNodes,
+    chooseSourceDatanodes(getStoredBlock(block), containingNodes,
         containingLiveReplicasNodes, numReplicas,
-        UnderReplicatedBlocks.LEVEL);
+        new LinkedList<Short>(), 1, UnderReplicatedBlocks.LEVEL);
     
     // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are 
     // not included in the numReplicas.liveReplicas() count
@@ -1327,15 +1328,15 @@ public class BlockManager {
   }
 
   /**
-   * Scan blocks in {@link #neededReplications} and assign replication
-   * work to data-nodes they belong to.
+   * Scan blocks in {@link #neededReplications} and assign recovery
+   * (replication or erasure coding) work to data-nodes they belong to.
    *
    * The number of process blocks equals either twice the number of live
    * data-nodes or the number of under-replicated blocks whichever is less.
    *
    * @return number of blocks scheduled for replication during this iteration.
    */
-  int computeReplicationWork(int blocksToProcess) {
+  int computeBlockRecoveryWork(int blocksToProcess) {
     List<List<BlockInfo>> blocksToReplicate = null;
     namesystem.writeLock();
     try {
@@ -1345,30 +1346,32 @@ public class BlockManager {
     } finally {
       namesystem.writeUnlock();
     }
-    return computeReplicationWorkForBlocks(blocksToReplicate);
+    return computeRecoveryWorkForBlocks(blocksToReplicate);
   }
 
-  /** Replicate a set of blocks
+  /**
+   * Recover a set of blocks to full strength through replication or
+   * erasure coding
    *
-   * @param blocksToReplicate blocks to be replicated, for each priority
+   * @param blocksToRecover blocks to be recovered, for each priority
    * @return the number of blocks scheduled for replication
    */
   @VisibleForTesting
-  int computeReplicationWorkForBlocks(List<List<BlockInfo>> blocksToReplicate) {
+  int computeRecoveryWorkForBlocks(List<List<BlockInfo>> blocksToRecover) {
     int requiredReplication, numEffectiveReplicas;
     List<DatanodeDescriptor> containingNodes;
-    DatanodeDescriptor srcNode;
     BlockCollection bc = null;
     int additionalReplRequired;
 
     int scheduledWork = 0;
-    List<ReplicationWork> work = new LinkedList<ReplicationWork>();
+    List<BlockRecoveryWork> recovWork = new LinkedList<>();
 
+    // Step 1: categorize at-risk blocks into replication and EC tasks
     namesystem.writeLock();
     try {
       synchronized (neededReplications) {
-        for (int priority = 0; priority < blocksToReplicate.size(); priority++) {
-          for (BlockInfo block : blocksToReplicate.get(priority)) {
+        for (int priority = 0; priority < blocksToRecover.size(); priority++) {
+          for (BlockInfo block : blocksToRecover.get(priority)) {
             // block should belong to a file
             bc = blocksMap.getBlockCollection(block);
             // abandoned block or block reopened for append
@@ -1382,25 +1385,31 @@ public class BlockManager {
             requiredReplication = bc.getBlockReplication();
 
             // get a source data-node
-            containingNodes = new ArrayList<DatanodeDescriptor>();
-            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<DatanodeStorageInfo>();
+            containingNodes = new ArrayList<>();
+            List<DatanodeStorageInfo> liveReplicaNodes = new ArrayList<>();
             NumberReplicas numReplicas = new NumberReplicas();
-            srcNode = chooseSourceDatanode(
+            List<Short> missingBlockIndices = new LinkedList<>();
+            DatanodeDescriptor[] srcNodes;
+            int numSourceNodes = bc.isStriped() ?
+                HdfsConstants.NUM_DATA_BLOCKS : 1;
+            srcNodes = chooseSourceDatanodes(
                 block, containingNodes, liveReplicaNodes, numReplicas,
-                priority);
-            if(srcNode == null) { // block can not be replicated from any node
-              LOG.debug("Block " + block + " cannot be repl from any node");
+                missingBlockIndices, numSourceNodes, priority);
+            if(srcNodes == null || srcNodes.length == 0) {
+              // block can not be replicated from any node
+              LOG.debug("Block " + block + " cannot be recovered " +
+                  "from any node");
               continue;
             }
 
-            // liveReplicaNodes can include READ_ONLY_SHARED replicas which are 
+            // liveReplicaNodes can include READ_ONLY_SHARED replicas which are
             // not included in the numReplicas.liveReplicas() count
             assert liveReplicaNodes.size() >= numReplicas.liveReplicas();
 
             // do not schedule more if enough replicas is already pending
             numEffectiveReplicas = numReplicas.liveReplicas() +
                                     pendingReplications.getNumReplicas(block);
-      
+
             if (numEffectiveReplicas >= requiredReplication) {
               if ( (pendingReplications.getNumReplicas(block) > 0) ||
                    (blockHasEnoughRacks(block)) ) {
@@ -1417,9 +1426,21 @@ public class BlockManager {
             } else {
               additionalReplRequired = 1; // Needed on a new rack
             }
-            work.add(new ReplicationWork(block, bc, srcNode,
-                containingNodes, liveReplicaNodes, additionalReplRequired,
-                priority));
+            if (bc.isStriped()) {
+              ErasureCodingWork ecw = new ErasureCodingWork(block, bc, srcNodes,
+                  containingNodes, liveReplicaNodes, additionalReplRequired,
+                  priority);
+              short[] missingBlockArray = new short[missingBlockIndices.size()];
+              for (int i = 0 ; i < missingBlockIndices.size(); i++) {
+                missingBlockArray[i] = missingBlockIndices.get(i);
+              }
+              ecw.setMissingBlockIndices(missingBlockArray);
+              recovWork.add(ecw);
+            } else {
+              recovWork.add(new ReplicationWork(block, bc, srcNodes,
+                  containingNodes, liveReplicaNodes, additionalReplRequired,
+                  priority));
+            }
           }
         }
       }
@@ -1427,8 +1448,9 @@ public class BlockManager {
       namesystem.writeUnlock();
     }
 
+    // Step 2: choose target nodes for each recovery task
     final Set<Node> excludedNodes = new HashSet<Node>();
-    for(ReplicationWork rw : work){
+    for(BlockRecoveryWork rw : recovWork){
       // Exclude all of the containing nodes from being targets.
       // This list includes decommissioning or corrupt nodes.
       excludedNodes.clear();
@@ -1442,9 +1464,10 @@ public class BlockManager {
       rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes);
     }
 
+    // Step 3: add tasks to the DN
     namesystem.writeLock();
     try {
-      for(ReplicationWork rw : work){
+      for(BlockRecoveryWork rw : recovWork){
         final DatanodeStorageInfo[] targets = rw.targets;
         if(targets == null || targets.length == 0){
           rw.targets = null;
@@ -1483,7 +1506,7 @@ public class BlockManager {
 
           if ( (numReplicas.liveReplicas() >= requiredReplication) &&
                (!blockHasEnoughRacks(block)) ) {
-            if (rw.srcNode.getNetworkLocation().equals(
+            if (rw.srcNodes[0].getNetworkLocation().equals(
                 targets[0].getDatanodeDescriptor().getNetworkLocation())) {
               //No use continuing, unless a new rack in this case
               continue;
@@ -1491,7 +1514,17 @@ public class BlockManager {
           }
 
           // Add block to the to be replicated list
-          rw.srcNode.addBlockToBeReplicated(block, targets);
+          if (bc.isStriped()) {
+            assert rw instanceof ErasureCodingWork;
+            assert rw.targets.length > 0;
+            rw.targets[0].getDatanodeDescriptor().addBlockToBeErasureCoded(
+                new ExtendedBlock(namesystem.getBlockPoolId(), block),
+                rw.srcNodes, rw.targets,
+                ((ErasureCodingWork)rw).getMissingBlockIndicies());
+          }
+          else {
+            rw.srcNodes[0].addBlockToBeReplicated(block, targets);
+          }
           scheduledWork++;
           DatanodeStorageInfo.incrementBlocksScheduled(targets);
 
@@ -1515,7 +1548,7 @@ public class BlockManager {
 
     if (blockLog.isInfoEnabled()) {
       // log which blocks have been scheduled for replication
-      for(ReplicationWork rw : work){
+      for(BlockRecoveryWork rw : recovWork){
         DatanodeStorageInfo[] targets = rw.targets;
         if (targets != null && targets.length != 0) {
           StringBuilder targetList = new StringBuilder("datanode(s)");
@@ -1523,7 +1556,7 @@ public class BlockManager {
             targetList.append(' ');
             targetList.append(targets[k].getDatanodeDescriptor());
           }
-          blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNode,
+          blockLog.info("BLOCK* ask {} to replicate {} to {}", rw.srcNodes,
               rw.block, targetList);
         }
       }
@@ -1609,55 +1642,66 @@ public class BlockManager {
   }
 
   /**
-   * Parse the data-nodes the block belongs to and choose one,
-   * which will be the replication source.
+   * Parse the data-nodes the block belongs to and choose a certain number
+   * from them to be the recovery sources.
    *
    * We prefer nodes that are in DECOMMISSION_INPROGRESS state to other nodes
    * since the former do not have write traffic and hence are less busy.
    * We do not use already decommissioned nodes as a source.
-   * Otherwise we choose a random node among those that did not reach their
-   * replication limits.  However, if the replication is of the highest priority
-   * and all nodes have reached their replication limits, we will choose a
-   * random node despite the replication limit.
+   * Otherwise we randomly choose nodes among those that did not reach their
+   * replication limits. However, if the recovery work is of the highest
+   * priority and all nodes have reached their replication limits, we will
+   * randomly choose the desired number of nodes despite the replication limit.
    *
    * In addition form a list of all nodes containing the block
    * and calculate its replication numbers.
    *
    * @param block Block for which a replication source is needed
-   * @param containingNodes List to be populated with nodes found to contain the 
-   *                        given block
-   * @param nodesContainingLiveReplicas List to be populated with nodes found to
-   *                                    contain live replicas of the given block
-   * @param numReplicas NumberReplicas instance to be initialized with the 
-   *                                   counts of live, corrupt, excess, and
-   *                                   decommissioned replicas of the given
-   *                                   block.
+   * @param containingNodes List to be populated with nodes found to contain
+   *                        the given block
+   * @param nodesContainingLiveReplicas List to be populated with nodes found
+   *                                    to contain live replicas of the given
+   *                                    block
+   * @param numReplicas NumberReplicas instance to be initialized with the
+   *                    counts of live, corrupt, excess, and decommissioned
+   *                    replicas of the given block.
+   * @param missingBlockIndices List to be populated with indices of missing
+   *                            blocks in a striped block group or missing
+   *                            replicas of a replicated block
+   * @param numSourceNodes integer specifying the number of source nodes to
+   *                       choose
    * @param priority integer representing replication priority of the given
    *                 block
-   * @return the DatanodeDescriptor of the chosen node from which to replicate
-   *         the given block
-   */
-   @VisibleForTesting
-   DatanodeDescriptor chooseSourceDatanode(Block block,
-       List<DatanodeDescriptor> containingNodes,
-       List<DatanodeStorageInfo>  nodesContainingLiveReplicas,
-       NumberReplicas numReplicas,
-       int priority) {
+   * @return the array of DatanodeDescriptor of the chosen nodes from which to
+   *         recover the given block
+   */
+  @VisibleForTesting
+  DatanodeDescriptor[] chooseSourceDatanodes(BlockInfo block,
+      List<DatanodeDescriptor> containingNodes,
+      List<DatanodeStorageInfo> nodesContainingLiveReplicas,
+      NumberReplicas numReplicas,
+      List<Short> missingBlockIndices, int numSourceNodes, int priority) {
     containingNodes.clear();
     nodesContainingLiveReplicas.clear();
-    DatanodeDescriptor srcNode = null;
+    LinkedList<DatanodeDescriptor> srcNodes = new LinkedList<>();
     int live = 0;
     int decommissioned = 0;
     int decommissioning = 0;
     int corrupt = 0;
     int excess = 0;
-    
+    missingBlockIndices.clear();
+    Set<Short> healthyIndices = new HashSet<>();
+
     Collection<DatanodeDescriptor> nodesCorrupt = corruptReplicas.getNodes(block);
     for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) {
+      if (block.isStriped()) {
+        healthyIndices.add((short) ((BlockInfoStriped) block).
+            getStorageBlockIndex(storage));
+      }
       final DatanodeDescriptor node = storage.getDatanodeDescriptor();
       LightWeightLinkedSet<Block> excessBlocks =
         excessReplicateMap.get(node.getDatanodeUuid());
-      int countableReplica = storage.getState() == State.NORMAL ? 1 : 0; 
+      int countableReplica = storage.getState() == State.NORMAL ? 1 : 0;
       if ((nodesCorrupt != null) && (nodesCorrupt.contains(node)))
         corrupt += countableReplica;
       else if (node.isDecommissionInProgress()) {
@@ -1693,20 +1737,32 @@ public class BlockManager {
         continue;
 
       // We got this far, current node is a reasonable choice
-      if (srcNode == null) {
-        srcNode = node;
+      if(srcNodes.size() < numSourceNodes) {
+        srcNodes.add(node);
         continue;
       }
       // switch to a different node randomly
       // this to prevent from deterministically selecting the same node even
       // if the node failed to replicate the block on previous iterations
-      if(DFSUtil.getRandom().nextBoolean())
-        srcNode = node;
+      if(DFSUtil.getRandom().nextBoolean()) {
+        int pos = DFSUtil.getRandom().nextInt(numSourceNodes);
+        if(!srcNodes.get(pos).isDecommissionInProgress()) {
+          srcNodes.set(pos, node);
+        }
+      }
+    }
+    if (block.isStriped()) {
+      for (short i = 0; i < HdfsConstants.NUM_DATA_BLOCKS +
+          HdfsConstants.NUM_PARITY_BLOCKS; i++) {
+        if (!healthyIndices.contains(i)) {
+          missingBlockIndices.add(i);
+        }
+      }
     }
     if(numReplicas != null)
       numReplicas.initialize(live, decommissioned, decommissioning, corrupt,
           excess, 0);
-    return srcNode;
+    return srcNodes.toArray(new DatanodeDescriptor[srcNodes.size()]);
   }
 
   /**
@@ -1736,7 +1792,7 @@ public class BlockManager {
        */
     }
   }
-  
+
   /**
    * StatefulBlockInfo is used to build the "toUC" list, which is a list of
    * updates to the information about under-construction blocks.
@@ -3675,7 +3731,7 @@ public class BlockManager {
   }
 
   /**
-   * Periodically calls computeReplicationWork().
+   * Periodically calls computeBlockRecoveryWork().
    */
   private class ReplicationMonitor implements Runnable {
 
@@ -3733,7 +3789,7 @@ public class BlockManager {
     final int nodesToProcess = (int) Math.ceil(numlive
         * this.blocksInvalidateWorkPct);
 
-    int workFound = this.computeReplicationWork(blocksToProcess);
+    int workFound = this.computeBlockRecoveryWork(blocksToProcess);
 
     // Update counters
     namesystem.writeLock();
@@ -3760,49 +3816,118 @@ public class BlockManager {
     postponedMisreplicatedBlocks.clear();
     postponedMisreplicatedBlocksCount.set(0);
   }
-  
 
-  private static class ReplicationWork {
-
-    private final BlockInfo block;
-    private final BlockCollection bc;
+  /**
+   * This class is used internally by {@link this#computeRecoveryWorkForBlocks}
+   * to represent a task to recover a block through replication or erasure
+   * coding. Recovery is done by transferring data from {@link srcNodes} to
+   * {@link targets}
+   */
+  private static class BlockRecoveryWork {
+    protected final BlockInfo block;
+    protected final BlockCollection bc;
 
-    private final DatanodeDescriptor srcNode;
-    private final List<DatanodeDescriptor> containingNodes;
-    private final List<DatanodeStorageInfo> liveReplicaStorages;
-    private final int additionalReplRequired;
+    /**
+     * An erasure coding recovery task has multiple source nodes.
+     * A replication task only has 1 source node, stored on top of the array
+     */
+    protected final DatanodeDescriptor[] srcNodes;
+    /** Nodes containing the block; avoid them in choosing new targets */
+    protected final List<DatanodeDescriptor> containingNodes;
+    /** Required by {@link BlockPlacementPolicy#chooseTarget} */
+    protected final List<DatanodeStorageInfo> liveReplicaStorages;
+    protected final int additionalReplRequired;
 
-    private DatanodeStorageInfo targets[];
-    private final int priority;
+    protected DatanodeStorageInfo[] targets;
+    protected final int priority;
 
-    public ReplicationWork(BlockInfo block,
+    public BlockRecoveryWork(BlockInfo block,
         BlockCollection bc,
-        DatanodeDescriptor srcNode,
+        DatanodeDescriptor[] srcNodes,
         List<DatanodeDescriptor> containingNodes,
         List<DatanodeStorageInfo> liveReplicaStorages,
         int additionalReplRequired,
         int priority) {
       this.block = block;
       this.bc = bc;
-      this.srcNode = srcNode;
-      this.srcNode.incrementPendingReplicationWithoutTargets();
+      this.srcNodes = srcNodes;
       this.containingNodes = containingNodes;
       this.liveReplicaStorages = liveReplicaStorages;
       this.additionalReplRequired = additionalReplRequired;
       this.priority = priority;
       this.targets = null;
     }
-    
-    private void chooseTargets(BlockPlacementPolicy blockplacement,
+
+    protected void chooseTargets(BlockPlacementPolicy blockplacement,
+        BlockStoragePolicySuite storagePolicySuite,
+        Set<Node> excludedNodes) {
+    }
+  }
+
+  private static class ReplicationWork extends BlockRecoveryWork {
+
+    public ReplicationWork(BlockInfo block,
+        BlockCollection bc,
+        DatanodeDescriptor[] srcNodes,
+        List<DatanodeDescriptor> containingNodes,
+        List<DatanodeStorageInfo> liveReplicaStorages,
+        int additionalReplRequired,
+        int priority) {
+      super(block, bc, srcNodes, containingNodes,
+          liveReplicaStorages, additionalReplRequired, priority);
+      LOG.debug("Creating a ReplicationWork to recover " + block);
+    }
+
+    protected void chooseTargets(BlockPlacementPolicy blockplacement,
+        BlockStoragePolicySuite storagePolicySuite,
+        Set<Node> excludedNodes) {
+      assert srcNodes.length > 0
+          : "At least 1 source node should have been selected";
+      try {
+        targets = blockplacement.chooseTarget(bc.getName(),
+            additionalReplRequired, srcNodes[0], liveReplicaStorages, false,
+            excludedNodes, block.getNumBytes(),
+            storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
+      } finally {
+        srcNodes[0].decrementPendingReplicationWithoutTargets();
+      }
+    }
+  }
+
+  private static class ErasureCodingWork extends BlockRecoveryWork {
+
+    private short[] missingBlockIndicies = null;
+
+    public ErasureCodingWork(BlockInfo block,
+        BlockCollection bc,
+        DatanodeDescriptor[] srcNodes,
+        List<DatanodeDescriptor> containingNodes,
+        List<DatanodeStorageInfo> liveReplicaStorages,
+        int additionalReplRequired,
+        int priority) {
+      super(block, bc, srcNodes, containingNodes,
+          liveReplicaStorages, additionalReplRequired, priority);
+      LOG.debug("Creating an ErasureCodingWork to recover " + block);
+    }
+
+    public short[] getMissingBlockIndicies() {
+      return missingBlockIndicies;
+    }
+
+    public void setMissingBlockIndices(short[] missingBlockIndicies) {
+      this.missingBlockIndicies = missingBlockIndicies;
+    }
+
+    protected void chooseTargets(BlockPlacementPolicy blockplacement,
         BlockStoragePolicySuite storagePolicySuite,
         Set<Node> excludedNodes) {
       try {
+        // TODO: new placement policy for EC considering multiple writers
         targets = blockplacement.chooseTarget(bc.getName(),
-            additionalReplRequired, srcNode, liveReplicaStorages, false,
+            additionalReplRequired, srcNodes[0], liveReplicaStorages, false,
             excludedNodes, block.getNumBytes(),
             storagePolicySuite.getPolicy(bc.getStoragePolicyID()));
       } finally {
-        srcNode.decrementPendingReplicationWithoutTargets();
       }
     }
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index b7a3489..7bc5e7e 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -29,6 +29,7 @@ import java.util.List;
 import java.util.Map;
 import java.util.Queue;
 import java.util.Set;
+import java.util.Arrays;
 
 import com.google.common.annotations.VisibleForTesting;
 
@@ -41,6 +42,7 @@ import org.apache.hadoop.fs.StorageType;
 import org.apache.hadoop.hdfs.protocol.Block;
 import org.apache.hadoop.hdfs.protocol.DatanodeID;
 import org.apache.hadoop.hdfs.protocol.DatanodeInfo;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.protocol.BlockReportContext;
 import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage;
@@ -97,6 +99,33 @@ public class DatanodeDescriptor extends DatanodeInfo {
     }
   }
 
+  /** Block and targets pair */
+  @InterfaceAudience.Private
+  @InterfaceStability.Evolving
+  public static class BlockECRecoveryInfo {
+    public final ExtendedBlock block;
+    public final DatanodeDescriptor[] sources;
+    public final DatanodeStorageInfo[] targets;
+    public final short[] missingBlockIndices;
+
+    BlockECRecoveryInfo(ExtendedBlock block, DatanodeDescriptor[] sources,
+        DatanodeStorageInfo[] targets, short[] missingBlockIndices) {
+      this.block = block;
+      this.sources = sources;
+      this.targets = targets;
+      this.missingBlockIndices = missingBlockIndices;
+    }
+
+    @Override
+    public String toString() {
+      return new StringBuilder().append("BlockECRecoveryInfo(\n  ").
+          append("Recovering ").append(block).
+          append(" From: ").append(Arrays.asList(sources)).
+          append(" To: ").append(Arrays.asList(targets)).append(")\n").
+          toString();
+    }
+  }
+
   /** A BlockTargetPair queue. */
   private static class BlockQueue<E> {
     private final Queue<E> blockq = new LinkedList<E>();
@@ -217,12 +246,17 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private long bandwidth;
 
   /** A queue of blocks to be replicated by this datanode */
-  private final BlockQueue<BlockTargetPair> replicateBlocks = new BlockQueue<BlockTargetPair>();
+  private final BlockQueue<BlockTargetPair> replicateBlocks =
+      new BlockQueue<>();
+  /** A queue of blocks to be erasure coded by this datanode */
+  private final BlockQueue<BlockECRecoveryInfo> erasurecodeBlocks =
+      new BlockQueue<>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoContiguousUnderConstruction> recoverBlocks =
-                                new BlockQueue<BlockInfoContiguousUnderConstruction>();
+  private final BlockQueue<BlockInfoContiguousUnderConstruction>
+      recoverBlocks = new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
-  private final LightWeightHashSet<Block> invalidateBlocks = new LightWeightHashSet<Block>();
+  private final LightWeightHashSet<Block> invalidateBlocks =
+      new LightWeightHashSet<>();
 
   /* Variables for maintaining number of blocks scheduled to be written to
    * this storage. This count is approximate and might be slightly bigger
@@ -375,6 +409,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
       this.invalidateBlocks.clear();
       this.recoverBlocks.clear();
       this.replicateBlocks.clear();
+      this.erasurecodeBlocks.clear();
     }
     // pendingCached, cached, and pendingUncached are protected by the
     // FSN lock.
@@ -597,6 +632,20 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
+   * Store block erasure coding work.
+   */
+  void addBlockToBeErasureCoded(ExtendedBlock block, DatanodeDescriptor[] sources,
+      DatanodeStorageInfo[] targets, short[] missingBlockIndicies) {
+    assert(block != null && sources != null && sources.length > 0);
+    BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets,
+        missingBlockIndicies);
+    erasurecodeBlocks.offer(task);
+    BlockManager.LOG.debug("Adding block recovery task " + task +
+        "to " + getName() + ", current queue size is " +
+        erasurecodeBlocks.size());
+  }
+
+  /**
    * Store block recovery work.
    */
   void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
@@ -628,6 +677,13 @@ public class DatanodeDescriptor extends DatanodeInfo {
   }
 
   /**
+   * The number of work items that are pending to be replicated
+   */
+  int getNumberOfBlocksToBeErasureCoded() {
+    return erasurecodeBlocks.size();
+  }
+
+  /**
    * The number of block invalidation items that are pending to 
    * be sent to the datanode
    */
@@ -641,6 +697,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return replicateBlocks.poll(maxTransfers);
   }
 
+  public List<BlockECRecoveryInfo> getErasureCodeCommand(int maxTransfers) {
+    return erasurecodeBlocks.poll(maxTransfers);
+  }
+
   public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
     List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
@@ -841,6 +901,10 @@ public class DatanodeDescriptor extends DatanodeInfo {
     if (repl > 0) {
       sb.append(" ").append(repl).append(" blocks to be replicated;");
     }
+    int ec = erasurecodeBlocks.size();
+    if(ec > 0) {
+      sb.append(" ").append(ec).append(" blocks to be erasure coded;");
+    }
     int inval = invalidateBlocks.size();
     if (inval > 0) {
       sb.append(" ").append(inval).append(" blocks to be invalidated;");      

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index f68c4fd..6228f86 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -34,6 +34,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration;
 import org.apache.hadoop.hdfs.protocol.*;
 import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockTargetPair;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.CachedBlocksList;
 import org.apache.hadoop.hdfs.server.namenode.CachedBlock;
 import org.apache.hadoop.hdfs.server.namenode.NameNode;
@@ -1344,7 +1345,7 @@ public class DatanodeManager {
       VolumeFailureSummary volumeFailureSummary) throws IOException {
     synchronized (heartbeatManager) {
       synchronized (datanodeMap) {
-        DatanodeDescriptor nodeinfo = null;
+        DatanodeDescriptor nodeinfo;
         try {
           nodeinfo = getDatanode(nodeReg);
         } catch(UnregisteredNodeException e) {
@@ -1382,10 +1383,10 @@ public class DatanodeManager {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
-                new ArrayList<DatanodeStorageInfo>(storages.length);
-            for (int i = 0; i < storages.length; i++) {
-              if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) {
-                recoveryLocations.add(storages[i]);
+                new ArrayList<>(storages.length);
+            for (DatanodeStorageInfo storage : storages) {
+              if (!storage.getDatanodeDescriptor().isStale(staleInterval)) {
+                recoveryLocations.add(storage);
               }
             }
             // If we are performing a truncate recovery than set recovery fields
@@ -1424,7 +1425,7 @@ public class DatanodeManager {
           return new DatanodeCommand[] { brCommand };
         }
 
-        final List<DatanodeCommand> cmds = new ArrayList<DatanodeCommand>();
+        final List<DatanodeCommand> cmds = new ArrayList<>();
         //check pending replication
         List<BlockTargetPair> pendingList = nodeinfo.getReplicationCommand(
               maxTransfers);
@@ -1432,6 +1433,13 @@ public class DatanodeManager {
           cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId,
               pendingList));
         }
+        // checking pending erasure coding tasks
+        List<BlockECRecoveryInfo> pendingECList =
+            nodeinfo.getErasureCodeCommand(maxTransfers);
+        if (pendingECList != null) {
+          cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_CODEC,
+              pendingECList));
+        }
         //check block invalidation
         Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit);
         if (blks != null) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
index d0b4c32..97bbb9d 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFile.java
@@ -407,6 +407,7 @@ public class INodeFile extends INodeWithAdditionalFields
   }
 
   @Override // BlockCollection
+  // TODO: rename to reflect both replication and EC
   public short getBlockReplication() {
     short max = getFileReplication(CURRENT_STATE_ID);
     FileWithSnapshotFeature sf = this.getFileWithSnapshotFeature();
@@ -417,7 +418,8 @@ public class INodeFile extends INodeWithAdditionalFields
       }
       max = maxInSnapshot > max ? maxInSnapshot : max;
     }
-    return max;
+    return isStriped()?
+        HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS : max;
   }
 
   /** Set the replication factor of this file. */
@@ -1092,11 +1094,12 @@ public class INodeFile extends INodeWithAdditionalFields
         Arrays.asList(snapshotBlocks).contains(block);
   }
 
-  @VisibleForTesting
   /**
    * @return true if the file is in the striping layout.
    */
-  // TODO: move erasure coding policy to file XAttr (HDFS-7337)
+  @VisibleForTesting
+  @Override
+  // TODO: move erasure coding policy to file XAttr
   public boolean isStriped() {
     return getStoragePolicyID() == HdfsConstants.EC_STORAGE_POLICY_ID;
   }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
new file mode 100644
index 0000000..f7f02fd
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.java
@@ -0,0 +1,63 @@
+/**
+ * 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.server.protocol;
+
+import com.google.common.base.Joiner;
+import org.apache.hadoop.classification.InterfaceAudience;
+import org.apache.hadoop.classification.InterfaceStability;
+import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor.BlockECRecoveryInfo;
+
+import java.util.Collection;
+
+/**
+ * A BlockECRecoveryCommand is an instruction to a DataNode to reconstruct a
+ * striped block group with missing blocks.
+ *
+ * Upon receiving this command, the DataNode pulls data from other DataNodes
+ * hosting blocks in this group and reconstructs the lost blocks through codec
+ * calculation.
+ *
+ * After the reconstruction, the DataNode pushes the reconstructed blocks to
+ * their final destinations if necessary (e.g., the destination is different
+ * from the reconstruction node, or multiple blocks in a group are to be
+ * reconstructed).
+ */
+@InterfaceAudience.Private
+@InterfaceStability.Evolving
+public class BlockECRecoveryCommand extends DatanodeCommand {
+  final Collection<BlockECRecoveryInfo> ecTasks;
+
+  /**
+   * Create BlockECRecoveryCommand from a collection of
+   * {@link BlockECRecoveryInfo}, each representing a recovery task
+   */
+  public BlockECRecoveryCommand(int action,
+      Collection<BlockECRecoveryInfo> blockECRecoveryInfoList) {
+    super(action);
+    this.ecTasks = blockECRecoveryInfoList;
+  }
+
+  @Override
+  public String toString() {
+    StringBuilder sb = new StringBuilder();
+    sb.append("BlockECRecoveryCommand(\n  ");
+    Joiner.on("\n  ").appendTo(sb, ecTasks);
+    sb.append("\n)");
+    return sb.toString();
+  }
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
index a3b6004..b8ac165 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/DatanodeProtocol.java
@@ -76,6 +76,7 @@ public interface DatanodeProtocol {
   final static int DNA_BALANCERBANDWIDTHUPDATE = 8; // update balancer bandwidth
   final static int DNA_CACHE = 9;      // cache blocks
   final static int DNA_UNCACHE = 10;   // uncache blocks
+  final static int DNA_CODEC = 11;   // uncache blocks
 
   /** 
    * Register Datanode.

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
index 148135b..e25ee31 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockManagerTestUtil.java
@@ -161,7 +161,7 @@ public class BlockManagerTestUtil {
    */
   public static int computeAllPendingWork(BlockManager bm) {
     int work = computeInvalidationWork(bm);
-    work += bm.computeReplicationWork(Integer.MAX_VALUE);
+    work += bm.computeBlockRecoveryWork(Integer.MAX_VALUE);
     return work;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
index 2fca5be..cbea3d8 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockManager.java
@@ -448,8 +448,8 @@ public class TestBlockManager {
     assertEquals("Block not initially pending replication", 0,
         bm.pendingReplications.getNumReplicas(block));
     assertEquals(
-        "computeReplicationWork should indicate replication is needed", 1,
-        bm.computeReplicationWorkForBlocks(list_all));
+        "computeBlockRecoveryWork should indicate replication is needed", 1,
+        bm.computeRecoveryWorkForBlocks(list_all));
     assertTrue("replication is pending after work is computed",
         bm.pendingReplications.getNumReplicas(block) > 0);
 
@@ -503,22 +503,22 @@ public class TestBlockManager {
     assertNotNull("Chooses source node for a highest-priority replication"
         + " even if all available source nodes have reached their replication"
         + " limits below the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY));
+            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
 
     assertNull("Does not choose a source node for a less-than-highest-priority"
         + " replication since all available source nodes have reached"
         + " their replication limits.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED));
+            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)[0]);
 
     // Increase the replication count to test replication count > hard limit
     DatanodeStorageInfo targets[] = { origNodes.get(1).getStorageInfos()[0] };
@@ -526,12 +526,12 @@ public class TestBlockManager {
 
     assertNull("Does not choose a source node for a highest-priority"
         + " replication when all available nodes exceed the hard limit.",
-        bm.chooseSourceDatanode(
-            aBlock,
+        bm.chooseSourceDatanodes(
+            bm.getStoredBlock(aBlock),
             cntNodes,
             liveNodes,
             new NumberReplicas(),
-            UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY));
+            new LinkedList<Short>(), 1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]);
   }
 
   @Test

http://git-wip-us.apache.org/repos/asf/hadoop/blob/2c549c4e/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
new file mode 100644
index 0000000..d883c9b
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestRecoverStripedBlocks.java
@@ -0,0 +1,107 @@
+/**
+ * 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.server.blockmanagement;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.hdfs.DistributedFileSystem;
+import org.apache.hadoop.hdfs.DFSConfigKeys;
+import org.apache.hadoop.hdfs.DFSTestUtil;
+import org.apache.hadoop.hdfs.HdfsConfiguration;
+import org.apache.hadoop.hdfs.MiniDFSCluster;
+import org.apache.hadoop.hdfs.protocol.ExtendedBlock;
+import org.apache.hadoop.hdfs.client.HdfsAdmin;
+import org.apache.hadoop.hdfs.protocol.HdfsConstants;
+import org.apache.hadoop.hdfs.server.datanode.DataNode;
+import org.apache.hadoop.hdfs.server.namenode.FSNamesystem;
+import org.junit.Before;
+import org.junit.Test;
+
+import java.io.IOException;
+import java.util.Iterator;
+
+import static org.apache.hadoop.hdfs.protocol.HdfsConstants.EC_STORAGE_POLICY_NAME;
+import static org.junit.Assert.assertTrue;
+
+public class TestRecoverStripedBlocks {
+  private final short GROUP_SIZE =
+      HdfsConstants.NUM_DATA_BLOCKS + HdfsConstants.NUM_PARITY_BLOCKS;
+  private final short NUM_OF_DATANODES = GROUP_SIZE + 1;
+  private Configuration conf;
+  private MiniDFSCluster cluster;
+  private DistributedFileSystem fs;
+  private static final int BLOCK_SIZE = 1024;
+  private HdfsAdmin dfsAdmin;
+  private FSNamesystem namesystem;
+  private Path ECFilePath;
+
+  @Before
+  public void setupCluster() throws IOException {
+    conf = new HdfsConfiguration();
+    conf.setInt(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE);
+    // Large value to make sure the pending replication request can stay in
+    // DatanodeDescriptor.replicateBlocks before test timeout.
+    conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 100);
+    // Make sure BlockManager can pull all blocks from UnderReplicatedBlocks via
+    // chooseUnderReplicatedBlocks at once.
+    conf.setInt(
+        DFSConfigKeys.DFS_NAMENODE_REPLICATION_WORK_MULTIPLIER_PER_ITERATION, 5);
+
+    cluster = new MiniDFSCluster.Builder(conf).
+        numDataNodes(NUM_OF_DATANODES).build();
+    cluster.waitActive();
+    fs = cluster.getFileSystem();
+    dfsAdmin = new HdfsAdmin(cluster.getURI(), conf);
+    namesystem = cluster.getNamesystem();
+    ECFilePath = new Path("/ecfile");
+    DFSTestUtil.createFile(fs, ECFilePath, 4 * BLOCK_SIZE, GROUP_SIZE, 0);
+    dfsAdmin.setStoragePolicy(ECFilePath, EC_STORAGE_POLICY_NAME);
+  }
+
+  @Test
+  public void testMissingStripedBlock() throws Exception {
+    final BlockManager bm = cluster.getNamesystem().getBlockManager();
+    ExtendedBlock b = DFSTestUtil.getFirstBlock(fs, ECFilePath);
+    Iterator<DatanodeStorageInfo> storageInfos =
+        bm.blocksMap.getStorages(b.getLocalBlock())
+            .iterator();
+
+    DatanodeDescriptor firstDn = storageInfos.next().getDatanodeDescriptor();
+    Iterator<BlockInfo> it = firstDn.getBlockIterator();
+    int missingBlkCnt = 0;
+    while (it.hasNext()) {
+      BlockInfo blk = it.next();
+      BlockManager.LOG.debug("Block " + blk + " will be lost");
+      missingBlkCnt++;
+    }
+    BlockManager.LOG.debug("Missing in total " + missingBlkCnt + " blocks");
+
+    bm.getDatanodeManager().removeDatanode(firstDn);
+
+    bm.computeDatanodeWork();
+
+    short cnt = 0;
+    for (DataNode dn : cluster.getDataNodes()) {
+      DatanodeDescriptor dnDescriptor =
+          bm.getDatanodeManager().getDatanode(dn.getDatanodeUuid());
+      cnt += dnDescriptor.getNumberOfBlocksToBeErasureCoded();
+    }
+
+    assertTrue("Counting the number of outstanding EC tasks", cnt == missingBlkCnt);
+  }
+}


[35/50] [abbrv] hadoop git commit: HDFS-7969. Erasure coding: NameNode support for lease recovery of striped block groups. Contributed by Zhe Zhang.

Posted by zh...@apache.org.
HDFS-7969. Erasure coding: NameNode support for lease recovery of striped block groups. Contributed by Zhe Zhang.


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/9426f64f
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/9426f64f
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/9426f64f

Branch: refs/heads/HDFS-7285
Commit: 9426f64f3b69a3a4fd3003ee2a00180933bfae72
Parents: abbcfef
Author: Zhe Zhang <zh...@apache.org>
Authored: Mon Apr 6 12:52:44 2015 -0700
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 10:15:34 2015 -0700

----------------------------------------------------------------------
 .../BlockInfoContiguousUnderConstruction.java   | 33 ++++----
 .../BlockInfoStripedUnderConstruction.java      | 80 ++++++++++++++++----
 .../BlockInfoUnderConstruction.java             | 57 ++++++++++++++
 .../blockmanagement/DatanodeDescriptor.java     | 12 +--
 .../server/blockmanagement/DatanodeManager.java | 10 +--
 .../hdfs/server/namenode/FSNamesystem.java      | 24 +++---
 .../TestBlockInfoUnderConstruction.java         |  2 +-
 7 files changed, 163 insertions(+), 55 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/9426f64f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
index 7a052fd..9ba2978 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoContiguousUnderConstruction.java
@@ -31,7 +31,8 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode;
  * Represents a block that is currently being constructed.<br>
  * This is usually the last block of a file opened for write or append.
  */
-public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
+public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous
+    implements BlockInfoUnderConstruction{
   /** Block state. See {@link BlockUCState} */
   private BlockUCState blockUCState;
 
@@ -94,7 +95,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     return new BlockInfoContiguous(this);
   }
 
-  /** Set expected locations */
+  @Override
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ArrayList<>(numLocations);
@@ -104,10 +105,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     }
   }
 
-  /**
-   * Create array of expected replica locations
-   * (as has been assigned by chooseTargets()).
-   */
+  @Override
   public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = replicas == null ? 0 : replicas.size();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
@@ -117,7 +115,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     return storages;
   }
 
-  /** Get the number of expected locations */
+  @Override
   public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.size();
   }
@@ -135,25 +133,26 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     blockUCState = s;
   }
 
-  /** Get block recovery ID */
+  @Override
   public long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
-  /** Get recover block */
+  @Override
   public Block getTruncateBlock() {
     return truncateBlock;
   }
 
+  @Override
+  public Block toBlock(){
+    return this;
+  }
+
   public void setTruncateBlock(Block recoveryBlock) {
     this.truncateBlock = recoveryBlock;
   }
 
-  /**
-   * Process the recorded replicas. When about to commit or finish the
-   * pipeline recovery sort out bad replicas.
-   * @param genStamp  The final generation stamp for the block.
-   */
+  @Override
   public void setGenerationStampAndVerifyReplicas(long genStamp) {
     // Set the generation stamp for the block.
     setGenerationStamp(genStamp);
@@ -187,11 +186,7 @@ public class BlockInfoContiguousUnderConstruction extends BlockInfoContiguous {
     setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
   }
 
-  /**
-   * Initialize lease recovery for this block.
-   * Find the first alive data-node starting from the previous primary and
-   * make it primary.
-   */
+  @Override
   public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9426f64f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
index b1857bb..cfaf3a0 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStripedUnderConstruction.java
@@ -31,7 +31,8 @@ import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCSt
  * Represents a striped block that is currently being constructed.
  * This is usually the last block of a file opened for write or append.
  */
-public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
+public class BlockInfoStripedUnderConstruction extends BlockInfoStriped
+    implements BlockInfoUnderConstruction{
   private BlockUCState blockUCState;
 
   /**
@@ -40,6 +41,12 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   private ReplicaUnderConstruction[] replicas;
 
   /**
+   * Index of the primary data node doing the recovery. Useful for log
+   * messages.
+   */
+  private int primaryNodeIndex = -1;
+
+  /**
    * The new generation stamp, which this block will have
    * after the recovery succeeds. Also used as a recovery id to identify
    * the right recovery if any of the abandoned recoveries re-appear.
@@ -82,6 +89,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   }
 
   /** Set expected locations */
+  @Override
   public void setExpectedLocations(DatanodeStorageInfo[] targets) {
     int numLocations = targets == null ? 0 : targets.length;
     this.replicas = new ReplicaUnderConstruction[numLocations];
@@ -98,6 +106,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
    * Create array of expected replica locations
    * (as has been assigned by chooseTargets()).
    */
+  @Override
   public DatanodeStorageInfo[] getExpectedStorageLocations() {
     int numLocations = getNumExpectedLocations();
     DatanodeStorageInfo[] storages = new DatanodeStorageInfo[numLocations];
@@ -117,7 +126,7 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     return indices;
   }
 
-  /** Get the number of expected locations */
+  @Override
   public int getNumExpectedLocations() {
     return replicas == null ? 0 : replicas.length;
   }
@@ -135,16 +144,22 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     blockUCState = s;
   }
 
-  /** Get block recovery ID */
+  @Override
   public long getBlockRecoveryId() {
     return blockRecoveryId;
   }
 
-  /**
-   * Process the recorded replicas. When about to commit or finish the
-   * pipeline recovery sort out bad replicas.
-   * @param genStamp  The final generation stamp for the block.
-   */
+  @Override
+  public Block getTruncateBlock() {
+    return null;
+  }
+
+  @Override
+  public Block toBlock(){
+    return this;
+  }
+
+  @Override
   public void setGenerationStampAndVerifyReplicas(long genStamp) {
     // Set the generation stamp for the block.
     setGenerationStamp(genStamp);
@@ -178,18 +193,53 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
     setGenerationStampAndVerifyReplicas(block.getGenerationStamp());
   }
 
-  /**
-   * Initialize lease recovery for this striped block.
-   */
+  @Override
   public void initializeBlockRecovery(long recoveryId) {
     setBlockUCState(BlockUCState.UNDER_RECOVERY);
     blockRecoveryId = recoveryId;
     if (replicas == null || replicas.length == 0) {
       NameNode.blockStateChangeLog.warn("BLOCK*" +
-          " BlockInfoUnderConstruction.initLeaseRecovery:" +
+          " BlockInfoStripedUnderConstruction.initLeaseRecovery:" +
           " No blocks found, lease removed.");
     }
-    // TODO we need to implement different recovery logic here
+    boolean allLiveReplicasTriedAsPrimary = true;
+    for (ReplicaUnderConstruction replica : replicas) {
+      // Check if all replicas have been tried or not.
+      if (replica.isAlive()) {
+        allLiveReplicasTriedAsPrimary = (allLiveReplicasTriedAsPrimary &&
+            replica.getChosenAsPrimary());
+      }
+    }
+    if (allLiveReplicasTriedAsPrimary) {
+      // Just set all the replicas to be chosen whether they are alive or not.
+      for (ReplicaUnderConstruction replica : replicas) {
+        replica.setChosenAsPrimary(false);
+      }
+    }
+    long mostRecentLastUpdate = 0;
+    ReplicaUnderConstruction primary = null;
+    primaryNodeIndex = -1;
+    for(int i = 0; i < replicas.length; i++) {
+      // Skip alive replicas which have been chosen for recovery.
+      if (!(replicas[i].isAlive() && !replicas[i].getChosenAsPrimary())) {
+        continue;
+      }
+      final ReplicaUnderConstruction ruc = replicas[i];
+      final long lastUpdate = ruc.getExpectedStorageLocation()
+          .getDatanodeDescriptor().getLastUpdateMonotonic();
+      if (lastUpdate > mostRecentLastUpdate) {
+        primaryNodeIndex = i;
+        primary = ruc;
+        mostRecentLastUpdate = lastUpdate;
+      }
+    }
+    if (primary != null) {
+      primary.getExpectedStorageLocation().getDatanodeDescriptor()
+          .addBlockToBeRecovered(this);
+      primary.setChosenAsPrimary(true);
+      NameNode.blockStateChangeLog.info(
+          "BLOCK* {} recovery started, primary={}", this, primary);
+    }
   }
 
   void addReplicaIfNotPresent(DatanodeStorageInfo storage, Block reportedBlock,
@@ -238,7 +288,9 @@ public class BlockInfoStripedUnderConstruction extends BlockInfoStriped {
   }
 
   private void appendUCParts(StringBuilder sb) {
-    sb.append("{UCState=").append(blockUCState).append(", replicas=[");
+    sb.append("{UCState=").append(blockUCState).
+        append(", primaryNodeIndex=").append(primaryNodeIndex).
+        append(", replicas=[");
     if (replicas != null) {
       int i = 0;
       for (ReplicaUnderConstruction r : replicas) {

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9426f64f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
new file mode 100644
index 0000000..bfdd386
--- /dev/null
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoUnderConstruction.java
@@ -0,0 +1,57 @@
+/**
+ * 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.server.blockmanagement;
+
+import org.apache.hadoop.hdfs.protocol.Block;
+
+public interface BlockInfoUnderConstruction {
+  /**
+   * Create array of expected replica locations
+   * (as has been assigned by chooseTargets()).
+   */
+  public DatanodeStorageInfo[] getExpectedStorageLocations();
+
+  /** Get recover block */
+  public Block getTruncateBlock();
+
+  /** Convert to a Block object */
+  public Block toBlock();
+
+  /** Get block recovery ID */
+  public long getBlockRecoveryId();
+
+  /** Get the number of expected locations */
+  public int getNumExpectedLocations();
+
+  /** Set expected locations */
+  public void setExpectedLocations(DatanodeStorageInfo[] targets);
+
+  /**
+   * Process the recorded replicas. When about to commit or finish the
+   * pipeline recovery sort out bad replicas.
+   * @param genStamp  The final generation stamp for the block.
+   */
+  public void setGenerationStampAndVerifyReplicas(long genStamp);
+
+  /**
+   * Initialize lease recovery for this block.
+   * Find the first alive data-node starting from the previous primary and
+   * make it primary.
+   */
+  public void initializeBlockRecovery(long recoveryId);
+}

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9426f64f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
index 15427f7..7ec71a2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeDescriptor.java
@@ -253,8 +253,8 @@ public class DatanodeDescriptor extends DatanodeInfo {
   private final BlockQueue<BlockECRecoveryInfo> erasurecodeBlocks =
       new BlockQueue<>();
   /** A queue of blocks to be recovered by this datanode */
-  private final BlockQueue<BlockInfoContiguousUnderConstruction>
-      recoverBlocks = new BlockQueue<>();
+  private final BlockQueue<BlockInfoUnderConstruction> recoverBlocks =
+      new BlockQueue<>();
   /** A set of blocks to be invalidated by this datanode */
   private final LightWeightHashSet<Block> invalidateBlocks =
       new LightWeightHashSet<>();
@@ -649,7 +649,7 @@ public class DatanodeDescriptor extends DatanodeInfo {
   /**
    * Store block recovery work.
    */
-  void addBlockToBeRecovered(BlockInfoContiguousUnderConstruction block) {
+  void addBlockToBeRecovered(BlockInfoUnderConstruction block) {
     if(recoverBlocks.contains(block)) {
       // this prevents adding the same block twice to the recovery queue
       BlockManager.LOG.info(block + " is already in the recovery queue");
@@ -703,11 +703,11 @@ public class DatanodeDescriptor extends DatanodeInfo {
     return erasurecodeBlocks.poll(maxTransfers);
   }
 
-  public BlockInfoContiguousUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
-    List<BlockInfoContiguousUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
+  public BlockInfoUnderConstruction[] getLeaseRecoveryCommand(int maxTransfers) {
+    List<BlockInfoUnderConstruction> blocks = recoverBlocks.poll(maxTransfers);
     if(blocks == null)
       return null;
-    return blocks.toArray(new BlockInfoContiguousUnderConstruction[blocks.size()]);
+    return blocks.toArray(new BlockInfoUnderConstruction[blocks.size()]);
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9426f64f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
index 6228f86..36fcae2 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/DatanodeManager.java
@@ -1374,12 +1374,12 @@ public class DatanodeManager {
         }
 
         //check lease recovery
-        BlockInfoContiguousUnderConstruction[] blocks = nodeinfo
+        BlockInfoUnderConstruction[] blocks = nodeinfo
             .getLeaseRecoveryCommand(Integer.MAX_VALUE);
         if (blocks != null) {
           BlockRecoveryCommand brCommand = new BlockRecoveryCommand(
               blocks.length);
-          for (BlockInfoContiguousUnderConstruction b : blocks) {
+          for (BlockInfoUnderConstruction b : blocks) {
             final DatanodeStorageInfo[] storages = b.getExpectedStorageLocations();
             // Skip stale nodes during recovery - not heart beated for some time (30s by default).
             final List<DatanodeStorageInfo> recoveryLocations =
@@ -1393,10 +1393,10 @@ public class DatanodeManager {
             // to old block.
             boolean truncateRecovery = b.getTruncateBlock() != null;
             boolean copyOnTruncateRecovery = truncateRecovery &&
-                b.getTruncateBlock().getBlockId() != b.getBlockId();
+                b.getTruncateBlock().getBlockId() != b.toBlock().getBlockId();
             ExtendedBlock primaryBlock = (copyOnTruncateRecovery) ?
                 new ExtendedBlock(blockPoolId, b.getTruncateBlock()) :
-                new ExtendedBlock(blockPoolId, b);
+                new ExtendedBlock(blockPoolId, b.toBlock());
             // If we only get 1 replica after eliminating stale nodes, then choose all
             // replicas for recovery and let the primary data node handle failures.
             DatanodeInfo[] recoveryInfos;
@@ -1413,7 +1413,7 @@ public class DatanodeManager {
               recoveryInfos = DatanodeStorageInfo.toDatanodeInfos(storages);
             }
             if(truncateRecovery) {
-              Block recoveryBlock = (copyOnTruncateRecovery) ? b :
+              Block recoveryBlock = (copyOnTruncateRecovery) ? b.toBlock() :
                   b.getTruncateBlock();
               brCommand.add(new RecoveringBlock(primaryBlock, recoveryInfos,
                                                 recoveryBlock));

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9426f64f/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
index 7c087d0..79d6f21 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSNamesystem.java
@@ -208,6 +208,7 @@ import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguousUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStripedUnderConstruction;
+import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoUnderConstruction;
 import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor;
 import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager;
@@ -4097,18 +4098,19 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
       throw new AlreadyBeingCreatedException(message);
     case UNDER_CONSTRUCTION:
     case UNDER_RECOVERY:
-      // TODO support Striped block's recovery
-      final BlockInfoContiguousUnderConstruction uc =
-          (BlockInfoContiguousUnderConstruction)lastBlock;
+      // TODO support truncate of striped blocks
+      final BlockInfoUnderConstruction uc =
+          (BlockInfoUnderConstruction)lastBlock;
       // determine if last block was intended to be truncated
       Block recoveryBlock = uc.getTruncateBlock();
       boolean truncateRecovery = recoveryBlock != null;
       boolean copyOnTruncate = truncateRecovery &&
-          recoveryBlock.getBlockId() != uc.getBlockId();
+          recoveryBlock.getBlockId() != uc.toBlock().getBlockId();
       assert !copyOnTruncate ||
-          recoveryBlock.getBlockId() < uc.getBlockId() &&
-          recoveryBlock.getGenerationStamp() < uc.getGenerationStamp() &&
-          recoveryBlock.getNumBytes() > uc.getNumBytes() :
+          recoveryBlock.getBlockId() < uc.toBlock().getBlockId() &&
+          recoveryBlock.getGenerationStamp() < uc.toBlock().
+              getGenerationStamp() &&
+          recoveryBlock.getNumBytes() > uc.toBlock().getNumBytes() :
             "wrong recoveryBlock";
 
       // setup the last block locations from the blockManager if not known
@@ -4116,7 +4118,8 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         uc.setExpectedLocations(blockManager.getStorages(lastBlock));
       }
 
-      if (uc.getNumExpectedLocations() == 0 && uc.getNumBytes() == 0) {
+      if (uc.getNumExpectedLocations() == 0 &&
+          uc.toBlock().getNumBytes() == 0) {
         // There is no datanode reported to this block.
         // may be client have crashed before writing data to pipeline.
         // This blocks doesn't need any recovery.
@@ -4129,10 +4132,11 @@ public class FSNamesystem implements Namesystem, FSNamesystemMBean,
         return true;
       }
       // start recovery of the last block for this file
-      long blockRecoveryId = nextGenerationStamp(blockIdManager.isLegacyBlock(uc));
+      long blockRecoveryId =
+          nextGenerationStamp(blockIdManager.isLegacyBlock(uc.toBlock()));
       lease = reassignLease(lease, src, recoveryLeaseHolder, pendingFile);
       if(copyOnTruncate) {
-        uc.setGenerationStamp(blockRecoveryId);
+        uc.toBlock().setGenerationStamp(blockRecoveryId);
       } else if(truncateRecovery) {
         recoveryBlock.setGenerationStamp(blockRecoveryId);
       }

http://git-wip-us.apache.org/repos/asf/hadoop/blob/9426f64f/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
----------------------------------------------------------------------
diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
index a7ba293..f5a9cc4 100644
--- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
+++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoUnderConstruction.java
@@ -51,7 +51,7 @@ public class TestBlockInfoUnderConstruction {
     DFSTestUtil.resetLastUpdatesWithOffset(dd2, -1 * 1000);
     DFSTestUtil.resetLastUpdatesWithOffset(dd3, -2 * 1000);
     blockInfo.initializeBlockRecovery(1);
-    BlockInfoContiguousUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
+    BlockInfoUnderConstruction[] blockInfoRecovery = dd2.getLeaseRecoveryCommand(1);
     assertEquals(blockInfoRecovery[0], blockInfo);
 
     // Recovery attempt #2.


[10/50] [abbrv] hadoop git commit: HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng

Posted by zh...@apache.org.
HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng


Project: http://git-wip-us.apache.org/repos/asf/hadoop/repo
Commit: http://git-wip-us.apache.org/repos/asf/hadoop/commit/d984ffbf
Tree: http://git-wip-us.apache.org/repos/asf/hadoop/tree/d984ffbf
Diff: http://git-wip-us.apache.org/repos/asf/hadoop/diff/d984ffbf

Branch: refs/heads/HDFS-7285
Commit: d984ffbf1514cb4f2a0f9fd97b0adcb33eced18a
Parents: 94a1ea4
Author: drankye <ka...@intel.com>
Authored: Thu Mar 12 23:35:22 2015 +0800
Committer: Zhe Zhang <zh...@apache.org>
Committed: Mon Apr 13 09:41:58 2015 -0700

----------------------------------------------------------------------
 .../hadoop-common/CHANGES-HDFS-EC-7285.txt      |  4 +++
 .../erasurecode/coder/AbstractErasureCoder.java |  5 ++-
 .../rawcoder/AbstractRawErasureCoder.java       |  5 ++-
 .../hadoop/io/erasurecode/TestCoderBase.java    |  6 ++++
 .../erasurecode/coder/TestErasureCoderBase.java | 36 +++++++++++++++++---
 .../erasurecode/rawcoder/TestRawCoderBase.java  | 13 +++++--
 6 files changed, 60 insertions(+), 9 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/hadoop/blob/d984ffbf/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
index c17a1bd..a97dc34 100644
--- a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
+++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt
@@ -18,3 +18,7 @@
     HADOOP-11646. Erasure Coder API for encoding and decoding of block group
     ( Kai Zheng via vinayakumarb )
 
+    HADOOP-11705. Make erasure coder configurable. Contributed by Kai Zheng
+    ( Kai Zheng )    
+
+

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d984ffbf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
index f2cc041..8d3bc34 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.java
@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configured;
+
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
  * It implements the {@link ErasureCoder} interface.
  */
-public abstract class AbstractErasureCoder implements ErasureCoder {
+public abstract class AbstractErasureCoder
+    extends Configured implements ErasureCoder {
 
   private int numDataUnits;
   private int numParityUnits;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d984ffbf/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
index 74d2ab6..e6f3d92 100644
--- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
+++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java
@@ -17,12 +17,15 @@
  */
 package org.apache.hadoop.io.erasurecode.rawcoder;
 
+import org.apache.hadoop.conf.Configured;
+
 /**
  * A common class of basic facilities to be shared by encoder and decoder
  *
  * It implements the {@link RawErasureCoder} interface.
  */
-public abstract class AbstractRawErasureCoder implements RawErasureCoder {
+public abstract class AbstractRawErasureCoder
+    extends Configured implements RawErasureCoder {
 
   private int numDataUnits;
   private int numParityUnits;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d984ffbf/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
index 3c4288c..194413a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java
@@ -43,6 +43,12 @@ public abstract class TestCoderBase {
   // may go to different coding implementations.
   protected boolean usingDirectBuffer = true;
 
+  /**
+   * Prepare before running the case.
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedIndexes
+   */
   protected void prepare(int numDataUnits, int numParityUnits,
                          int[] erasedIndexes) {
     this.numDataUnits = numDataUnits;

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d984ffbf/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
index ca5c1c9..36e061a 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java
@@ -17,6 +17,7 @@
  */
 package org.apache.hadoop.io.erasurecode.coder;
 
+import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.io.erasurecode.ECBlock;
 import org.apache.hadoop.io.erasurecode.ECChunk;
 import org.apache.hadoop.io.erasurecode.ECBlockGroup;
@@ -29,6 +30,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   protected Class<? extends ErasureEncoder> encoderClass;
   protected Class<? extends ErasureDecoder> decoderClass;
 
+  private Configuration conf;
   protected int numChunksInBlock = 16;
 
   /**
@@ -46,6 +48,19 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
   }
 
   /**
+   * Prepare before running the case.
+   * @param conf
+   * @param numDataUnits
+   * @param numParityUnits
+   * @param erasedIndexes
+   */
+  protected void prepare(Configuration conf, int numDataUnits,
+                         int numParityUnits, int[] erasedIndexes) {
+    this.conf = conf;
+    super.prepare(numDataUnits, numParityUnits, erasedIndexes);
+  }
+
+  /**
    * Generating source data, encoding, recovering and then verifying.
    * RawErasureCoder mainly uses ECChunk to pass input and output data buffers,
    * it supports two kinds of ByteBuffers, one is array backed, the other is
@@ -56,6 +71,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     this.usingDirectBuffer = usingDirectBuffer;
 
     ErasureEncoder encoder = createEncoder();
+
     // Generate data and encode
     ECBlockGroup blockGroup = prepareBlockGroupForEncoding();
     // Backup all the source chunks for later recovering because some coders
@@ -65,17 +81,25 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     // Make a copy of a strip for later comparing
     TestBlock[] toEraseBlocks = copyDataBlocksToErase(clonedDataBlocks);
 
-    ErasureCodingStep codingStep = encoder.encode(blockGroup);
-    performCodingStep(codingStep);
+    ErasureCodingStep codingStep;
+    try {
+      codingStep = encoder.encode(blockGroup);
+      performCodingStep(codingStep);
+    } finally {
+      encoder.release();
+    }
     // Erase the copied sources
     eraseSomeDataBlocks(clonedDataBlocks);
 
     //Decode
     blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks());
     ErasureDecoder decoder = createDecoder();
-    codingStep = decoder.decode(blockGroup);
-    performCodingStep(codingStep);
-
+    try {
+      codingStep = decoder.decode(blockGroup);
+      performCodingStep(codingStep);
+    } finally {
+      decoder.release();
+    }
     //Compare
     compareAndVerify(toEraseBlocks, codingStep.getOutputBlocks());
   }
@@ -138,6 +162,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     encoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    encoder.setConf(conf);
     return encoder;
   }
 
@@ -154,6 +179,7 @@ public abstract class TestErasureCoderBase extends TestCoderBase {
     }
 
     decoder.initialize(numDataUnits, numParityUnits, chunkSize);
+    decoder.setConf(conf);
     return decoder;
   }
 

http://git-wip-us.apache.org/repos/asf/hadoop/blob/d984ffbf/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
----------------------------------------------------------------------
diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
index 5f6ccda..890f632 100644
--- a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
+++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java
@@ -49,7 +49,11 @@ public abstract class TestRawCoderBase extends TestCoderBase {
     // Make a copy of a strip for later comparing
     ECChunk[] toEraseDataChunks = copyDataChunksToErase(clonedDataChunks);
 
-    encoder.encode(dataChunks, parityChunks);
+    try {
+      encoder.encode(dataChunks, parityChunks);
+    } finally {
+      encoder.release();
+    }
     // Erase the copied sources
     eraseSomeDataBlocks(clonedDataChunks);
 
@@ -58,7 +62,12 @@ public abstract class TestRawCoderBase extends TestCoderBase {
         parityChunks);
     ECChunk[] recoveredChunks = prepareOutputChunksForDecoding();
     RawErasureDecoder decoder = createDecoder();
-    decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks);
+    try {
+      decoder.decode(inputChunks,
+          getErasedIndexesForDecoding(), recoveredChunks);
+    } finally {
+      decoder.release();
+    }
 
     //Compare
     compareAndVerify(toEraseDataChunks, recoveredChunks);