diff --git a/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt new file mode 100644 index 0000000000000..1f3006e14ba9c --- /dev/null +++ b/hadoop-common-project/hadoop-common/CHANGES-HDFS-EC-7285.txt @@ -0,0 +1,74 @@ + BREAKDOWN OF HADOOP-11264 SUBTASKS AND RELATED JIRAS (Common part of HDFS-7285) + + HADOOP-11514. Raw Erasure Coder API for concrete encoding and decoding + (Kai Zheng via umamahesh) + + HADOOP-11534. Minor improvements for raw erasure coders + ( Kai Zheng via vinayakumarb ) + + HADOOP-11541. Raw XOR coder + ( Kai Zheng ) + + 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 ) + + 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 ) + + 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 ) + + 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 ) + + 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 ) + + 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) + + HADOOP-11841. Remove unused ecschema-def.xml files. (szetszwo) + + HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng via Zhe Zhang) + + HADOOP-11920. Refactor some codes for erasure coders. (Kai Zheng via Zhe Zhang) + + HADOOP-11566. Add tests and fix for erasure coders to recover erased parity + units. (Kai Zheng via Zhe Zhang) + + HADOOP-11938. Enhance ByteBuffer version encode/decode API of raw erasure + coder. (Kai Zheng via Zhe Zhang) + + HADOOP-12013. Generate fixed data to perform erasure coder test. (Kai Zheng) + + HADOOP-12029. Remove chunkSize from ECSchema as its not required for coders + (vinayakumarb) + + HADOOP-11847. Enhance raw coder allowing to read least required inputs in decoding. + (Kai Zheng) + + HADOOP-12011. Allow to dump verbose information to ease debugging in raw erasure coders + (Kai Zheng) + + HADOOP-12065. Using more meaningful keys in EC schema. (Kai Zheng) + + HDFS-8557. Allow to configure RS and XOR raw coders (Kai Zheng) + + HADOOP-12060. Fix ByteBuffer usage for raw erasure coders. (Kai Zheng via + jing9) 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 09d255046c6f9..ea517df8d595f 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 @@ -136,6 +136,21 @@ public class CommonConfigurationKeys extends CommonConfigurationKeysPublic { public static final boolean IO_COMPRESSION_CODEC_LZ4_USELZ4HC_DEFAULT = false; + /** + * Erasure Coding configuration family + */ + + /** Supported erasure codec classes */ + public static final String IO_ERASURECODE_CODECS_KEY = "io.erasurecode.codecs"; + + /** Raw coder factory for the RS codec. */ + public static final String IO_ERASURECODE_CODEC_RS_RAWCODER_KEY = + "io.erasurecode.codec.rs.rawcoder"; + + /** Raw coder factory for the XOR codec. */ + public static final String IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY = + "io.erasurecode.codec.xor.rawcoder"; + /** * Service Authorization */ diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java index 648043ef89e97..616ff3a8392ff 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FSOutputSummer.java @@ -198,6 +198,10 @@ protected int getChecksumSize() { return sum.getChecksumSize(); } + protected DataChecksum getDataChecksum() { + return sum; + } + protected TraceScope createWriteTraceScope() { return null; } diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java index d91866ed7a46e..334b6bcca15e4 100644 --- a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/fs/FsShell.java @@ -121,6 +121,10 @@ public Path getCurrentTrashDir() throws IOException { return getTrash().getCurrentTrashDir(); } + protected String getUsagePrefix() { + return usagePrefix; + } + // NOTE: Usage/Help are inner classes to allow access to outer methods // that access commandFactory @@ -204,7 +208,7 @@ private void printInfo(PrintStream out, String cmd, boolean showHelp) { } } else { // display help or usage for all commands - out.println(usagePrefix); + out.println(getUsagePrefix()); // display list of short usages ArrayList instances = new ArrayList(); @@ -228,7 +232,7 @@ private void printInfo(PrintStream out, String cmd, boolean showHelp) { } private void printInstanceUsage(PrintStream out, Command instance) { - out.println(usagePrefix + " " + instance.getUsage()); + out.println(getUsagePrefix() + " " + instance.getUsage()); } private void printInstanceHelp(PrintStream out, Command instance) { diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java new file mode 100644 index 0000000000000..027d58b0260f6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/CodecUtil.java @@ -0,0 +1,144 @@ +/** + * 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.apache.hadoop.io.erasurecode.rawcoder.*; + +/** + * A codec & coder utility to help create raw coders conveniently. + */ +public final class CodecUtil { + + private CodecUtil() { } + + /** + * Create RS raw encoder according to configuration. + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group + * @return raw encoder + */ + public static RawErasureEncoder createRSRawEncoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + true, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new RSRawEncoder(numDataUnits, numParityUnits); + } + + return (RawErasureEncoder) rawCoder; + } + + /** + * Create RS raw decoder according to configuration. + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group + * @return raw decoder + */ + public static RawErasureDecoder createRSRawDecoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + false, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new RSRawDecoder(numDataUnits, numParityUnits); + } + + return (RawErasureDecoder) rawCoder; + } + + /** + * Create XOR raw encoder according to configuration. + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group + * @return raw encoder + */ + public static RawErasureEncoder createXORRawEncoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY, + true, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new XORRawEncoder(numDataUnits, numParityUnits); + } + + return (RawErasureEncoder) rawCoder; + } + + /** + * Create XOR raw decoder according to configuration. + * @param conf configuration possibly with some items to configure the coder + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group + * @return raw decoder + */ + public static RawErasureDecoder createXORRawDecoder( + Configuration conf, int numDataUnits, int numParityUnits) { + RawErasureCoder rawCoder = createRawCoder(conf, + CommonConfigurationKeys.IO_ERASURECODE_CODEC_XOR_RAWCODER_KEY, + false, numDataUnits, numParityUnits); + if (rawCoder == null) { + rawCoder = new XORRawDecoder(numDataUnits, numParityUnits); + } + + return (RawErasureDecoder) rawCoder; + } + + /** + * Create raw coder using specified conf and raw coder factory key. + * @param conf configuration possibly with some items to configure the coder + * @param rawCoderFactoryKey configuration key to find the raw coder factory + * @param isEncoder is encoder or not we're going to create + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group + * @return raw coder + */ + public static RawErasureCoder createRawCoder(Configuration conf, + String rawCoderFactoryKey, boolean isEncoder, int numDataUnits, + int numParityUnits) { + + if (conf == null) { + return null; + } + + Class 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); + } + + return isEncoder ? fact.createEncoder(numDataUnits, numParityUnits) : + fact.createDecoder(numDataUnits, numParityUnits); + } +} 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 0000000000000..5c0a16006f867 --- /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 is a parity block + * @param isErased is erased or not + */ + public ECBlock(boolean isParity, boolean isErased) { + this.isParity = isParity; + this.isErased = isErased; + } + + /** + * Set true if it's for a parity block. + * @param isParity is parity or not + */ + public void setParity(boolean isParity) { + this.isParity = isParity; + } + + /** + * Set true if the block is missing. + * @param isErased is erased or not + */ + public void setErased(boolean isErased) { + this.isErased = isErased; + } + + /** + * + * @return true if it's parity block, otherwise false + */ + public boolean isParity() { + return isParity; + } + + /** + * + * @return true if it's erased due to erasure, otherwise false + */ + public boolean isErased() { + return isErased; + } + +} 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 0000000000000..91e4fb8f51731 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECBlockGroup.java @@ -0,0 +1,100 @@ +/** + * 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 data blocks in the group + * @param parityBlocks parity blocks in the group + */ + 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; + } + + /** + * Get erased blocks count + * @return erased count of blocks + */ + public int getErasedCount() { + int erasedCount = 0; + + for (ECBlock dataBlock : dataBlocks) { + if (dataBlock.isErased()) erasedCount++; + } + + for (ECBlock parityBlock : parityBlocks) { + if (parityBlock.isErased()) erasedCount++; + } + + return erasedCount; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java new file mode 100644 index 0000000000000..d0120d849c457 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECChunk.java @@ -0,0 +1,87 @@ +/** + * 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.nio.ByteBuffer; + +/** + * A wrapper for ByteBuffer or bytes array for an erasure code chunk. + */ +public class ECChunk { + + private ByteBuffer chunkBuffer; + + /** + * Wrapping a ByteBuffer + * @param buffer buffer to be wrapped by the chunk + */ + public ECChunk(ByteBuffer buffer) { + this.chunkBuffer = buffer; + } + + /** + * Wrapping a bytes array + * @param buffer buffer to be wrapped by the chunk + */ + public ECChunk(byte[] buffer) { + this.chunkBuffer = ByteBuffer.wrap(buffer); + } + + /** + * Convert to ByteBuffer + * @return ByteBuffer + */ + public ByteBuffer getBuffer() { + return chunkBuffer; + } + + /** + * Convert an array of this chunks to an array of ByteBuffers + * @param chunks chunks to convert into buffers + * @return an array of ByteBuffers + */ + public static ByteBuffer[] toBuffers(ECChunk[] chunks) { + ByteBuffer[] buffers = new ByteBuffer[chunks.length]; + + ECChunk chunk; + for (int i = 0; i < chunks.length; i++) { + chunk = chunks[i]; + if (chunk == null) { + buffers[i] = null; + } else { + buffers[i] = chunk.getBuffer(); + } + } + + return buffers; + } + + /** + * Convert to a bytes array, just for test usage. + * @return bytes array + */ + public byte[] toBytesArray() { + byte[] bytesArr = new byte[chunkBuffer.remaining()]; + // Avoid affecting the original one + chunkBuffer.mark(); + chunkBuffer.get(bytesArr); + chunkBuffer.reset(); + + return bytesArr; + } +} 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 0000000000000..0d5bf8f0688a4 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/ECSchema.java @@ -0,0 +1,227 @@ +/** + * 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.HashMap; +import java.util.Map; + +/** + * Erasure coding schema to housekeeper relevant information. + */ +public final class ECSchema { + public static final String NUM_DATA_UNITS_KEY = "numDataUnits"; + public static final String NUM_PARITY_UNITS_KEY = "numParityUnits"; + public static final String CODEC_NAME_KEY = "codec"; + + /** + * The erasure codec name associated. + */ + private final String codecName; + + /** + * Number of source data units coded + */ + private final int numDataUnits; + + /** + * Number of parity units generated in a coding + */ + private final int numParityUnits; + + /* + * An erasure code can have its own specific advanced parameters, subject to + * itself to interpret these key-value settings. + */ + private final Map extraOptions; + + /** + * Constructor with schema name and provided all options. Note the options may + * contain additional information for the erasure codec to interpret further. + * @param allOptions all schema options + */ + public ECSchema(Map allOptions) { + if (allOptions == null || allOptions.isEmpty()) { + throw new IllegalArgumentException("No schema options are provided"); + } + + this.codecName = allOptions.get(CODEC_NAME_KEY); + if (codecName == null || codecName.isEmpty()) { + throw new IllegalArgumentException("No codec option is provided"); + } + + int tmpNumDataUnits = extractIntOption(NUM_DATA_UNITS_KEY, allOptions); + int tmpNumParityUnits = extractIntOption(NUM_PARITY_UNITS_KEY, allOptions); + if (tmpNumDataUnits < 0 || tmpNumParityUnits < 0) { + throw new IllegalArgumentException( + "No good option for numDataUnits or numParityUnits found "); + } + this.numDataUnits = tmpNumDataUnits; + this.numParityUnits = tmpNumParityUnits; + + allOptions.remove(CODEC_NAME_KEY); + allOptions.remove(NUM_DATA_UNITS_KEY); + allOptions.remove(NUM_PARITY_UNITS_KEY); + // After some cleanup + this.extraOptions = Collections.unmodifiableMap(allOptions); + } + + /** + * Constructor with key parameters provided. + * @param codecName codec name + * @param numDataUnits number of data units used in the schema + * @param numParityUnits number os parity units used in the schema + */ + public ECSchema(String codecName, int numDataUnits, int numParityUnits) { + this(codecName, numDataUnits, numParityUnits, null); + } + + /** + * Constructor with key parameters provided. Note the extraOptions may contain + * additional information for the erasure codec to interpret further. + * @param codecName codec name + * @param numDataUnits number of data units used in the schema + * @param numParityUnits number os parity units used in the schema + * @param extraOptions extra options to configure the codec + */ + public ECSchema(String codecName, int numDataUnits, int numParityUnits, + Map extraOptions) { + assert (codecName != null && ! codecName.isEmpty()); + assert (numDataUnits > 0 && numParityUnits > 0); + + this.codecName = codecName; + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; + + if (extraOptions == null) { + extraOptions = new HashMap<>(); + } + + // After some cleanup + this.extraOptions = Collections.unmodifiableMap(extraOptions); + } + + private int extractIntOption(String optionKey, Map options) { + int result = -1; + + try { + if (options.containsKey(optionKey)) { + result = Integer.parseInt(options.get(optionKey)); + if (result <= 0) { + throw new IllegalArgumentException("Bad option value " + result + + " found for " + optionKey); + } + } + } catch (NumberFormatException e) { + throw new IllegalArgumentException("Option value " + + options.get(optionKey) + " for " + optionKey + + " is found. It should be an integer"); + } + + return result; + } + + /** + * Get the codec name + * @return codec name + */ + public String getCodecName() { + return codecName; + } + + /** + * Get extra options specific to a erasure code. + * @return extra options + */ + public Map getExtraOptions() { + return extraOptions; + } + + /** + * 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; + } + + /** + * Make a meaningful string representation for log output. + * @return string representation + */ + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ECSchema=["); + + sb.append("Codec=" + codecName + ", "); + sb.append(NUM_DATA_UNITS_KEY + "=" + numDataUnits + ", "); + sb.append(NUM_PARITY_UNITS_KEY + "=" + numParityUnits); + sb.append((extraOptions.isEmpty() ? "" : ", ")); + + int i = 0; + for (String opt : extraOptions.keySet()) { + sb.append(opt + "=" + extraOptions.get(opt) + + (++i < extraOptions.size() ? ", " : "")); + } + + sb.append("]"); + + 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 (!codecName.equals(ecSchema.codecName)) { + return false; + } + return extraOptions.equals(ecSchema.extraOptions); + } + + @Override + public int hashCode() { + int result = codecName.hashCode(); + result = 31 * result + extraOptions.hashCode(); + result = 31 * result + numDataUnits; + result = 31 * result + numParityUnits; + + return result; + } +} 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 0000000000000..0cacfbcacd626 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/AbstractErasureCodec.java @@ -0,0 +1,51 @@ +/** + * 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.grouper.BlockGrouper; + +/** + * Abstract Erasure Codec that implements {@link ErasureCodec}. + */ +public abstract class AbstractErasureCodec extends Configured + implements ErasureCodec { + + private final ECSchema schema; + + public AbstractErasureCodec(ECSchema schema) { + this.schema = schema; + } + + public String getName() { + return schema.getCodecName(); + } + + public ECSchema getSchema() { + return schema; + } + + @Override + public BlockGrouper createBlockGrouper() { + BlockGrouper blockGrouper = new BlockGrouper(); + blockGrouper.setSchema(getSchema()); + + return blockGrouper; + } +} 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 0000000000000..9aa3db265e30f --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/ErasureCodec.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.codec; + +import org.apache.hadoop.conf.Configurable; +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 { + + /** + * 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(); + +} 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 0000000000000..6edd638dc5406 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/RSErasureCodec.java @@ -0,0 +1,43 @@ +/** + * 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.RSErasureDecoder; +import org.apache.hadoop.io.erasurecode.coder.RSErasureEncoder; + +/** + * A Reed-Solomon erasure codec. + */ +public class RSErasureCodec extends AbstractErasureCodec { + + public RSErasureCodec(ECSchema schema) { + super(schema); + } + + @Override + public ErasureCoder createEncoder() { + return new RSErasureEncoder(getSchema()); + } + + @Override + public ErasureCoder createDecoder() { + return new RSErasureDecoder(getSchema()); + } +} 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 0000000000000..e2dcfa7983c8a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/codec/XORErasureCodec.java @@ -0,0 +1,44 @@ +/** + * 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 { + + public XORErasureCodec(ECSchema schema) { + super(schema); + assert(schema.getNumParityUnits() == 1); + } + + @Override + public ErasureCoder createEncoder() { + return new XORErasureEncoder(getSchema()); + } + + @Override + public ErasureCoder createDecoder() { + return new XORErasureDecoder(getSchema()); + } +} 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 0000000000000..5cd0ee8c18300 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureCoder.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.io.erasurecode.coder; + +import org.apache.hadoop.conf.Configured; +import org.apache.hadoop.io.erasurecode.ECSchema; + +/** + * A common class of basic facilities to be shared by encoder and decoder + * + * It implements the {@link ErasureCoder} interface. + */ +public abstract class AbstractErasureCoder + extends Configured implements ErasureCoder { + + private final int numDataUnits; + private final int numParityUnits; + + public AbstractErasureCoder(int numDataUnits, int numParityUnits) { + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; + } + + public AbstractErasureCoder(ECSchema schema) { + this(schema.getNumDataUnits(), schema.getNumParityUnits()); + } + + @Override + public int getNumDataUnits() { + return numDataUnits; + } + + @Override + public int getNumParityUnits() { + return numParityUnits; + } + + @Override + public boolean preferDirectBuffer() { + return false; + } + + @Override + public void release() { + // Nothing to do by default + } +} 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 0000000000000..c429d49a36c1f --- /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 + } + +} 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 0000000000000..3ea9311d194ad --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureDecoder.java @@ -0,0 +1,167 @@ +/** + * 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.ECSchema; + +/** + * An abstract erasure decoder that's to be inherited by new decoders. + * + * It implements the {@link ErasureCoder} interface. + */ +public abstract class AbstractErasureDecoder extends AbstractErasureCoder { + + public AbstractErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public AbstractErasureDecoder(ECSchema schema) { + super(schema); + } + + @Override + public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) { + // We may have more than this when considering complicate cases. HADOOP-11550 + return prepareDecodingStep(blockGroup); + } + + /** + * Perform decoding against a block blockGroup. + * @param blockGroup + * @return decoding step for caller to do the real work + */ + protected abstract ErasureCodingStep prepareDecodingStep( + 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 ? + * @param blockGroup + * @return output blocks to recover + */ + protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) { + ECBlock[] outputBlocks = new ECBlock[getNumErasedBlocks(blockGroup)]; + + 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; + } + + /** + * 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; + } + +} 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 0000000000000..7c887e8e2cc8d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/AbstractErasureEncoder.java @@ -0,0 +1,60 @@ +/** + * 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.ECSchema; + +/** + * An abstract erasure encoder that's to be inherited by new encoders. + * + * It implements the {@link ErasureCoder} interface. + */ +public abstract class AbstractErasureEncoder extends AbstractErasureCoder { + + public AbstractErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public AbstractErasureEncoder(ECSchema schema) { + super(schema); + } + + @Override + public ErasureCodingStep calculateCoding(ECBlockGroup blockGroup) { + // We may have more than this when considering complicate cases. HADOOP-11550 + return prepareEncodingStep(blockGroup); + } + + /** + * Perform encoding against a block group. + * @param blockGroup + * @return encoding step for caller to do the real work + */ + protected abstract ErasureCodingStep prepareEncodingStep( + ECBlockGroup blockGroup); + + protected ECBlock[] getInputBlocks(ECBlockGroup blockGroup) { + return blockGroup.getDataBlocks(); + } + + protected ECBlock[] getOutputBlocks(ECBlockGroup blockGroup) { + return blockGroup.getParityBlocks(); + } +} 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 0000000000000..f05ea417463f6 --- /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; + +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 + * 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 extends Configurable { + + /** + * 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(); + + /** + * 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 direct 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 direct buffer is preferred for performance consideration, + * otherwise false. + */ + public boolean preferDirectBuffer(); + + /** + * Release the resources if any. Good chance to invoke RawErasureCoder#release. + */ + public void release(); +} 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 0000000000000..a3b177f2d2f46 --- /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(); +} 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 0000000000000..980c5805de1c6 --- /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); + } + +} 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 0000000000000..bd7587fa3c2dc --- /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); + } + +} 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 0000000000000..f56674d32c240 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureDecoder.java @@ -0,0 +1,67 @@ +/** + * 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.CodecUtil; +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; + +/** + * Reed-Solomon erasure decoder that decodes a block group. + * + * It implements {@link ErasureCoder}. + */ +public class RSErasureDecoder extends AbstractErasureDecoder { + private RawErasureDecoder rsRawDecoder; + + public RSErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public RSErasureDecoder(ECSchema schema) { + super(schema); + } + + @Override + protected ErasureCodingStep prepareDecodingStep(final ECBlockGroup blockGroup) { + + ECBlock[] inputBlocks = getInputBlocks(blockGroup); + ECBlock[] outputBlocks = getOutputBlocks(blockGroup); + + RawErasureDecoder rawDecoder = checkCreateRSRawDecoder(); + return new ErasureDecodingStep(inputBlocks, + getErasedIndexes(inputBlocks), outputBlocks, rawDecoder); + } + + private RawErasureDecoder checkCreateRSRawDecoder() { + if (rsRawDecoder == null) { + rsRawDecoder = CodecUtil.createRSRawDecoder(getConf(), + getNumDataUnits(), getNumParityUnits()); + } + return rsRawDecoder; + } + + @Override + public void release() { + if (rsRawDecoder != null) { + rsRawDecoder.release(); + } + } +} 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 0000000000000..3ed3e2091d9ce --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/RSErasureEncoder.java @@ -0,0 +1,67 @@ +/** + * 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.CodecUtil; +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; + +/** + * Reed-Solomon erasure encoder that encodes a block group. + * + * It implements {@link ErasureCoder}. + */ +public class RSErasureEncoder extends AbstractErasureEncoder { + private RawErasureEncoder rawEncoder; + + public RSErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public RSErasureEncoder(ECSchema schema) { + super(schema); + } + + @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 = CodecUtil.createRSRawEncoder(getConf(), + getNumDataUnits(), getNumParityUnits()); + } + return rawEncoder; + } + + @Override + public void release() { + if (rawEncoder != null) { + rawEncoder.release(); + } + } +} 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 0000000000000..a84741824e877 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureDecoder.java @@ -0,0 +1,86 @@ +/** + * 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.CodecUtil; +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; + +/** + * Xor erasure decoder that decodes a block group. + * + * It implements {@link ErasureCoder}. + */ +public class XORErasureDecoder extends AbstractErasureDecoder { + + public XORErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public XORErasureDecoder(ECSchema schema) { + super(schema); + } + + @Override + protected ErasureCodingStep prepareDecodingStep( + final ECBlockGroup blockGroup) { + RawErasureDecoder rawDecoder = CodecUtil.createXORRawDecoder(getConf(), + getNumDataUnits(), getNumParityUnits()); + + 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; + } +} 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 0000000000000..5c4bcddb5150e --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/coder/XORErasureEncoder.java @@ -0,0 +1,53 @@ +/** + * 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.CodecUtil; +import org.apache.hadoop.io.erasurecode.ECBlock; +import org.apache.hadoop.io.erasurecode.ECBlockGroup; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; + +/** + * Xor erasure encoder that encodes a block group. + * + * It implements {@link ErasureCoder}. + */ +public class XORErasureEncoder extends AbstractErasureEncoder { + + public XORErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + public XORErasureEncoder(ECSchema schema) { + super(schema); + } + + @Override + protected ErasureCodingStep prepareEncodingStep( + final ECBlockGroup blockGroup) { + RawErasureEncoder rawEncoder = CodecUtil.createXORRawEncoder(getConf(), + getNumDataUnits(), getNumParityUnits()); + + ECBlock[] inputBlocks = getInputBlocks(blockGroup); + + return new ErasureEncodingStep(inputBlocks, + getOutputBlocks(blockGroup), rawEncoder); + } + +} 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 0000000000000..bdc162478892f --- /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(); + } + +} 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 new file mode 100644 index 0000000000000..4b7461e118da2 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureCoder.java @@ -0,0 +1,138 @@ +/** + * 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.HadoopIllegalArgumentException; +import org.apache.hadoop.conf.Configured; + +import java.nio.ByteBuffer; +import java.util.Arrays; + +/** + * A common class of basic facilities to be shared by encoder and decoder + * + * It implements the {@link RawErasureCoder} interface. + */ +public abstract class AbstractRawErasureCoder + extends Configured implements RawErasureCoder { + + private final int numDataUnits; + private final int numParityUnits; + + public AbstractRawErasureCoder(int numDataUnits, int numParityUnits) { + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; + } + + @Override + public int getNumDataUnits() { + return numDataUnits; + } + + @Override + public int getNumParityUnits() { + return numParityUnits; + } + + @Override + public boolean preferDirectBuffer() { + return false; + } + + @Override + public void release() { + // Nothing to do by default + } + + /** + * Ensure a buffer filled with ZERO bytes from current readable/writable + * position. + * @param buffer a buffer ready to read / write certain size bytes + * @return the buffer itself, with ZERO bytes written, the position and limit + * are not changed after the call + */ + protected ByteBuffer resetBuffer(ByteBuffer buffer) { + int pos = buffer.position(); + for (int i = pos; i < buffer.limit(); ++i) { + buffer.put((byte) 0); + } + buffer.position(pos); + + return buffer; + } + + /** + * Ensure the buffer (either input or output) ready to read or write with ZERO + * bytes fully in specified length of len. + * @param buffer bytes array buffer + * @return the buffer itself + */ + protected byte[] resetBuffer(byte[] buffer, int offset, int len) { + for (int i = offset; i < len; ++i) { + buffer[i] = (byte) 0; + } + + return buffer; + } + + /** + * Check and ensure the buffers are of the length specified by dataLen, also + * ensure the buffers are direct buffers or not according to isDirectBuffer. + * @param buffers the buffers to check + * @param allowNull whether to allow any element to be null or not + * @param dataLen the length of data available in the buffer to ensure with + * @param isDirectBuffer is direct buffer or not to ensure with + */ + protected void ensureLengthAndType(ByteBuffer[] buffers, boolean allowNull, + int dataLen, boolean isDirectBuffer) { + for (ByteBuffer buffer : buffers) { + if (buffer == null && !allowNull) { + throw new HadoopIllegalArgumentException( + "Invalid buffer found, not allowing null"); + } else if (buffer != null) { + if (buffer.remaining() != dataLen) { + throw new HadoopIllegalArgumentException( + "Invalid buffer, not of length " + dataLen); + } + if (buffer.isDirect() != isDirectBuffer) { + throw new HadoopIllegalArgumentException( + "Invalid buffer, isDirect should be " + isDirectBuffer); + } + } + } + } + + /** + * Check and ensure the buffers are of the length specified by dataLen. + * @param buffers the buffers to check + * @param allowNull whether to allow any element to be null or not + * @param dataLen the length of data available in the buffer to ensure with + */ + protected void ensureLength(byte[][] buffers, + boolean allowNull, int dataLen) { + for (byte[] buffer : buffers) { + if (buffer == null && !allowNull) { + throw new HadoopIllegalArgumentException( + "Invalid buffer found, not allowing null"); + } else if (buffer != null && buffer.length != dataLen) { + throw new HadoopIllegalArgumentException( + "Invalid buffer not of length " + dataLen); + } + } + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java new file mode 100644 index 0000000000000..931cda1ce7409 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureDecoder.java @@ -0,0 +1,207 @@ +/** + * 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.HadoopIllegalArgumentException; +import org.apache.hadoop.io.erasurecode.ECChunk; + +import java.nio.ByteBuffer; +import java.util.Arrays; + +/** + * An abstract raw erasure decoder that's to be inherited by new decoders. + * + * It implements the {@link RawErasureDecoder} interface. + */ +public abstract class AbstractRawErasureDecoder extends AbstractRawErasureCoder + implements RawErasureDecoder { + + public AbstractRawErasureDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + @Override + public void decode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + checkParameters(inputs, erasedIndexes, outputs); + + ByteBuffer validInput = findFirstValidInput(inputs); + boolean usingDirectBuffer = validInput.isDirect(); + int dataLen = validInput.remaining(); + if (dataLen == 0) { + return; + } + ensureLengthAndType(inputs, true, dataLen, usingDirectBuffer); + ensureLengthAndType(outputs, false, dataLen, usingDirectBuffer); + + if (usingDirectBuffer) { + doDecode(inputs, erasedIndexes, outputs); + return; + } + + int[] inputOffsets = new int[inputs.length]; + int[] outputOffsets = new int[outputs.length]; + byte[][] newInputs = new byte[inputs.length][]; + byte[][] newOutputs = new byte[outputs.length][]; + + ByteBuffer buffer; + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + if (buffer != null) { + inputOffsets[i] = buffer.arrayOffset() + buffer.position(); + newInputs[i] = buffer.array(); + } + } + + for (int i = 0; i < outputs.length; ++i) { + buffer = outputs[i]; + outputOffsets[i] = buffer.arrayOffset() + buffer.position(); + newOutputs[i] = buffer.array(); + } + + doDecode(newInputs, inputOffsets, dataLen, + erasedIndexes, newOutputs, outputOffsets); + + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + if (buffer != null) { + // dataLen bytes consumed + buffer.position(buffer.position() + dataLen); + } + } + } + + /** + * Perform the real decoding using Direct ByteBuffer. + * @param inputs Direct ByteBuffers expected + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs Direct ByteBuffers expected + */ + protected abstract void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs); + + @Override + public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs) { + checkParameters(inputs, erasedIndexes, outputs); + + byte[] validInput = findFirstValidInput(inputs); + int dataLen = validInput.length; + if (dataLen == 0) { + return; + } + ensureLength(inputs, true, dataLen); + ensureLength(outputs, false, dataLen); + + int[] inputOffsets = new int[inputs.length]; // ALL ZERO + int[] outputOffsets = new int[outputs.length]; // ALL ZERO + + doDecode(inputs, inputOffsets, dataLen, erasedIndexes, outputs, + outputOffsets); + } + + /** + * Perform the real decoding using bytes array, supporting offsets and + * lengths. + * @param inputs the input byte arrays to read data from + * @param inputOffsets offsets for the input byte arrays to read data from + * @param dataLen how much data are to be read from + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs the output byte arrays to write resultant data into + * @param outputOffsets offsets from which to write resultant data into + */ + protected abstract void doDecode(byte[][] inputs, int[] inputOffsets, + int dataLen, int[] erasedIndexes, + byte[][] outputs, int[] outputOffsets); + + @Override + public void decode(ECChunk[] inputs, int[] erasedIndexes, + ECChunk[] outputs) { + ByteBuffer[] newInputs = ECChunk.toBuffers(inputs); + ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs); + decode(newInputs, erasedIndexes, newOutputs); + } + + /** + * Check and validate decoding parameters, throw exception accordingly. The + * checking assumes it's a MDS code. Other code can override this. + * @param inputs input buffers to check + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs output buffers to check + */ + protected void checkParameters(T[] inputs, int[] erasedIndexes, + T[] outputs) { + if (inputs.length != getNumParityUnits() + getNumDataUnits()) { + throw new IllegalArgumentException("Invalid inputs length"); + } + + if (erasedIndexes.length != outputs.length) { + throw new HadoopIllegalArgumentException( + "erasedIndexes and outputs mismatch in length"); + } + + if (erasedIndexes.length > getNumParityUnits()) { + throw new HadoopIllegalArgumentException( + "Too many erased, not recoverable"); + } + + int validInputs = 0; + for (T input : inputs) { + if (input != null) { + validInputs += 1; + } + } + + if (validInputs < getNumDataUnits()) { + throw new HadoopIllegalArgumentException( + "No enough valid inputs are provided, not recoverable"); + } + } + + /** + * Get indexes into inputs array for items marked as null, either erased or + * not to read. + * @return indexes into inputs array + */ + protected int[] getErasedOrNotToReadIndexes(T[] inputs) { + int[] invalidIndexes = new int[inputs.length]; + int idx = 0; + for (int i = 0; i < inputs.length; i++) { + if (inputs[i] == null) { + invalidIndexes[idx++] = i; + } + } + + return Arrays.copyOf(invalidIndexes, idx); + } + + /** + * Find the valid input from all the inputs. + * @param inputs input buffers to look for valid input + * @return the first valid input + */ + protected static T findFirstValidInput(T[] inputs) { + for (T input : inputs) { + if (input != null) { + return input; + } + } + + throw new HadoopIllegalArgumentException( + "Invalid inputs are found, all being null"); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java new file mode 100644 index 0000000000000..a0b3cfef2a167 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/AbstractRawErasureEncoder.java @@ -0,0 +1,136 @@ +/** + * 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.HadoopIllegalArgumentException; +import org.apache.hadoop.io.erasurecode.ECChunk; + +import java.nio.ByteBuffer; + +/** + * An abstract raw erasure encoder that's to be inherited by new encoders. + * + * It implements the {@link RawErasureEncoder} interface. + */ +public abstract class AbstractRawErasureEncoder extends AbstractRawErasureCoder + implements RawErasureEncoder { + + public AbstractRawErasureEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + @Override + public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs) { + checkParameters(inputs, outputs); + + boolean usingDirectBuffer = inputs[0].isDirect(); + int dataLen = inputs[0].remaining(); + if (dataLen == 0) { + return; + } + ensureLengthAndType(inputs, false, dataLen, usingDirectBuffer); + ensureLengthAndType(outputs, false, dataLen, usingDirectBuffer); + + if (usingDirectBuffer) { + doEncode(inputs, outputs); + return; + } + + int[] inputOffsets = new int[inputs.length]; + int[] outputOffsets = new int[outputs.length]; + byte[][] newInputs = new byte[inputs.length][]; + byte[][] newOutputs = new byte[outputs.length][]; + + ByteBuffer buffer; + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + inputOffsets[i] = buffer.arrayOffset() + buffer.position(); + newInputs[i] = buffer.array(); + } + + for (int i = 0; i < outputs.length; ++i) { + buffer = outputs[i]; + outputOffsets[i] = buffer.arrayOffset() + buffer.position(); + newOutputs[i] = buffer.array(); + } + + doEncode(newInputs, inputOffsets, dataLen, newOutputs, outputOffsets); + + for (int i = 0; i < inputs.length; ++i) { + buffer = inputs[i]; + buffer.position(buffer.position() + dataLen); // dataLen bytes consumed + } + } + + /** + * Perform the real encoding work using direct ByteBuffer + * @param inputs Direct ByteBuffers expected + * @param outputs Direct ByteBuffers expected + */ + protected abstract void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs); + + @Override + public void encode(byte[][] inputs, byte[][] outputs) { + checkParameters(inputs, outputs); + int dataLen = inputs[0].length; + if (dataLen == 0) { + return; + } + ensureLength(inputs, false, dataLen); + ensureLength(outputs, false, dataLen); + + int[] inputOffsets = new int[inputs.length]; // ALL ZERO + int[] outputOffsets = new int[outputs.length]; // ALL ZERO + + doEncode(inputs, inputOffsets, dataLen, outputs, outputOffsets); + } + + /** + * Perform the real encoding work using bytes array, supporting offsets + * and lengths. + * @param inputs the input byte arrays to read data from + * @param inputOffsets offsets for the input byte arrays to read data from + * @param dataLen how much data are to be read from + * @param outputs the output byte arrays to write resultant data into + * @param outputOffsets offsets from which to write resultant data into + */ + protected abstract void doEncode(byte[][] inputs, int[] inputOffsets, + int dataLen, byte[][] outputs, + int[] outputOffsets); + + @Override + public void encode(ECChunk[] inputs, ECChunk[] outputs) { + ByteBuffer[] newInputs = ECChunk.toBuffers(inputs); + ByteBuffer[] newOutputs = ECChunk.toBuffers(outputs); + encode(newInputs, newOutputs); + } + + /** + * Check and validate decoding parameters, throw exception accordingly. + * @param inputs input buffers to check + * @param outputs output buffers to check + */ + protected void checkParameters(T[] inputs, T[] outputs) { + if (inputs.length != getNumDataUnits()) { + throw new HadoopIllegalArgumentException("Invalid inputs length"); + } + if (outputs.length != getNumParityUnits()) { + throw new HadoopIllegalArgumentException("Invalid outputs length"); + } + } +} 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 0000000000000..57e6957435ca3 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawDecoder.java @@ -0,0 +1,216 @@ +/** + * 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.HadoopIllegalArgumentException; +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. + * + * Currently this implementation will compute and decode not to read units + * unnecessarily due to the underlying implementation limit in GF. This will be + * addressed in HADOOP-11871. + */ +public class RSRawDecoder extends AbstractRawErasureDecoder { + // To describe and calculate the needed Vandermonde matrix + private int[] errSignature; + private int[] primitivePower; + + /** + * We need a set of reusable buffers either for the bytes array + * decoding version or direct buffer decoding version. Normally not both. + * + * For output, in addition to the valid buffers from the caller + * passed from above, we need to provide extra buffers for the internal + * decoding implementation. For output, the caller should provide no more + * than numParityUnits but at least one buffers. And the left buffers will be + * borrowed from either bytesArrayBuffers, for the bytes array version. + * + */ + // Reused buffers for decoding with bytes arrays + private byte[][] bytesArrayBuffers = new byte[getNumParityUnits()][]; + private byte[][] adjustedByteArrayOutputsParameter = + new byte[getNumParityUnits()][]; + private int[] adjustedOutputOffsets = new int[getNumParityUnits()]; + + // Reused buffers for decoding with direct ByteBuffers + private ByteBuffer[] directBuffers = new ByteBuffer[getNumParityUnits()]; + private ByteBuffer[] adjustedDirectBufferOutputsParameter = + new ByteBuffer[getNumParityUnits()]; + + public RSRawDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + if (numDataUnits + numParityUnits >= RSUtil.GF.getFieldSize()) { + throw new HadoopIllegalArgumentException( + "Invalid numDataUnits and numParityUnits"); + } + + this.errSignature = new int[numParityUnits]; + this.primitivePower = RSUtil.getPrimitivePower(numDataUnits, + numParityUnits); + } + + private void doDecodeImpl(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + ByteBuffer valid = findFirstValidInput(inputs); + int dataLen = valid.remaining(); + for (int i = 0; i < erasedIndexes.length; i++) { + errSignature[i] = primitivePower[erasedIndexes[i]]; + RSUtil.GF.substitute(inputs, dataLen, outputs[i], primitivePower[i]); + } + + RSUtil.GF.solveVandermondeSystem(errSignature, + outputs, erasedIndexes.length); + } + + private void doDecodeImpl(byte[][] inputs, int[] inputOffsets, + int dataLen, int[] erasedIndexes, + byte[][] outputs, int[] outputOffsets) { + for (int i = 0; i < erasedIndexes.length; i++) { + errSignature[i] = primitivePower[erasedIndexes[i]]; + RSUtil.GF.substitute(inputs, inputOffsets, dataLen, outputs[i], + outputOffsets[i], primitivePower[i]); + } + + RSUtil.GF.solveVandermondeSystem(errSignature, outputs, outputOffsets, + erasedIndexes.length, dataLen); + } + + @Override + protected void doDecode(byte[][] inputs, int[] inputOffsets, + int dataLen, int[] erasedIndexes, + byte[][] outputs, int[] outputOffsets) { + /** + * As passed parameters are friendly to callers but not to the underlying + * implementations, so we have to adjust them before calling doDecodeImpl. + */ + + int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs); + + // Prepare for adjustedOutputsParameter + + // First reset the positions needed this time + for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) { + adjustedByteArrayOutputsParameter[i] = null; + adjustedOutputOffsets[i] = 0; + } + // Use the caller passed buffers in erasedIndexes positions + for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) { + boolean found = false; + for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) { + // If this index is one requested by the caller via erasedIndexes, then + // we use the passed output buffer to avoid copying data thereafter. + if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) { + found = true; + adjustedByteArrayOutputsParameter[j] = resetBuffer( + outputs[outputIdx], outputOffsets[outputIdx], dataLen); + adjustedOutputOffsets[j] = outputOffsets[outputIdx]; + outputIdx++; + } + } + if (!found) { + throw new HadoopIllegalArgumentException( + "Inputs not fully corresponding to erasedIndexes in null places"); + } + } + // Use shared buffers for other positions (not set yet) + for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) { + if (adjustedByteArrayOutputsParameter[i] == null) { + adjustedByteArrayOutputsParameter[i] = resetBuffer( + checkGetBytesArrayBuffer(bufferIdx, dataLen), 0, dataLen); + adjustedOutputOffsets[i] = 0; // Always 0 for such temp output + bufferIdx++; + } + } + + doDecodeImpl(inputs, inputOffsets, dataLen, erasedOrNotToReadIndexes, + adjustedByteArrayOutputsParameter, adjustedOutputOffsets); + } + + @Override + protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + ByteBuffer validInput = findFirstValidInput(inputs); + int dataLen = validInput.remaining(); + + /** + * As passed parameters are friendly to callers but not to the underlying + * implementations, so we have to adjust them before calling doDecodeImpl. + */ + + int[] erasedOrNotToReadIndexes = getErasedOrNotToReadIndexes(inputs); + + // Prepare for adjustedDirectBufferOutputsParameter + + // First reset the positions needed this time + for (int i = 0; i < erasedOrNotToReadIndexes.length; i++) { + adjustedDirectBufferOutputsParameter[i] = null; + } + // Use the caller passed buffers in erasedIndexes positions + for (int outputIdx = 0, i = 0; i < erasedIndexes.length; i++) { + boolean found = false; + for (int j = 0; j < erasedOrNotToReadIndexes.length; j++) { + // If this index is one requested by the caller via erasedIndexes, then + // we use the passed output buffer to avoid copying data thereafter. + if (erasedIndexes[i] == erasedOrNotToReadIndexes[j]) { + found = true; + adjustedDirectBufferOutputsParameter[j] = + resetBuffer(outputs[outputIdx++]); + } + } + if (!found) { + throw new HadoopIllegalArgumentException( + "Inputs not fully corresponding to erasedIndexes in null places"); + } + } + // Use shared buffers for other positions (not set yet) + for (int bufferIdx = 0, i = 0; i < erasedOrNotToReadIndexes.length; i++) { + if (adjustedDirectBufferOutputsParameter[i] == null) { + ByteBuffer buffer = checkGetDirectBuffer(bufferIdx, dataLen); + buffer.position(0); + buffer.limit(dataLen); + adjustedDirectBufferOutputsParameter[i] = resetBuffer(buffer); + bufferIdx++; + } + } + + doDecodeImpl(inputs, erasedOrNotToReadIndexes, + adjustedDirectBufferOutputsParameter); + } + + private byte[] checkGetBytesArrayBuffer(int idx, int bufferLen) { + if (bytesArrayBuffers[idx] == null || + bytesArrayBuffers[idx].length < bufferLen) { + bytesArrayBuffers[idx] = new byte[bufferLen]; + } + return bytesArrayBuffers[idx]; + } + + private ByteBuffer checkGetDirectBuffer(int idx, int bufferLen) { + if (directBuffers[idx] == null || + directBuffers[idx].capacity() < bufferLen) { + directBuffers[idx] = ByteBuffer.allocateDirect(bufferLen); + } + return directBuffers[idx]; + } +} 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 0000000000000..efeee9030c9b5 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RSRawEncoder.java @@ -0,0 +1,79 @@ +/** + * 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; + + public RSRawEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + + assert (getNumDataUnits() + getNumParityUnits() < RSUtil.GF.getFieldSize()); + + int[] primitivePower = RSUtil.getPrimitivePower(numDataUnits, + numParityUnits); + // compute generating polynomial + int[] gen = {1}; + int[] poly = new int[2]; + for (int i = 0; i < numParityUnits; 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) { + // parity units + data units + ByteBuffer[] all = new ByteBuffer[outputs.length + inputs.length]; + System.arraycopy(outputs, 0, all, 0, outputs.length); + System.arraycopy(inputs, 0, all, outputs.length, inputs.length); + + // Compute the remainder + RSUtil.GF.remainder(all, generatingPolynomial); + } + + @Override + protected void doEncode(byte[][] inputs, int[] inputOffsets, + int dataLen, byte[][] outputs, + int[] outputOffsets) { + // parity units + data units + byte[][] all = new byte[outputs.length + inputs.length][]; + System.arraycopy(outputs, 0, all, 0, outputs.length); + System.arraycopy(inputs, 0, all, outputs.length, inputs.length); + + int[] offsets = new int[inputOffsets.length + outputOffsets.length]; + System.arraycopy(outputOffsets, 0, offsets, 0, outputOffsets.length); + System.arraycopy(inputOffsets, 0, offsets, + outputOffsets.length, inputOffsets.length); + + // Compute the remainder + RSUtil.GF.remainder(all, offsets, dataLen, generatingPolynomial); + } +} 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 0000000000000..5db49e388fb36 --- /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(int numDataUnits, int numParityUnits) { + return new RSRawEncoder(numDataUnits, numParityUnits); + } + + @Override + public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) { + return new RSRawDecoder(numDataUnits, numParityUnits); + } +} 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 new file mode 100644 index 0000000000000..0ddb460345cf7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoder.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.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. + * + * RawErasureCoder is part of ErasureCodec framework, where ErasureCoder is + * used to encode/decode a group of blocks (BlockGroup) according to the codec + * specific BlockGroup layout and logic. An ErasureCoder extracts chunks of + * data from the blocks and can employ various low level RawErasureCoders to + * perform encoding/decoding against the chunks. + * + * To distinguish from ErasureCoder, here RawErasureCoder is used to mean the + * low level constructs, since it only takes care of the math calculation with + * a group of byte buffers. + */ +public interface RawErasureCoder extends Configurable { + + /** + * 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(); + + /** + * Tell if direct buffer is preferred or not. It's for callers to + * decide how to allocate coding chunk buffers, using DirectByteBuffer or + * bytes array. It will return false by default. + * @return true if native buffer is preferred for performance consideration, + * otherwise false. + */ + public boolean preferDirectBuffer(); + + /** + * Should be called when release this coder. Good chance to release encoding + * or decoding buffers + */ + public void release(); +} 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 0000000000000..280daf3119955 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureCoderFactory.java @@ -0,0 +1,42 @@ +/** + * 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. + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group + * @return raw erasure encoder + */ + public RawErasureEncoder createEncoder(int numDataUnits, int numParityUnits); + + /** + * Create raw erasure decoder. + * @param numDataUnits number of data units in a coding group + * @param numParityUnits number of parity units in a coding group + * @return raw erasure decoder + */ + public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits); +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.java new file mode 100644 index 0000000000000..e2d01d9c6fa4d --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureDecoder.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.rawcoder; + +import org.apache.hadoop.io.erasurecode.ECChunk; + +import java.nio.ByteBuffer; + +/** + * RawErasureDecoder performs decoding given chunks of input data and generates + * missing data that corresponds to an erasure code scheme, like XOR and + * Reed-Solomon. + * + * It extends the {@link RawErasureCoder} interface. + */ +public interface RawErasureDecoder extends RawErasureCoder { + + /** + * Decode with inputs and erasedIndexes, generates outputs. + * How to prepare for inputs: + * 1. Create an array containing parity units + data units. Please note the + * parity units should be first or before the data units. + * 2. Set null in the array locations specified via erasedIndexes to indicate + * they're erased and no data are to read from; + * 3. Set null in the array locations for extra redundant items, as they're + * not necessary to read when decoding. For example in RS-6-3, if only 1 + * unit is really erased, then we have 2 extra items as redundant. They can + * be set as null to indicate no data will be used from them. + * + * For an example using RS (6, 3), assuming sources (d0, d1, d2, d3, d4, d5) + * and parities (p0, p1, p2), d2 being erased. We can and may want to use only + * 6 units like (d1, d3, d4, d5, p0, p2) to recover d2. We will have: + * inputs = [p0, null(p1), p2, null(d0), d1, null(d2), d3, d4, d5] + * erasedIndexes = [5] // index of d2 into inputs array + * outputs = [a-writable-buffer] + * + * Note, for both inputs and outputs, no mixing of on-heap buffers and direct + * buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs outputs to write into for data generated according to + * erasedIndexes, ready for reading the result data from after + * the call + */ + public void decode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs); + + /** + * Decode with inputs and erasedIndexes, generates outputs. More see above. + * @param inputs inputs to read data from, contents may change after the call + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs outputs to write into for data generated according to + * erasedIndexes, ready for reading the result data from after + * the call + */ + public void decode(byte[][] inputs, int[] erasedIndexes, byte[][] outputs); + + /** + * Decode with inputs and erasedIndexes, generates outputs. More see above. + * + * Note, for both input and output ECChunks, no mixing of on-heap buffers and + * direct buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call + * @param erasedIndexes indexes of erased units in the inputs array + * @param outputs outputs to write into for data generated according to + * erasedIndexes, ready for reading the result data from after + * the call + */ + public void decode(ECChunk[] inputs, int[] erasedIndexes, ECChunk[] outputs); + +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java new file mode 100644 index 0000000000000..7571f095caa6a --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/RawErasureEncoder.java @@ -0,0 +1,64 @@ +/** + * 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 java.nio.ByteBuffer; + +/** + * RawErasureEncoder performs encoding given chunks of input data and generates + * parity outputs that corresponds to an erasure code scheme, like XOR and + * Reed-Solomon. + * + * It extends the {@link RawErasureCoder} interface. + */ +public interface RawErasureEncoder extends RawErasureCoder { + + /** + * Encode with inputs and generates outputs. + * + * Note, for both inputs and outputs, no mixing of on-heap buffers and direct + * buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call + * @param outputs + */ + public void encode(ByteBuffer[] inputs, ByteBuffer[] outputs); + + /** + * Encode with inputs and generates outputs + * @param inputs inputs to read data from, contents may change after the call + * @param outputs outputs to write into for data generated, ready for reading + * the result data from after the call + */ + public void encode(byte[][] inputs, byte[][] outputs); + + /** + * Encode with inputs and generates outputs. + * + * Note, for both input and output ECChunks, no mixing of on-heap buffers and + * direct buffers are allowed. + * + * @param inputs inputs to read data from, contents may change after the call + * @param outputs outputs to write into for data generated, ready for reading + * the result data from after the call + */ + public void encode(ECChunk[] inputs, ECChunk[] outputs); + +} 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 0000000000000..e20e543422cad --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawDecoder.java @@ -0,0 +1,83 @@ +/** + * 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. + * + * XOR code is an important primitive code scheme in erasure coding and often + * used in advanced codes, like HitchHiker and LRC, though itself is rarely + * deployed independently. + */ +public class XORRawDecoder extends AbstractRawErasureDecoder { + + public XORRawDecoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + @Override + protected void doDecode(ByteBuffer[] inputs, int[] erasedIndexes, + ByteBuffer[] outputs) { + ByteBuffer output = outputs[0]; + resetBuffer(output); + + int erasedIdx = erasedIndexes[0]; + + // Process the inputs. + int iIdx, oIdx; + for (int i = 0; i < inputs.length; i++) { + // Skip the erased location. + if (i == erasedIdx) { + continue; + } + + for (iIdx = inputs[i].position(), oIdx = output.position(); + iIdx < inputs[i].limit(); + iIdx++, oIdx++) { + output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx))); + } + } + } + + @Override + protected void doDecode(byte[][] inputs, int[] inputOffsets, int dataLen, + int[] erasedIndexes, byte[][] outputs, + int[] outputOffsets) { + byte[] output = outputs[0]; + resetBuffer(output, outputOffsets[0], dataLen); + + int erasedIdx = erasedIndexes[0]; + + // Process the inputs. + int iIdx, oIdx; + for (int i = 0; i < inputs.length; i++) { + // Skip the erased location. + if (i == erasedIdx) { + continue; + } + + for (iIdx = inputOffsets[i], oIdx = outputOffsets[0]; + iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) { + output[oIdx] ^= inputs[i][iIdx]; + } + } + } + +} 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 0000000000000..f4d242eefb026 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/XORRawEncoder.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.rawcoder; + +import java.nio.ByteBuffer; + +/** + * A raw encoder in XOR code scheme in pure Java, adapted from HDFS-RAID. + * + * XOR code is an important primitive code scheme in erasure coding and often + * used in advanced codes, like HitchHiker and LRC, though itself is rarely + * deployed independently. + */ +public class XORRawEncoder extends AbstractRawErasureEncoder { + + public XORRawEncoder(int numDataUnits, int numParityUnits) { + super(numDataUnits, numParityUnits); + } + + protected void doEncode(ByteBuffer[] inputs, ByteBuffer[] outputs) { + ByteBuffer output = outputs[0]; + resetBuffer(output); + + // Get the first buffer's data. + int iIdx, oIdx; + for (iIdx = inputs[0].position(), oIdx = output.position(); + iIdx < inputs[0].limit(); iIdx++, oIdx++) { + output.put(oIdx, inputs[0].get(iIdx)); + } + + // XOR with everything else. + for (int i = 1; i < inputs.length; i++) { + for (iIdx = inputs[i].position(), oIdx = output.position(); + iIdx < inputs[i].limit(); + iIdx++, oIdx++) { + output.put(oIdx, (byte) (output.get(oIdx) ^ inputs[i].get(iIdx))); + } + } + } + + @Override + protected void doEncode(byte[][] inputs, int[] inputOffsets, int dataLen, + byte[][] outputs, int[] outputOffsets) { + byte[] output = outputs[0]; + resetBuffer(output, outputOffsets[0], dataLen); + + // Get the first buffer's data. + int iIdx, oIdx; + for (iIdx = inputOffsets[0], oIdx = outputOffsets[0]; + iIdx < inputOffsets[0] + dataLen; iIdx++, oIdx++) { + output[oIdx] = inputs[0][iIdx]; + } + + // XOR with everything else. + for (int i = 1; i < inputs.length; i++) { + for (iIdx = inputOffsets[i], oIdx = outputOffsets[0]; + iIdx < inputOffsets[i] + dataLen; iIdx++, oIdx++) { + output[oIdx] ^= inputs[i][iIdx]; + } + } + } +} 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 0000000000000..de20c950eb795 --- /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(int numDataUnits, int numParityUnits) { + return new XORRawEncoder(numDataUnits, numParityUnits); + } + + @Override + public RawErasureDecoder createDecoder(int numDataUnits, int numParityUnits) { + return new XORRawDecoder(numDataUnits, numParityUnits); + } +} diff --git a/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java new file mode 100644 index 0000000000000..c8f133f765585 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/DumpUtil.java @@ -0,0 +1,85 @@ +/** + * 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 org.apache.hadoop.io.erasurecode.ECChunk; + +/** + * A dump utility class for debugging data erasure coding/decoding issues. Don't + * suggest they are used in runtime production codes. + */ +public final class DumpUtil { + private static final String HEX_CHARS_STR = "0123456789ABCDEF"; + private static final char[] HEX_CHARS = HEX_CHARS_STR.toCharArray(); + + private DumpUtil() { + // No called + } + + /** + * Convert bytes into format like 0x02 02 00 80. + */ + public static String bytesToHex(byte[] bytes, int limit) { + if (limit > bytes.length) { + limit = bytes.length; + } + int len = limit * 2; + len += limit; // for ' ' appended for each char + len += 2; // for '0x' prefix + char[] hexChars = new char[len]; + hexChars[0] = '0'; + hexChars[1] = 'x'; + for (int j = 0; j < limit; j++) { + int v = bytes[j] & 0xFF; + hexChars[j * 3 + 2] = HEX_CHARS[v >>> 4]; + hexChars[j * 3 + 3] = HEX_CHARS[v & 0x0F]; + hexChars[j * 3 + 4] = ' '; + } + + return new String(hexChars); + } + + /** + * Print data in hex format in an array of chunks. + * @param header + * @param chunks + */ + public static void dumpChunks(String header, ECChunk[] chunks) { + System.out.println(); + System.out.println(header); + for (int i = 0; i < chunks.length; i++) { + dumpChunk(chunks[i]); + } + System.out.println(); + } + + /** + * Print data in hex format in a chunk. + * @param chunk + */ + public static void dumpChunk(ECChunk chunk) { + String str; + if (chunk == null) { + str = ""; + } else { + byte[] bytes = chunk.toBytesArray(); + str = DumpUtil.bytesToHex(bytes, 16); + } + System.out.println(str); + } +} 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 0000000000000..03683b03deffc --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/GaloisField.java @@ -0,0 +1,561 @@ +/** + * 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 instances = + new HashMap(); + 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[] outputOffsets, + int len, int dataLen) { + int idx1, idx2; + for (int i = 0; i < len - 1; i++) { + for (int j = len - 1; j > i; j--) { + for (idx2 = outputOffsets[j-1], idx1 = outputOffsets[j]; + idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) { + y[j][idx1] = (byte) (y[j][idx1] ^ mulTable[x[i]][y[j - 1][idx2] & + 0x000000FF]); + } + } + } + for (int i = len - 1; i >= 0; i--) { + for (int j = i + 1; j < len; j++) { + for (idx1 = outputOffsets[j]; + idx1 < outputOffsets[j] + dataLen; idx1++) { + y[j][idx1] = (byte) (divTable[y[j][idx1] & 0x000000FF][x[j] ^ + x[j - i - 1]]); + } + } + for (int j = i; j < len - 1; j++) { + for (idx2 = outputOffsets[j+1], idx1 = outputOffsets[j]; + idx1 < outputOffsets[j] + dataLen; idx1++, idx2++) { + y[j][idx1] = (byte) (y[j][idx1] ^ y[j + 1][idx2]); + } + } + } + } + + /** + * A "bulk" version of the solveVandermondeSystem, using ByteBuffer. + */ + public void solveVandermondeSystem(int[] x, ByteBuffer[] y, int len) { + ByteBuffer p; + int idx1, idx2; + for (int i = 0; i < len - 1; i++) { + for (int j = len - 1; j > i; j--) { + p = y[j]; + for (idx1 = p.position(), idx2 = y[j-1].position(); + idx1 < p.limit(); idx1++, idx2++) { + p.put(idx1, (byte) (p.get(idx1) ^ mulTable[x[i]][y[j-1].get(idx2) & + 0x000000FF])); + } + } + } + + for (int i = len - 1; i >= 0; i--) { + for (int j = i + 1; j < len; j++) { + p = y[j]; + for (idx1 = p.position(); idx1 < p.limit(); idx1++) { + p.put(idx1, (byte) (divTable[p.get(idx1) & + 0x000000FF][x[j] ^ x[j - i - 1]])); + } + } + + for (int j = i; j < len - 1; j++) { + p = y[j]; + for (idx1 = p.position(), idx2 = y[j+1].position(); + idx1 < p.limit(); idx1++, idx2++) { + p.put(idx1, (byte) (p.get(idx1) ^ y[j+1].get(idx2))); + } + } + } + } + + /** + * 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. + * Tends to be 2X faster than the "int" substitute in a loop. + * + * @param p input polynomial + * @param offsets + * @param len + * @param q store the return result + * @param offset + * @param x input field + */ + public void substitute(byte[][] p, int[] offsets, + int len, byte[] q, int offset, int x) { + int y = 1, iIdx, oIdx; + for (int i = 0; i < p.length; i++) { + byte[] pi = p[i]; + for (iIdx = offsets[i], oIdx = offset; + iIdx < offsets[i] + len; iIdx++, oIdx++) { + int pij = pi != null ? pi[iIdx] & 0x000000FF : 0; + q[oIdx] = (byte) (q[oIdx] ^ 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, int len, ByteBuffer q, int x) { + int y = 1, iIdx, oIdx; + for (int i = 0; i < p.length; i++) { + ByteBuffer pi = p[i]; + int pos = pi != null ? pi.position() : 0; + int limit = pi != null ? pi.limit() : len; + for (oIdx = q.position(), iIdx = pos; + iIdx < limit; iIdx++, oIdx++) { + int pij = pi != null ? pi.get(iIdx) & 0x000000FF : 0; + q.put(oIdx, (byte) (q.get(oIdx) ^ 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. + * Warning: This function will modify the "dividend" inputs. + */ + public void remainder(byte[][] dividend, int[] offsets, + int len, int[] divisor) { + int idx1, idx2; + for (int i = dividend.length - divisor.length; i >= 0; i--) { + for (int j = 0; j < divisor.length; j++) { + for (idx2 = offsets[j + i], idx1 = offsets[i + divisor.length - 1]; + idx1 < offsets[i + divisor.length - 1] + len; + idx1++, idx2++) { + int ratio = divTable[dividend[i + divisor.length - 1][idx1] & + 0x00FF][divisor[divisor.length - 1]]; + dividend[j + i][idx2] = (byte) ((dividend[j + i][idx2] & 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) { + int idx1, idx2; + ByteBuffer b1, b2; + for (int i = dividend.length - divisor.length; i >= 0; i--) { + for (int j = 0; j < divisor.length; j++) { + b1 = dividend[i + divisor.length - 1]; + b2 = dividend[j + i]; + for (idx1 = b1.position(), idx2 = b2.position(); + idx1 < b1.limit(); idx1++, idx2++) { + int ratio = divTable[b1.get(idx1) & + 0x00FF][divisor[divisor.length - 1]]; + b2.put(idx2, (byte) ((b2.get(idx2) & 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])); + } + } + } + } + +} 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 0000000000000..8badf021d2f79 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/main/java/org/apache/hadoop/io/erasurecode/rawcoder/util/RSUtil.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.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; + } + +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java new file mode 100644 index 0000000000000..8f552b753eae7 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/BufferAllocator.java @@ -0,0 +1,91 @@ +/** + * 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.nio.ByteBuffer; + +/** + * An abstract buffer allocator used for test. + */ +public abstract class BufferAllocator { + private boolean usingDirect = false; + + public BufferAllocator(boolean usingDirect) { + this.usingDirect = usingDirect; + } + + protected boolean isUsingDirect() { + return usingDirect; + } + + /** + * Allocate and return a ByteBuffer of specified length. + * @param bufferLen + * @return + */ + public abstract ByteBuffer allocate(int bufferLen); + + /** + * A simple buffer allocator that just uses ByteBuffer's + * allocate/allocateDirect API. + */ + public static class SimpleBufferAllocator extends BufferAllocator { + + public SimpleBufferAllocator(boolean usingDirect) { + super(usingDirect); + } + + @Override + public ByteBuffer allocate(int bufferLen) { + return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) : + ByteBuffer.allocate(bufferLen); + } + } + + /** + * A buffer allocator that allocates a buffer from an existing large buffer by + * slice calling, but if no available space just degrades as + * SimpleBufferAllocator. So please ensure enough space for it. + */ + public static class SlicedBufferAllocator extends BufferAllocator { + private ByteBuffer overallBuffer; + + public SlicedBufferAllocator(boolean usingDirect, int totalBufferLen) { + super(usingDirect); + overallBuffer = isUsingDirect() ? + ByteBuffer.allocateDirect(totalBufferLen) : + ByteBuffer.allocate(totalBufferLen); + } + + @Override + public ByteBuffer allocate(int bufferLen) { + if (bufferLen > overallBuffer.capacity() - overallBuffer.position()) { + // If no available space for the requested length, then allocate new + return isUsingDirect() ? ByteBuffer.allocateDirect(bufferLen) : + ByteBuffer.allocate(bufferLen); + } + + overallBuffer.limit(overallBuffer.position() + bufferLen); + ByteBuffer result = overallBuffer.slice(); + overallBuffer.position(overallBuffer.position() + bufferLen); + return result; + } + } + +} 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 new file mode 100644 index 0000000000000..8f277f49f82e1 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestCoderBase.java @@ -0,0 +1,500 @@ +/** + * 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.io.erasurecode.BufferAllocator.SimpleBufferAllocator; +import org.apache.hadoop.io.erasurecode.BufferAllocator.SlicedBufferAllocator; +import org.apache.hadoop.io.erasurecode.rawcoder.util.DumpUtil; + +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.Random; + +import static org.junit.Assert.assertTrue; + +/** + * Test base of common utilities for tests not only raw coders but also block + * coders. + */ +public abstract class TestCoderBase { + protected static Random RAND = new Random(); + + private boolean allowDump = true; + + private Configuration conf; + protected int numDataUnits; + protected int numParityUnits; + protected int baseChunkSize = 513; + private int chunkSize = baseChunkSize; + private BufferAllocator allocator; + + private byte[] zeroChunkBytes; + + private boolean startBufferWithZero = true; + + // Indexes of erased data units. + protected int[] erasedDataIndexes = new int[] {0}; + + // Indexes of erased parity units. + protected int[] erasedParityIndexes = new int[] {0}; + + // Data buffers are either direct or on-heap, for performance the two cases + // may go to different coding implementations. + protected boolean usingDirectBuffer = true; + + protected boolean usingFixedData = true; + // Using this the generated data can be repeatable across multiple calls to + // encode(), in order for troubleshooting. + private static int FIXED_DATA_GENERATOR = 0; + protected byte[][] fixedData; + + protected int getChunkSize() { + return chunkSize; + } + + protected void setChunkSize(int chunkSize) { + this.chunkSize = chunkSize; + this.zeroChunkBytes = new byte[chunkSize]; // With ZERO by default + } + + protected void prepareBufferAllocator(boolean usingSlicedBuffer) { + if (usingSlicedBuffer) { + int roughEstimationSpace = + chunkSize * (numDataUnits + numParityUnits) * 10; + allocator = new SlicedBufferAllocator(usingDirectBuffer, + roughEstimationSpace); + } else { + allocator = new SimpleBufferAllocator(usingDirectBuffer); + } + } + + /** + * Set true during setup if want to dump test settings and coding data, + * useful in debugging. + * @param allowDump + */ + protected void setAllowDump(boolean allowDump) { + this.allowDump = allowDump; + } + + /** + * Prepare before running the case. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @param erasedDataIndexes + * @param erasedParityIndexes + * @param usingFixedData Using fixed or pre-generated data to test instead of + * generating data + */ + protected void prepare(Configuration conf, int numDataUnits, + int numParityUnits, int[] erasedDataIndexes, + int[] erasedParityIndexes, boolean usingFixedData) { + this.conf = conf; + this.numDataUnits = numDataUnits; + this.numParityUnits = numParityUnits; + this.erasedDataIndexes = erasedDataIndexes != null ? + erasedDataIndexes : new int[] {0}; + this.erasedParityIndexes = erasedParityIndexes != null ? + erasedParityIndexes : new int[] {0}; + this.usingFixedData = usingFixedData; + if (usingFixedData) { + prepareFixedData(); + } + } + + /** + * Prepare before running the case. + * @param conf + * @param numDataUnits + * @param numParityUnits + * @param erasedDataIndexes + * @param erasedParityIndexes + */ + protected void prepare(Configuration conf, int numDataUnits, + int numParityUnits, int[] erasedDataIndexes, + int[] erasedParityIndexes) { + prepare(conf, numDataUnits, numParityUnits, erasedDataIndexes, + erasedParityIndexes, false); + } + + /** + * Prepare before running the case. + * @param numDataUnits + * @param numParityUnits + * @param erasedDataIndexes + * @param erasedParityIndexes + */ + protected void prepare(int numDataUnits, int numParityUnits, + int[] erasedDataIndexes, int[] erasedParityIndexes) { + prepare(null, numDataUnits, numParityUnits, erasedDataIndexes, + erasedParityIndexes, false); + } + + /** + * 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 + */ + protected void compareAndVerify(ECChunk[] erasedChunks, + ECChunk[] recoveredChunks) { + byte[][] erased = toArrays(erasedChunks); + byte[][] recovered = toArrays(recoveredChunks); + boolean result = Arrays.deepEquals(erased, recovered); + assertTrue("Decoding and comparing failed.", result); + } + + /** + * Adjust and return erased indexes altogether, including erased data indexes + * and parity indexes. + * @return erased indexes altogether + */ + protected int[] getErasedIndexesForDecoding() { + int[] erasedIndexesForDecoding = + new int[erasedParityIndexes.length + erasedDataIndexes.length]; + + int idx = 0; + + for (int i = 0; i < erasedParityIndexes.length; i++) { + erasedIndexesForDecoding[idx ++] = erasedParityIndexes[i]; + } + + for (int i = 0; i < erasedDataIndexes.length; i++) { + erasedIndexesForDecoding[idx ++] = erasedDataIndexes[i] + numParityUnits; + } + + return erasedIndexesForDecoding; + } + + /** + * Return input chunks for decoding, which is parityChunks + dataChunks. + * @param dataChunks + * @param parityChunks + * @return + */ + protected ECChunk[] prepareInputChunksForDecoding(ECChunk[] dataChunks, + ECChunk[] parityChunks) { + ECChunk[] inputChunks = new ECChunk[numParityUnits + numDataUnits]; + + int idx = 0; + for (int i = 0; i < numParityUnits; i++) { + inputChunks[idx ++] = parityChunks[i]; + } + for (int i = 0; i < numDataUnits; i++) { + inputChunks[idx ++] = dataChunks[i]; + } + + return inputChunks; + } + + /** + * Erase some data chunks to test the recovering of them. As they're erased, + * we don't need to read them and will not have the buffers at all, so just + * set them as null. + * @param dataChunks + * @param parityChunks + * @return clone of erased chunks + */ + protected ECChunk[] backupAndEraseChunks(ECChunk[] dataChunks, + ECChunk[] parityChunks) { + ECChunk[] toEraseChunks = new ECChunk[erasedParityIndexes.length + + erasedDataIndexes.length]; + + int idx = 0; + + for (int i = 0; i < erasedParityIndexes.length; i++) { + toEraseChunks[idx ++] = parityChunks[erasedParityIndexes[i]]; + parityChunks[erasedParityIndexes[i]] = null; + } + + for (int i = 0; i < erasedDataIndexes.length; i++) { + toEraseChunks[idx ++] = dataChunks[erasedDataIndexes[i]]; + dataChunks[erasedDataIndexes[i]] = null; + } + + return toEraseChunks; + } + + /** + * Erase data from the specified chunks, just setting them as null. + * @param chunks + */ + protected void eraseDataFromChunks(ECChunk[] chunks) { + for (int i = 0; i < chunks.length; i++) { + chunks[i] = null; + } + } + + /** + * Clone chunks along with copying the associated data. It respects how the + * chunk buffer is allocated, direct or non-direct. It avoids affecting the + * original chunk buffers. + * @param chunks + * @return + */ + protected ECChunk[] cloneChunksWithData(ECChunk[] chunks) { + ECChunk[] results = new ECChunk[chunks.length]; + for (int i = 0; i < chunks.length; i++) { + results[i] = cloneChunkWithData(chunks[i]); + } + + return results; + } + + /** + * Clone chunk along with copying the associated data. It respects how the + * chunk buffer is allocated, direct or non-direct. It avoids affecting the + * original chunk. + * @param chunk + * @return a new chunk + */ + protected ECChunk cloneChunkWithData(ECChunk chunk) { + ByteBuffer srcBuffer = chunk.getBuffer(); + + byte[] bytesArr = new byte[srcBuffer.remaining()]; + srcBuffer.mark(); + srcBuffer.get(bytesArr, 0, bytesArr.length); + srcBuffer.reset(); + + ByteBuffer destBuffer = allocateOutputBuffer(bytesArr.length); + int pos = destBuffer.position(); + destBuffer.put(bytesArr); + destBuffer.flip(); + destBuffer.position(pos); + + return new ECChunk(destBuffer); + } + + /** + * Allocate a chunk for output or writing. + * @return + */ + protected ECChunk allocateOutputChunk() { + ByteBuffer buffer = allocateOutputBuffer(chunkSize); + + return new ECChunk(buffer); + } + + /** + * Allocate a buffer for output or writing. It can prepare for two kinds of + * data buffers: one with position as 0, the other with position > 0 + * @return a buffer ready to write chunkSize bytes from current position + */ + protected ByteBuffer allocateOutputBuffer(int bufferLen) { + /** + * When startBufferWithZero, will prepare a buffer as:--------------- + * otherwise, the buffer will be like: ___TO--BE--WRITTEN___, + * and in the beginning, dummy data are prefixed, to simulate a buffer of + * position > 0. + */ + int startOffset = startBufferWithZero ? 0 : 11; // 11 is arbitrary + int allocLen = startOffset + bufferLen + startOffset; + ByteBuffer buffer = allocator.allocate(allocLen); + buffer.limit(startOffset + bufferLen); + fillDummyData(buffer, startOffset); + startBufferWithZero = ! startBufferWithZero; + + return buffer; + } + + /** + * Prepare data chunks for each data unit, by generating random data. + * @return + */ + protected ECChunk[] prepareDataChunksForEncoding() { + if (usingFixedData) { + ECChunk[] chunks = new ECChunk[numDataUnits]; + for (int i = 0; i < chunks.length; i++) { + chunks[i] = makeChunkUsingData(fixedData[i]); + } + return chunks; + } + + return generateDataChunks(); + } + + private ECChunk makeChunkUsingData(byte[] data) { + ECChunk chunk = allocateOutputChunk(); + ByteBuffer buffer = chunk.getBuffer(); + int pos = buffer.position(); + buffer.put(data, 0, chunkSize); + buffer.flip(); + buffer.position(pos); + + return chunk; + } + + private ECChunk[] generateDataChunks() { + ECChunk[] chunks = new ECChunk[numDataUnits]; + for (int i = 0; i < chunks.length; i++) { + chunks[i] = generateDataChunk(); + } + + return chunks; + } + + private void prepareFixedData() { + // We may load test data from a resource, or just generate randomly. + // The generated data will be used across subsequent encode/decode calls. + this.fixedData = new byte[numDataUnits][]; + for (int i = 0; i < numDataUnits; i++) { + fixedData[i] = generateFixedData(baseChunkSize * 2); + } + } + + /** + * Generate data chunk by making random data. + * @return + */ + protected ECChunk generateDataChunk() { + ByteBuffer buffer = allocateOutputBuffer(chunkSize); + int pos = buffer.position(); + buffer.put(generateData(chunkSize)); + buffer.flip(); + buffer.position(pos); + + return new ECChunk(buffer); + } + + /** + * Fill len of dummy data in the buffer at the current position. + * @param buffer + * @param len + */ + protected void fillDummyData(ByteBuffer buffer, int len) { + byte[] dummy = new byte[len]; + RAND.nextBytes(dummy); + buffer.put(dummy); + } + + protected byte[] generateData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) RAND.nextInt(256); + } + return buffer; + } + + protected byte[] generateFixedData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) FIXED_DATA_GENERATOR++; + if (FIXED_DATA_GENERATOR == 256) { + FIXED_DATA_GENERATOR = 0; + } + } + return buffer; + } + + /** + * Prepare parity chunks for encoding, each chunk for each parity unit. + * @return + */ + protected ECChunk[] prepareParityChunksForEncoding() { + ECChunk[] chunks = new ECChunk[numParityUnits]; + for (int i = 0; i < chunks.length; i++) { + chunks[i] = allocateOutputChunk(); + } + + return chunks; + } + + /** + * Prepare output chunks for decoding, each output chunk for each erased + * chunk. + * @return + */ + protected ECChunk[] prepareOutputChunksForDecoding() { + ECChunk[] chunks = new ECChunk[erasedDataIndexes.length + + erasedParityIndexes.length]; + + for (int i = 0; i < chunks.length; i++) { + chunks[i] = allocateOutputChunk(); + } + + return chunks; + } + + /** + * Convert an array of this chunks to an array of byte array. + * Note the chunk buffers are not affected. + * @param chunks + * @return an array of byte array + */ + protected byte[][] toArrays(ECChunk[] chunks) { + byte[][] bytesArr = new byte[chunks.length][]; + + for (int i = 0; i < chunks.length; i++) { + bytesArr[i] = chunks[i].toBytesArray(); + } + + return bytesArr; + } + + /** + * Dump all the settings used in the test case if allowDump is enabled. + */ + protected void dumpSetting() { + if (allowDump) { + StringBuilder sb = new StringBuilder("Erasure coder test settings:\n"); + sb.append(" numDataUnits=").append(numDataUnits); + sb.append(" numParityUnits=").append(numParityUnits); + sb.append(" chunkSize=").append(chunkSize).append("\n"); + + sb.append(" erasedDataIndexes="). + append(Arrays.toString(erasedDataIndexes)); + sb.append(" erasedParityIndexes="). + append(Arrays.toString(erasedParityIndexes)); + sb.append(" usingDirectBuffer=").append(usingDirectBuffer).append("\n"); + + System.out.println(sb.toString()); + } + } + + /** + * Dump chunks prefixed with a header if allowDump is enabled. + * @param header + * @param chunks + */ + protected void dumpChunks(String header, ECChunk[] chunks) { + if (allowDump) { + DumpUtil.dumpChunks(header, chunks); + } + } + + /** + * Make some chunk messy or not correct any more + * @param chunks + */ + protected void corruptSomeChunk(ECChunk[] chunks) { + int idx = new Random().nextInt(chunks.length); + ByteBuffer buffer = chunks[idx].getBuffer(); + if (buffer.hasRemaining()) { + buffer.position(buffer.position() + 1); + } + } +} 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 0000000000000..1d399017234c9 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/TestECSchema.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; + +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() { + int numDataUnits = 6; + int numParityUnits = 3; + String codec = "rs"; + String extraOption = "extraOption"; + String extraOptionValue = "extraOptionValue"; + + Map options = new HashMap(); + 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(extraOption, extraOptionValue); + + ECSchema schema = new ECSchema(options); + System.out.println(schema.toString()); + + assertEquals(numDataUnits, schema.getNumDataUnits()); + assertEquals(numParityUnits, schema.getNumParityUnits()); + assertEquals(codec, schema.getCodecName()); + assertEquals(extraOptionValue, schema.getExtraOptions().get(extraOption)); + } +} 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 0000000000000..738d28e1c4d42 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestErasureCoderBase.java @@ -0,0 +1,297 @@ +/** + * 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.ECChunk; +import org.apache.hadoop.io.erasurecode.TestCoderBase; + +import java.lang.reflect.Constructor; + +/** + * Erasure coder test base with utilities. + */ +public abstract class TestErasureCoderBase extends TestCoderBase { + protected Class encoderClass; + protected Class decoderClass; + + private ErasureCoder encoder; + private ErasureCoder decoder; + + 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; + prepareCoders(); + + /** + * The following runs will use 3 different chunkSize for inputs and outputs, + * to verify the same encoder/decoder can process variable width of data. + */ + performTestCoding(baseChunkSize, true); + performTestCoding(baseChunkSize - 17, false); + performTestCoding(baseChunkSize + 16, true); + } + + private void performTestCoding(int chunkSize, boolean usingSlicedBuffer) { + setChunkSize(chunkSize); + prepareBufferAllocator(usingSlicedBuffer); + + // 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()); + TestBlock[] parityBlocks = (TestBlock[]) blockGroup.getParityBlocks(); + + ErasureCodingStep codingStep; + codingStep = encoder.calculateCoding(blockGroup); + performCodingStep(codingStep); + // Erase specified sources but return copies of them for later comparing + TestBlock[] backupBlocks = backupAndEraseBlocks(clonedDataBlocks, parityBlocks); + + // Decode + blockGroup = new ECBlockGroup(clonedDataBlocks, blockGroup.getParityBlocks()); + codingStep = decoder.calculateCoding(blockGroup); + performCodingStep(codingStep); + + // Compare + compareAndVerify(backupBlocks, 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); + } + } + + private void prepareCoders() { + if (encoder == null) { + encoder = createEncoder(); + } + + if (decoder == null) { + decoder = createDecoder(); + } + } + + /** + * Create the raw erasure encoder to test + * @return + */ + protected ErasureCoder createEncoder() { + ErasureCoder encoder; + try { + Constructor constructor = + (Constructor) + encoderClass.getConstructor(int.class, int.class); + encoder = constructor.newInstance(numDataUnits, numParityUnits); + } catch (Exception e) { + throw new RuntimeException("Failed to create encoder", e); + } + + encoder.setConf(getConf()); + return encoder; + } + + /** + * create the raw erasure decoder to test + * @return + */ + protected ErasureCoder createDecoder() { + ErasureCoder decoder; + try { + Constructor constructor = + (Constructor) + decoderClass.getConstructor(int.class, int.class); + decoder = constructor.newInstance(numDataUnits, numParityUnits); + } catch (Exception e) { + throw new RuntimeException("Failed to create decoder", e); + } + + decoder.setConf(getConf()); + 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); + } + + /** + * Erase blocks to test the recovering of them. Before erasure clone them + * first so could return themselves. + * @param dataBlocks + * @return clone of erased dataBlocks + */ + protected TestBlock[] backupAndEraseBlocks(TestBlock[] dataBlocks, + TestBlock[] parityBlocks) { + TestBlock[] toEraseBlocks = new TestBlock[erasedDataIndexes.length + + erasedParityIndexes.length]; + int idx = 0; + TestBlock block; + + for (int i = 0; i < erasedParityIndexes.length; i++) { + block = parityBlocks[erasedParityIndexes[i]]; + toEraseBlocks[idx ++] = cloneBlockWithData(block); + eraseDataFromBlock(block); + } + + for (int i = 0; i < erasedDataIndexes.length; i++) { + block = dataBlocks[erasedDataIndexes[i]]; + toEraseBlocks[idx ++] = cloneBlockWithData(block); + eraseDataFromBlock(block); + } + + return toEraseBlocks; + } + + /** + * 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 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 TestBlock cloneBlockWithData(TestBlock block) { + ECChunk[] newChunks = cloneChunksWithData(block.chunks); + + return new TestBlock(newChunks); + } + + /** + * Erase data from a block. + */ + protected void eraseDataFromBlock(TestBlock theBlock) { + eraseDataFromChunks(theBlock.chunks); + theBlock.setErased(true); + } +} 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 0000000000000..94f77db4fb305 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestRSErasureCoder.java @@ -0,0 +1,126 @@ +/** + * 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.RSRawErasureCoderFactory; +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_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + testCoding(false); + testCoding(false); + } + + @Test + public void testCodingDirectBufferWithConf_10x4_erasing_d0() { + /** + * This tests if the configuration items work or not. + */ + Configuration conf = new Configuration(); + conf.set(CommonConfigurationKeys.IO_ERASURECODE_CODEC_RS_RAWCODER_KEY, + RSRawErasureCoderFactory.class.getCanonicalName()); + prepare(conf, 10, 4, new int[]{0}, new int[0]); + + testCoding(true); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasing_p1() { + prepare(null, 10, 4, new int[]{}, new int[]{1}); + testCoding(true); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasing_d2() { + prepare(null, 10, 4, new int[] {2}, new int[] {}); + testCoding(true); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); + testCoding(true); + testCoding(true); + } + + @Test + public void testCodingBothBuffers_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ + testCoding(true); + testCoding(false); + testCoding(true); + testCoding(false); + } + + @Test + public void testCodingDirectBuffer_10x4_erasure_of_d2_d4_p0() { + prepare(null, 10, 4, new int[] {2, 4}, new int[] {0}); + testCoding(true); + } + + @Test + public void testCodingDirectBuffer_10x4_erasing_d0_d1_p0_p1() { + prepare(null, 10, 4, new int[] {0, 1}, new int[] {0, 1}); + testCoding(true); + } + + @Test + public void testCodingNoDirectBuffer_3x3_erasing_d0_p0() { + prepare(null, 3, 3, new int[] {0}, new int[] {0}); + testCoding(false); + } + + @Test + public void testCodingDirectBuffer_3x3_erasing_d0_p0() { + prepare(null, 3, 3, new int[] {0}, new int[] {0}); + testCoding(true); + } +} 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 0000000000000..06e0087992871 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/coder/TestXORCoder.java @@ -0,0 +1,64 @@ +/** + * 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.numChunksInBlock = 10; + } + + @Test + public void testCodingNoDirectBuffer_erasing_p0() { + prepare(null, 10, 1, new int[0], new int[] {0}); + + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + testCoding(false); + testCoding(false); + } + + @Test + public void testCodingBothBuffers_erasing_d5() { + prepare(null, 10, 1, new int[]{5}, new int[0]); + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ + testCoding(true); + testCoding(false); + testCoding(true); + testCoding(false); + } +} 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 0000000000000..a35a4dd5c3b86 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoder.java @@ -0,0 +1,118 @@ +/** + * 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 raw Reed-solomon coder implemented in Java. + */ +public class TestRSRawCoder extends TestRSRawCoderBase { + + @Before + public void setup() { + this.encoderClass = RSRawEncoder.class; + this.decoderClass = RSRawDecoder.class; + setAllowDump(false); // Change to true to allow verbose dump for debugging + } + + @Test + public void testCoding_6x3_erasing_all_d() { + prepare(null, 6, 3, new int[]{0, 1, 2}, new int[0], true); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_d0_d2() { + prepare(null, 6, 3, new int[] {0, 2}, new int[]{}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_d0() { + prepare(null, 6, 3, new int[]{0}, new int[0]); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_d2() { + prepare(null, 6, 3, new int[]{2}, new int[]{}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_d0_p0() { + prepare(null, 6, 3, new int[]{0}, new int[]{0}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_all_p() { + prepare(null, 6, 3, new int[0], new int[]{0, 1, 2}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_p0() { + prepare(null, 6, 3, new int[0], new int[]{0}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_p2() { + prepare(null, 6, 3, new int[0], new int[]{2}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasure_p0_p2() { + prepare(null, 6, 3, new int[0], new int[]{0, 2}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_d0_p0_p1() { + prepare(null, 6, 3, new int[]{0}, new int[]{0, 1}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_6x3_erasing_d0_d2_p2() { + prepare(null, 6, 3, new int[]{0, 2}, new int[]{2}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCodingNegative_6x3_erasing_d2_d4() { + prepare(null, 6, 3, new int[]{2, 4}, new int[0]); + testCodingDoMixAndTwice(); + } + + @Test + public void testCodingNegative_6x3_erasing_too_many() { + prepare(null, 6, 3, new int[]{2, 4}, new int[]{0, 1}); + testCodingWithErasingTooMany(); + } + + @Test + public void testCoding_10x4_erasing_d0_p0() { + prepare(null, 10, 4, new int[] {0}, new int[] {0}); + testCodingDoMixAndTwice(); + } +} diff --git a/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java new file mode 100644 index 0000000000000..efde33211a8e6 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRSRawCoderBase.java @@ -0,0 +1,58 @@ +/** + * 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; + +/** + * Test base for raw Reed-solomon coders. + */ +public abstract class TestRSRawCoderBase extends TestRawCoderBase { + + private static int symbolSize = 0; + private static int symbolMax = 0; + + private static int RS_FIXED_DATA_GENERATOR = 0; + + static { + symbolSize = (int) Math.round(Math.log( + RSUtil.GF.getFieldSize()) / Math.log(2)); + symbolMax = (int) Math.pow(2, symbolSize); + } + + @Override + protected byte[] generateData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) RAND.nextInt(symbolMax); + } + return buffer; + } + + @Override + protected byte[] generateFixedData(int len) { + byte[] buffer = new byte[len]; + for (int i = 0; i < buffer.length; i++) { + buffer[i] = (byte) RS_FIXED_DATA_GENERATOR++; + if (RS_FIXED_DATA_GENERATOR == symbolMax) { + RS_FIXED_DATA_GENERATOR = 0; + } + } + return buffer; + } +} 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 new file mode 100644 index 0000000000000..2b7a3c405b286 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestRawCoderBase.java @@ -0,0 +1,232 @@ +/** + * 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.TestCoderBase; +import org.junit.Assert; +import org.junit.Test; + +import java.lang.reflect.Constructor; + +/** + * Raw coder test base with utilities. + */ +public abstract class TestRawCoderBase extends TestCoderBase { + protected Class encoderClass; + protected Class decoderClass; + private RawErasureEncoder encoder; + private RawErasureDecoder decoder; + + /** + * Doing twice to test if the coders can be repeatedly reused. This matters + * as the underlying coding buffers are shared, which may have bugs. + */ + protected void testCodingDoMixAndTwice() { + testCodingDoMixed(); + testCodingDoMixed(); + } + + /** + * Doing in mixed buffer usage model to test if the coders can be repeatedly + * reused with different buffer usage model. This matters as the underlying + * coding buffers are shared, which may have bugs. + */ + protected void testCodingDoMixed() { + testCoding(true); + testCoding(false); + } + + /** + * 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. Use usingDirectBuffer indicate which case to test. + * + * @param usingDirectBuffer + */ + protected void testCoding(boolean usingDirectBuffer) { + this.usingDirectBuffer = usingDirectBuffer; + prepareCoders(); + + /** + * The following runs will use 3 different chunkSize for inputs and outputs, + * to verify the same encoder/decoder can process variable width of data. + */ + performTestCoding(baseChunkSize, true, false, false); + performTestCoding(baseChunkSize - 17, false, false, false); + performTestCoding(baseChunkSize + 16, true, false, false); + } + + /** + * Similar to above, but perform negative cases using bad input for encoding. + * @param usingDirectBuffer + */ + protected void testCodingWithBadInput(boolean usingDirectBuffer) { + this.usingDirectBuffer = usingDirectBuffer; + prepareCoders(); + + try { + performTestCoding(baseChunkSize, false, true, false); + Assert.fail("Encoding test with bad input should fail"); + } catch (Exception e) { + // Expected + } + } + + /** + * Similar to above, but perform negative cases using bad output for decoding. + * @param usingDirectBuffer + */ + protected void testCodingWithBadOutput(boolean usingDirectBuffer) { + this.usingDirectBuffer = usingDirectBuffer; + prepareCoders(); + + try { + performTestCoding(baseChunkSize, false, false, true); + Assert.fail("Decoding test with bad output should fail"); + } catch (Exception e) { + // Expected + } + } + + @Test + public void testCodingWithErasingTooMany() { + try { + testCoding(true); + Assert.fail("Decoding test erasing too many should fail"); + } catch (Exception e) { + // Expected + } + + try { + testCoding(false); + Assert.fail("Decoding test erasing too many should fail"); + } catch (Exception e) { + // Expected + } + } + + private void performTestCoding(int chunkSize, boolean usingSlicedBuffer, + boolean useBadInput, boolean useBadOutput) { + setChunkSize(chunkSize); + prepareBufferAllocator(usingSlicedBuffer); + + dumpSetting(); + + // Generate data and encode + ECChunk[] dataChunks = prepareDataChunksForEncoding(); + if (useBadInput) { + corruptSomeChunk(dataChunks); + } + dumpChunks("Testing data chunks", dataChunks); + + ECChunk[] parityChunks = prepareParityChunksForEncoding(); + + // Backup all the source chunks for later recovering because some coders + // may affect the source data. + ECChunk[] clonedDataChunks = cloneChunksWithData(dataChunks); + + encoder.encode(dataChunks, parityChunks); + dumpChunks("Encoded parity chunks", parityChunks); + + // Backup and erase some chunks + ECChunk[] backupChunks = backupAndEraseChunks(clonedDataChunks, parityChunks); + + // Decode + ECChunk[] inputChunks = prepareInputChunksForDecoding( + clonedDataChunks, parityChunks); + + // Remove unnecessary chunks, allowing only least required chunks to be read. + ensureOnlyLeastRequiredChunks(inputChunks); + + ECChunk[] recoveredChunks = prepareOutputChunksForDecoding(); + if (useBadOutput) { + corruptSomeChunk(recoveredChunks); + } + + dumpChunks("Decoding input chunks", inputChunks); + decoder.decode(inputChunks, getErasedIndexesForDecoding(), recoveredChunks); + dumpChunks("Decoded/recovered chunks", recoveredChunks); + + // Compare + compareAndVerify(backupChunks, recoveredChunks); + } + + private void prepareCoders() { + if (encoder == null) { + encoder = createEncoder(); + } + + if (decoder == null) { + decoder = createDecoder(); + } + } + + private void ensureOnlyLeastRequiredChunks(ECChunk[] inputChunks) { + int leastRequiredNum = numDataUnits; + int erasedNum = erasedDataIndexes.length + erasedParityIndexes.length; + int goodNum = inputChunks.length - erasedNum; + int redundantNum = goodNum - leastRequiredNum; + + for (int i = 0; i < inputChunks.length && redundantNum > 0; i++) { + if (inputChunks[i] != null) { + inputChunks[i] = null; // Setting it null, not needing it actually + redundantNum--; + } + } + } + + /** + * Create the raw erasure encoder to test + * @return + */ + protected RawErasureEncoder createEncoder() { + RawErasureEncoder encoder; + try { + Constructor constructor = + (Constructor) + encoderClass.getConstructor(int.class, int.class); + encoder = constructor.newInstance(numDataUnits, numParityUnits); + } catch (Exception e) { + throw new RuntimeException("Failed to create encoder", e); + } + + encoder.setConf(getConf()); + return encoder; + } + + /** + * create the raw erasure decoder to test + * @return + */ + protected RawErasureDecoder createDecoder() { + RawErasureDecoder decoder; + try { + Constructor constructor = + (Constructor) + decoderClass.getConstructor(int.class, int.class); + decoder = constructor.newInstance(numDataUnits, numParityUnits); + } catch (Exception e) { + throw new RuntimeException("Failed to create decoder", e); + } + + decoder.setConf(getConf()); + return decoder; + } +} 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 0000000000000..48463ad1fd347 --- /dev/null +++ b/hadoop-common-project/hadoop-common/src/test/java/org/apache/hadoop/io/erasurecode/rawcoder/TestXORRawCoder.java @@ -0,0 +1,66 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.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; + } + + @Test + public void testCoding_10x1_erasing_d0() { + prepare(null, 10, 1, new int[] {0}, new int[0]); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_10x1_erasing_p0() { + prepare(null, 10, 1, new int[0], new int[] {0}); + testCodingDoMixAndTwice(); + } + + @Test + public void testCoding_10x1_erasing_d5() { + prepare(null, 10, 1, new int[]{5}, new int[0]); + testCodingDoMixAndTwice(); + } + + @Test + public void testCodingNegative_10x1_erasing_too_many() { + prepare(null, 10, 1, new int[]{2}, new int[]{0}); + testCodingWithErasingTooMany(); + } + + @Test + public void testCodingNegative_10x1_erasing_d5() { + prepare(null, 10, 1, new int[]{5}, new int[0]); + testCodingWithBadInput(true); + testCodingWithBadOutput(false); + testCodingWithBadInput(true); + testCodingWithBadOutput(false); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml index 68bd289e79469..63c16d1374621 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml +++ b/hadoop-hdfs-project/hadoop-hdfs-client/pom.xml @@ -96,6 +96,7 @@ http://maven.apache.org/xsd/maven-4.0.0.xsd"> hdfs.proto encryption.proto inotify.proto + erasurecoding.proto ${project.build.directory}/generated-sources/java diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java index aa3e8ba990906..8f988afaa4650 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReader.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs; +import java.io.Closeable; import java.io.IOException; import java.util.EnumSet; @@ -24,13 +25,14 @@ import org.apache.hadoop.fs.ByteBufferReadable; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.hdfs.shortcircuit.ClientMmap; +import org.apache.hadoop.util.DataChecksum; /** * A BlockReader is responsible for reading a single block * from a single datanode. */ @InterfaceAudience.Private -public interface BlockReader extends ByteBufferReadable { +public interface BlockReader extends ByteBufferReadable, Closeable { /* same interface as inputStream java.io.InputStream#read() @@ -62,6 +64,7 @@ public interface BlockReader extends ByteBufferReadable { * * @throws IOException */ + @Override // java.io.Closeable void close() throws IOException; /** @@ -99,4 +102,9 @@ public interface BlockReader extends ByteBufferReadable { * supported. */ ClientMmap getClientMmap(EnumSet opts); + + /** + * @return The DataChecksum used by the read block + */ + DataChecksum getDataChecksum(); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java index 62e7af6322f68..2575f10fbbcc7 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocal.java @@ -751,4 +751,9 @@ void forceAnchorable() { void forceUnanchorable() { replica.getSlot().makeUnanchorable(); } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java index 523528761db33..c3d7202e6d4a3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/BlockReaderLocalLegacy.java @@ -735,4 +735,9 @@ public boolean isShortCircuit() { public ClientMmap getClientMmap(EnumSet opts) { return null; } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java index 92d117cc7a978..be346a4694158 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSClient.java @@ -157,6 +157,7 @@ import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MD5Hash; import org.apache.hadoop.io.Text; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.io.retry.LossyRetryInvocationHandler; import org.apache.hadoop.ipc.RPC; import org.apache.hadoop.ipc.RemoteException; @@ -229,6 +230,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 int smallBufferSize; public DfsClientConf getConf() { @@ -364,8 +366,12 @@ public DFSClient(URI nameNodeUri, ClientProtocol rpcNamenode, dfsClientConf); if (dfsClientConf.getHedgedReadThreadpoolSize() > 0) { - this.initThreadsNumForHedgedReads(dfsClientConf.getHedgedReadThreadpoolSize()); + this.initThreadsNumForHedgedReads(dfsClientConf. + getHedgedReadThreadpoolSize()); } + + this.initThreadsNumForStripedReads(dfsClientConf. + getStripedReadThreadpoolSize()); this.saslClient = new SaslDataTransferClient( conf, DataTransferSaslUtil.getSaslPropertiesResolver(conf), TrustedChannelResolver.getInstance(conf), nnFallbackToSimpleAuth); @@ -1052,7 +1058,17 @@ public DFSInputStream open(String src, int buffersize, boolean verifyChecksum) // Get block info from namenode TraceScope scope = newPathTraceScope("newDFSInputStream", src); try { - return new DFSInputStream(this, src, verifyChecksum, null); + LocatedBlocks locatedBlocks = getLocatedBlocks(src, 0); + if (locatedBlocks != null) { + ErasureCodingPolicy ecPolicy = locatedBlocks.getErasureCodingPolicy(); + if (ecPolicy != null) { + return new DFSStripedInputStream(this, src, verifyChecksum, ecPolicy, + locatedBlocks); + } + return new DFSInputStream(this, src, verifyChecksum, locatedBlocks); + } else { + throw new IOException("Cannot open filename " + src); + } } finally { scope.close(); } @@ -1184,7 +1200,7 @@ public DFSOutputStream create(String src, Progressable progress, int buffersize, ChecksumOpt checksumOpt) throws IOException { - return create(src, permission, flag, createParent, replication, blockSize, + return create(src, permission, flag, createParent, replication, blockSize, progress, buffersize, checksumOpt, null); } @@ -2879,6 +2895,22 @@ public RemoteIterator listEncryptionZones() return new EncryptionZoneIterator(namenode, tracer); } + + public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) + throws IOException { + checkOpen(); + TraceScope scope = newPathTraceScope("setErasureCodingPolicy", src); + try { + namenode.setErasureCodingPolicy(src, ecPolicy); + } 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 flag) throws IOException { checkOpen(); @@ -2991,6 +3023,16 @@ public void checkAccess(String src, FsAction mode) throws IOException { } } + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { + checkOpen(); + TraceScope scope = tracer.newScope("getErasureCodingPolicies"); + try { + return namenode.getErasureCodingPolicies(); + } finally { + scope.close(); + } + } + public DFSInotifyEventInputStream getInotifyEventStream() throws IOException { checkOpen(); return new DFSInotifyEventInputStream(namenode, tracer); @@ -3066,10 +3108,51 @@ public void rejectedExecution(Runnable runnable, } } + /** + * 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(), + 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); + } + } + } + ThreadPoolExecutor getHedgedReadsThreadPool() { 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; @@ -3129,6 +3212,28 @@ TraceScope newSrcDstTraceScope(String description, String src, String dst) { return scope; } + /** + * Get the erasure coding policy information for the specified path + * + * @param src path to get the information for + * @return Returns the policy information if file or directory on the path is + * erasure coded, null otherwise + * @throws IOException + */ + + public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException { + checkOpen(); + TraceScope scope = newPathTraceScope("getErasureCodingPolicy", src); + try { + return namenode.getErasureCodingPolicy(src); + } catch (RemoteException re) { + throw re.unwrapRemoteException(FileNotFoundException.class, + AccessControlException.class, UnresolvedPathException.class); + } finally { + scope.close(); + } + } + Tracer getTracer() { return tracer; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java index 7101753b25600..d9f409c5a2799 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSInputStream.java @@ -44,7 +44,6 @@ 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; @@ -1060,9 +1059,7 @@ protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block, } } if (chosenNode == null) { - DFSClient.LOG.warn("No live nodes contain block " + block.getBlock() + - " after checking nodes = " + Arrays.toString(nodes) + - ", ignoredNodes = " + ignoredNodes); + reportLostBlock(block, ignoredNodes); return null; } final String dnAddr = @@ -1074,6 +1071,17 @@ protected DNAddrPair getBestNodeDNAddrPair(LocatedBlock block, return new DNAddrPair(chosenNode, targetAddr, storageType); } + /** + * Warn the user of a lost block + */ + protected void reportLostBlock(LocatedBlock lostBlock, + Collection ignoredNodes) { + DatanodeInfo[] nodes = lostBlock.getLocations(); + DFSClient.LOG.warn("No live nodes contain block " + lostBlock.getBlock() + + " after checking nodes = " + Arrays.toString(nodes) + + ", ignoredNodes = " + ignoredNodes); + } + private static String getBestNodeDNAddrPairErrorString( DatanodeInfo nodes[], AbstractMap deadNodes, Collection ignoredNodes) { @@ -1141,42 +1149,26 @@ public ByteBuffer call() throws Exception { }; } - /** - * Used when reading contiguous blocks - */ - private void actualGetFromOneDataNode(final DNAddrPair datanode, - LocatedBlock block, final long start, final long end, byte[] buf, - int offset, Map> 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 located block containing the requested data - * @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 datanode the datanode from which to read data + * @param block the located block containing the requested data + * @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 offset the offset in buf * @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, + void actualGetFromOneDataNode(final DNAddrPair datanode, LocatedBlock block, + final long startInBlk, final long endInBlk, byte[] buf, int offset, Map> 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() @@ -1188,13 +1180,11 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, DFSClientFaultInjector.get().fetchFromDatanodeException(); reader = getBlockReader(block, startInBlk, len, datanode.addr, datanode.storageType, datanode.info); - for (int i = 0; i < offsets.length; i++) { - int nread = reader.readAll(buf, offsets[i], lengths[i]); - updateReadStatistics(readStatistics, nread, reader); - if (nread != lengths[i]) { - throw new IOException("truncated return from reader.read(): " + - "excpected " + lengths[i] + ", got " + nread); - } + int nread = reader.readAll(buf, offset, len); + updateReadStatistics(readStatistics, nread, reader); + if (nread != len) { + throw new IOException("truncated return from reader.read(): " + + "excpected " + len + ", got " + nread); } DFSClientFaultInjector.get().readFromDatanodeDelay(); return; @@ -1210,7 +1200,7 @@ void actualGetFromOneDataNode(final DNAddrPair datanode, throw new IOException(msg); } catch (IOException e) { if (e instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { - DFSClient.LOG.info("Will fetch a new encryption key and retry, " + DFSClient.LOG.info("Will fetch a new encryption key and retry, " + "encryption key was invalid when connecting to " + datanode.addr + " : " + e); // The encryption key used is invalid. @@ -1249,24 +1239,6 @@ protected LocatedBlock refreshLocatedBlock(LocatedBlock block) return getBlockAt(block.getStartOffset()); } - /** - * 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}except we start up a second, parallel, * 'hedged' read if the first read is taking longer than configured amount of @@ -1487,7 +1459,7 @@ private int pread(long position, byte[] buffer, int offset, int length) long targetStart = position - blk.getStartOffset(); long bytesToRead = Math.min(remaining, blk.getBlockSize() - targetStart); try { - if (dfsClient.isHedgedReadsEnabled()) { + if (dfsClient.isHedgedReadsEnabled() && !blk.isStriped()) { hedgedFetchBlockByteRange(blk, targetStart, targetStart + bytesToRead - 1, buffer, offset, corruptedBlockMap); } else { diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java index 7a40d73703607..78eaa6ca00d6f 100755 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSOutputStream.java @@ -52,6 +52,7 @@ import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException; import org.apache.hadoop.hdfs.server.namenode.RetryStartFileException; import org.apache.hadoop.hdfs.server.namenode.SafeModeException; import org.apache.hadoop.hdfs.util.ByteArrayManager; @@ -110,7 +111,7 @@ public class DFSOutputStream extends FSOutputSummer protected final int bytesPerChecksum; protected DFSPacket currentPacket = null; - private DataStreamer streamer; + protected DataStreamer streamer; protected int packetSize = 0; // write packet size, not including the header. protected int chunksPerPacket = 0; protected long lastFlushOffset = 0; // offset when flush was invoked @@ -136,7 +137,7 @@ protected DFSPacket createPacket(int packetSize, int chunksPerPkt, long offsetIn } return new DFSPacket(buf, chunksPerPkt, offsetInBlock, seqno, - getChecksumSize(), lastPacketInBlock); + getChecksumSize(), lastPacketInBlock); } @Override @@ -165,7 +166,7 @@ public synchronized DatanodeInfo[] getPipeline() { return value; } - /** + /** * @return the object for computing checksum. * The type is NULL if checksum is not computed. */ @@ -178,7 +179,7 @@ private static DataChecksum getChecksum4Compute(DataChecksum checksum, } return checksum; } - + private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress, HdfsFileStatus stat, DataChecksum checksum) throws IOException { super(getChecksum4Compute(checksum, stat)); @@ -194,7 +195,7 @@ private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress, DFSClient.LOG.debug( "Set non-null progress callback on DFSOutputStream " + src); } - + this.bytesPerChecksum = checksum.getBytesPerChecksum(); if (bytesPerChecksum <= 0) { throw new HadoopIllegalArgumentException( @@ -211,14 +212,17 @@ private DFSOutputStream(DFSClient dfsClient, String src, Progressable progress, /** Construct a new output stream for creating a file. */ protected DFSOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, EnumSet flag, Progressable progress, - DataChecksum checksum, String[] favoredNodes) throws IOException { + DataChecksum checksum, String[] favoredNodes, boolean createStreamer) + throws IOException { this(dfsClient, src, progress, stat, checksum); this.shouldSyncBlock = flag.contains(CreateFlag.SYNC_BLOCK); computePacketChunkSize(dfsClient.getConf().getWritePacketSize(), bytesPerChecksum); - streamer = new DataStreamer(stat, null, dfsClient, src, progress, checksum, - cachingStrategy, byteArrayManager, favoredNodes); + if (createStreamer) { + streamer = new DataStreamer(stat, null, dfsClient, src, progress, + checksum, cachingStrategy, byteArrayManager, favoredNodes); + } } static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, @@ -269,8 +273,14 @@ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, } } 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.getErasureCodingPolicy() != null) { + out = new DFSStripedOutputStream(dfsClient, src, stat, + flag, progress, checksum, favoredNodes); + } else { + out = new DFSOutputStream(dfsClient, src, stat, + flag, progress, checksum, favoredNodes, true); + } out.start(); return out; } finally { @@ -282,7 +292,7 @@ static DFSOutputStream newStreamForCreate(DFSClient dfsClient, String src, private DFSOutputStream(DFSClient dfsClient, String src, EnumSet flags, Progressable progress, LocatedBlock lastBlock, HdfsFileStatus stat, DataChecksum checksum, String[] favoredNodes) - throws IOException { + throws IOException { this(dfsClient, src, progress, stat, checksum); initialFileSize = stat.getLen(); // length of file when opened this.shouldSyncBlock = flags.contains(CreateFlag.SYNC_BLOCK); @@ -350,6 +360,9 @@ static DFSOutputStream newStreamForAppend(DFSClient dfsClient, String src, String[] favoredNodes) throws IOException { TraceScope scope = dfsClient.newPathTraceScope("newStreamForAppend", src); + if(stat.getErasureCodingPolicy() != null) { + 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, favoredNodes); @@ -397,13 +410,13 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, if (currentPacket == null) { currentPacket = createPacket(packetSize, chunksPerPacket, getStreamer() .getBytesCurBlock(), getStreamer().getAndIncCurrentSeqno(), false); - if (DFSClient.LOG.isDebugEnabled()) { - DFSClient.LOG.debug("DFSClient writeChunk allocating new packet seqno=" + + if (LOG.isDebugEnabled()) { + LOG.debug("WriteChunk allocating new packet seqno=" + currentPacket.getSeqno() + ", src=" + src + ", packetSize=" + packetSize + ", chunksPerPacket=" + chunksPerPacket + - ", bytesCurBlock=" + getStreamer().getBytesCurBlock()); + ", bytesCurBlock=" + getStreamer().getBytesCurBlock() + ", " + this); } } @@ -413,7 +426,6 @@ protected synchronized void writeChunk(byte[] b, int offset, int len, getStreamer().incBytesCurBlock(len); // If packet is full, enqueue it for transmission - // if (currentPacket.getNumChunks() == currentPacket.getMaxChunks() || getStreamer().getBytesCurBlock() == blockSize) { enqueueCurrentPacketFull(); @@ -427,8 +439,8 @@ void enqueueCurrentPacket() throws IOException { void enqueueCurrentPacketFull() throws IOException { LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={}," - + " appendChunk={}, {}", currentPacket, src, getStreamer() - .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(), + + " appendChunk={}, {}", currentPacket, src, getStreamer() + .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(), getStreamer()); enqueueCurrentPacket(); adjustChunkBoundary(); @@ -467,7 +479,7 @@ protected void adjustChunkBoundary() { * * @throws IOException */ - protected void endBlock() throws IOException { + void endBlock() throws IOException { if (getStreamer().getBytesCurBlock() == blockSize) { setCurrentPacketToEmpty(); enqueueCurrentPacket(); @@ -475,7 +487,7 @@ protected void endBlock() throws IOException { lastFlushOffset = 0; } } - + /** * Flushes out to all replicas of the block. The data is in the buffers * of the DNs but not necessarily in the DN's OS buffers. @@ -507,16 +519,16 @@ public void hsync() throws IOException { scope.close(); } } - + /** * The expected semantics is all data have flushed out to all replicas * and all replicas have done posix fsync equivalent - ie the OS has * flushed it to the disk device (but the disk may have it in its cache). - * + * * Note that only the current block is flushed to the disk device. * To guarantee durable sync across block boundaries the stream should * be created with {@link CreateFlag#SYNC_BLOCK}. - * + * * @param syncFlags * Indicate the semantic of the sync. Currently used to specify * whether or not to update the block length in NameNode. @@ -533,7 +545,7 @@ public void hsync(EnumSet syncFlags) throws IOException { /** * Flush/Sync buffered data to DataNodes. - * + * * @param isSync * Whether or not to require all replicas to flush data to the disk * device @@ -679,7 +691,7 @@ public synchronized int getNumCurrentReplicas() throws IOException { /** * Note that this is not a public API; * use {@link HdfsDataOutputStream#getCurrentBlockReplication()} instead. - * + * * @return the number of valid replicas of the current block */ public synchronized int getCurrentBlockReplication() throws IOException { @@ -694,7 +706,7 @@ public synchronized int getCurrentBlockReplication() throws IOException { } return currentNodes.length; } - + /** * Waits till all existing data is flushed and confirmations * received from datanodes. @@ -718,7 +730,7 @@ protected void flushInternal() throws IOException { protected synchronized void start() { getStreamer().start(); } - + /** * Aborts this output stream and releases any system * resources associated with this stream. @@ -756,7 +768,7 @@ protected void closeThreads(boolean force) throws IOException { setClosed(); } } - + /** * Closes this output stream and releases any system * resources associated with this stream. @@ -887,7 +899,7 @@ public void setDropBehind(Boolean dropBehind) throws IOException { do { prevStrategy = this.cachingStrategy.get(); nextStrategy = new CachingStrategy.Builder(prevStrategy). - setDropBehind(dropBehind).build(); + setDropBehind(dropBehind).build(); } while (!this.cachingStrategy.compareAndSet(prevStrategy, nextStrategy)); } @@ -914,4 +926,57 @@ String getSrc() { protected DataStreamer getStreamer() { return streamer; } + + @Override + public String toString() { + return getClass().getSimpleName() + ":" + streamer; + } + + static LocatedBlock addBlock(DatanodeInfo[] excludedNodes, DFSClient dfsClient, + String src, ExtendedBlock prevBlock, long fileId, String[] favoredNodes) + throws IOException { + final DfsClientConf conf = dfsClient.getConf(); + int retries = conf.getNumBlockWriteLocateFollowingRetry(); + long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs(); + long localstart = Time.monotonicNow(); + while (true) { + try { + return dfsClient.namenode.addBlock(src, dfsClient.clientName, prevBlock, + excludedNodes, fileId, favoredNodes); + } catch (RemoteException e) { + IOException ue = e.unwrapRemoteException(FileNotFoundException.class, + AccessControlException.class, + NSQuotaExceededException.class, + DSQuotaExceededException.class, + QuotaByStorageTypeExceededException.class, + UnresolvedPathException.class); + if (ue != e) { + throw ue; // no need to retry these exceptions + } + if (NotReplicatedYetException.class.getName().equals(e.getClassName())) { + if (retries == 0) { + throw e; + } else { + --retries; + LOG.info("Exception while adding a block", e); + long elapsed = Time.monotonicNow() - localstart; + if (elapsed > 5000) { + LOG.info("Waiting for replication for " + (elapsed / 1000) + + " seconds"); + } + try { + LOG.warn("NotReplicatedYetException sleeping " + src + + " retries left " + retries); + Thread.sleep(sleeptime); + sleeptime *= 2; + } catch (InterruptedException ie) { + LOG.warn("Caught exception", ie); + } + } + } else { + throw e; + } + } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java index 9a8ca6fc10c01..191691bf784ee 100755 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSPacket.java @@ -20,6 +20,7 @@ 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; @@ -38,7 +39,7 @@ ****************************************************************/ @InterfaceAudience.Private -class DFSPacket { +public class DFSPacket { public static final long HEART_BEAT_SEQNO = -1L; private static SpanId[] EMPTY = new SpanId[0]; private final long seqno; // sequence number of buffer in block @@ -81,7 +82,7 @@ class DFSPacket { * @param checksumSize the size of checksum * @param lastPacketInBlock if this is the last packet */ - DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno, + public DFSPacket(byte[] buf, int chunksPerPkt, long offsetInBlock, long seqno, int checksumSize, boolean lastPacketInBlock) { this.lastPacketInBlock = lastPacketInBlock; this.numChunks = 0; @@ -115,6 +116,19 @@ synchronized void writeData(byte[] inarray, int off, int len) dataPos += len; } + public 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 * @@ -123,7 +137,7 @@ synchronized void writeData(byte[] inarray, int off, int len) * @param len the length of checksums to write * @throws ClosedChannelException */ - synchronized void writeChecksum(byte[] inarray, int off, int len) + public synchronized void writeChecksum(byte[] inarray, int off, int len) throws ClosedChannelException { checkBuffer(); if (len == 0) { @@ -142,7 +156,7 @@ synchronized void writeChecksum(byte[] inarray, int off, int len) * @param stm * @throws IOException */ - synchronized void writeTo(DataOutputStream stm) throws IOException { + public synchronized void writeTo(DataOutputStream stm) throws IOException { checkBuffer(); final int dataLen = dataPos - dataStart; @@ -224,7 +238,7 @@ boolean isHeartbeatPacket() { * * @return true if the packet is the last packet */ - boolean isLastPacketInBlock(){ + boolean isLastPacketInBlock() { return lastPacketInBlock; } @@ -233,7 +247,7 @@ boolean isLastPacketInBlock(){ * * @return the sequence number of this packet */ - long getSeqno(){ + long getSeqno() { return seqno; } @@ -242,14 +256,14 @@ long getSeqno(){ * * @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++; } @@ -258,7 +272,7 @@ synchronized void incNumChunks(){ * * @return the maximum number of packets */ - int getMaxChunks(){ + int getMaxChunks() { return maxChunks; } @@ -267,7 +281,7 @@ int getMaxChunks(){ * * @param syncBlock if to sync block */ - synchronized void setSyncBlock(boolean syncBlock){ + synchronized void setSyncBlock(boolean syncBlock) { this.syncBlock = syncBlock; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java new file mode 100644 index 0000000000000..69105a0e3733f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedInputStream.java @@ -0,0 +1,972 @@ +/** + * 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.base.Preconditions; +import org.apache.hadoop.fs.ChecksumException; +import org.apache.hadoop.fs.ReadOption; +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.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.protocol.datatransfer.InvalidEncryptionKeyException; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.ByteBufferPool; + +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.AlignedStripe; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunk; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; + +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; + +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.util.DirectBufferPool; + +import java.io.EOFException; +import java.io.IOException; +import java.io.InterruptedIOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.EnumSet; +import java.util.List; +import java.util.Set; +import java.util.Collection; +import java.util.Map; +import java.util.HashMap; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ConcurrentHashMap; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Callable; +import java.util.concurrent.Future; + +/** + * DFSStripedInputStream reads from striped block groups + */ +public class DFSStripedInputStream extends DFSInputStream { + + private static class ReaderRetryPolicy { + private int fetchEncryptionKeyTimes = 1; + private int fetchTokenTimes = 1; + + void refetchEncryptionKey() { + fetchEncryptionKeyTimes--; + } + + void refetchToken() { + fetchTokenTimes--; + } + + boolean shouldRefetchEncryptionKey() { + return fetchEncryptionKeyTimes > 0; + } + + boolean shouldRefetchToken() { + return fetchTokenTimes > 0; + } + } + + /** Used to indicate the buffered data's range in the block group */ + private static class StripeRange { + /** start offset in the block group (inclusive) */ + final long offsetInBlock; + /** length of the stripe range */ + final long length; + + StripeRange(long offsetInBlock, long length) { + Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0); + this.offsetInBlock = offsetInBlock; + this.length = length; + } + + boolean include(long pos) { + return pos >= offsetInBlock && pos < offsetInBlock + length; + } + } + + private static class BlockReaderInfo { + final BlockReader reader; + final DatanodeInfo datanode; + /** + * when initializing block readers, their starting offsets are set to the same + * number: the smallest internal block offsets among all the readers. This is + * because it is possible that for some internal blocks we have to read + * "backwards" for decoding purpose. We thus use this offset array to track + * offsets for all the block readers so that we can skip data if necessary. + */ + long blockReaderOffset; + /** + * We use this field to indicate whether we should use this reader. In case + * we hit any issue with this reader, we set this field to true and avoid + * using it for the next stripe. + */ + boolean shouldSkip = false; + + BlockReaderInfo(BlockReader reader, DatanodeInfo dn, long offset) { + this.reader = reader; + this.datanode = dn; + this.blockReaderOffset = offset; + } + + void setOffset(long offset) { + this.blockReaderOffset = offset; + } + + void skip() { + this.shouldSkip = true; + } + } + + private static final DirectBufferPool bufferPool = new DirectBufferPool(); + + private final BlockReaderInfo[] blockReaders; + private final int cellSize; + private final short dataBlkNum; + private final short parityBlkNum; + private final int groupSize; + /** the buffer for a complete stripe */ + private ByteBuffer curStripeBuf; + private ByteBuffer parityBuf; + private final ErasureCodingPolicy ecPolicy; + private final RawErasureDecoder decoder; + + /** + * indicate the start/end offset of the current buffered stripe in the + * block group + */ + private StripeRange curStripeRange; + private final CompletionService readingService; + + /** + * When warning the user of a lost block in striping mode, we remember the + * dead nodes we've logged. All other striping blocks on these nodes can be + * considered lost too, and we don't want to log a warning for each of them. + * This is to prevent the log from being too verbose. Refer to HDFS-8920. + * + * To minimize the overhead, we only store the datanodeUuid in this set + */ + private final Set warnedNodes = Collections.newSetFromMap( + new ConcurrentHashMap()); + + DFSStripedInputStream(DFSClient dfsClient, String src, + boolean verifyChecksum, ErasureCodingPolicy ecPolicy, + LocatedBlocks locatedBlocks) throws IOException { + super(dfsClient, src, verifyChecksum, locatedBlocks); + + assert ecPolicy != null; + this.ecPolicy = ecPolicy; + this.cellSize = ecPolicy.getCellSize(); + dataBlkNum = (short) ecPolicy.getNumDataUnits(); + parityBlkNum = (short) ecPolicy.getNumParityUnits(); + groupSize = dataBlkNum + parityBlkNum; + blockReaders = new BlockReaderInfo[groupSize]; + curStripeRange = new StripeRange(0, 0); + readingService = + new ExecutorCompletionService<>(dfsClient.getStripedReadsThreadPool()); + decoder = CodecUtil.createRSRawDecoder(dfsClient.getConfiguration(), + dataBlkNum, parityBlkNum); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("Creating an striped input stream for file " + src); + } + } + + private void resetCurStripeBuffer() { + if (curStripeBuf == null) { + curStripeBuf = bufferPool.getBuffer(cellSize * dataBlkNum); + } + curStripeBuf.clear(); + curStripeRange = new StripeRange(0, 0); + } + + private ByteBuffer getParityBuffer() { + if (parityBuf == null) { + parityBuf = bufferPool.getBuffer(cellSize * parityBlkNum); + } + parityBuf.clear(); + return parityBuf; + } + + /** + * When seeking into a new block group, create blockReader for each internal + * block in the group. + */ + private synchronized void blockSeekTo(long target) throws IOException { + if (target >= getFileLength()) { + throw new IOException("Attempted to read past end of file"); + } + + // Will be getting a new BlockReader. + closeCurrentBlockReaders(); + + // Compute desired striped block group + LocatedStripedBlock targetBlockGroup = getBlockGroupAt(target); + // Update current position + this.pos = target; + this.blockEnd = targetBlockGroup.getStartOffset() + + targetBlockGroup.getBlockSize() - 1; + currentLocatedBlock = targetBlockGroup; + } + + @Override + public synchronized void close() throws IOException { + super.close(); + if (curStripeBuf != null) { + bufferPool.returnBuffer(curStripeBuf); + curStripeBuf = null; + } + if (parityBuf != null) { + bufferPool.returnBuffer(parityBuf); + parityBuf = null; + } + } + + /** + * Extend the super method with the logic of switching between cells. + * When reaching the end of a cell, proceed to the next cell and read it + * with the next blockReader. + */ + @Override + protected void closeCurrentBlockReaders() { + resetCurStripeBuffer(); + if (blockReaders == null || blockReaders.length == 0) { + return; + } + for (int i = 0; i < groupSize; i++) { + closeReader(blockReaders[i]); + blockReaders[i] = null; + } + blockEnd = -1; + } + + private void closeReader(BlockReaderInfo readerInfo) { + if (readerInfo != null) { +// IOUtils.cleanup(null, readerInfo.reader); + readerInfo.skip(); + } + } + + private long getOffsetInBlockGroup() { + return getOffsetInBlockGroup(pos); + } + + private long getOffsetInBlockGroup(long pos) { + return pos - currentLocatedBlock.getStartOffset(); + } + + /** + * Read a new stripe covering the current position, and store the data in the + * {@link #curStripeBuf}. + */ + private void readOneStripe( + Map> corruptedBlockMap) + throws IOException { + resetCurStripeBuffer(); + + // compute stripe range based on pos + final long offsetInBlockGroup = getOffsetInBlockGroup(); + final long stripeLen = cellSize * dataBlkNum; + final int stripeIndex = (int) (offsetInBlockGroup / stripeLen); + final int stripeBufOffset = (int) (offsetInBlockGroup % stripeLen); + final int stripeLimit = (int) Math.min(currentLocatedBlock.getBlockSize() + - (stripeIndex * stripeLen), stripeLen); + StripeRange stripeRange = new StripeRange(offsetInBlockGroup, + stripeLimit - stripeBufOffset); + + LocatedStripedBlock blockGroup = (LocatedStripedBlock) currentLocatedBlock; + AlignedStripe[] stripes = StripedBlockUtil.divideOneStripe(ecPolicy, cellSize, + blockGroup, offsetInBlockGroup, + offsetInBlockGroup + stripeRange.length - 1, curStripeBuf); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( + blockGroup, cellSize, dataBlkNum, parityBlkNum); + // read the whole stripe + for (AlignedStripe stripe : stripes) { + // Parse group to get chosen DN location + StripeReader sreader = new StatefulStripeReader(readingService, stripe, + blks, blockReaders, corruptedBlockMap); + sreader.readStripe(); + } + curStripeBuf.position(stripeBufOffset); + curStripeBuf.limit(stripeLimit); + curStripeRange = stripeRange; + } + + private Callable readCells(final BlockReader reader, + final DatanodeInfo datanode, final long currentReaderOffset, + final long targetReaderOffset, final ByteBufferStrategy[] strategies, + final ExtendedBlock currentBlock, + final Map> corruptedBlockMap) { + return new Callable() { + @Override + public Void call() throws Exception { + // reader can be null if getBlockReaderWithRetry failed or + // the reader hit exception before + if (reader == null) { + throw new IOException("The BlockReader is null. " + + "The BlockReader creation failed or the reader hit exception."); + } + Preconditions.checkState(currentReaderOffset <= targetReaderOffset); + if (currentReaderOffset < targetReaderOffset) { + long skipped = reader.skip(targetReaderOffset - currentReaderOffset); + Preconditions.checkState( + skipped == targetReaderOffset - currentReaderOffset); + } + int result = 0; + for (ByteBufferStrategy strategy : strategies) { + result += readToBuffer(reader, datanode, strategy, currentBlock, + corruptedBlockMap); + } + return null; + } + }; + } + + private int readToBuffer(BlockReader blockReader, + DatanodeInfo currentNode, ByteBufferStrategy strategy, + ExtendedBlock currentBlock, + Map> corruptedBlockMap) + throws IOException { + final int targetLength = strategy.buf.remaining(); + int length = 0; + try { + while (length < targetLength) { + int ret = strategy.doRead(blockReader, 0, 0); + if (ret < 0) { + throw new IOException("Unexpected EOS from the reader"); + } + length += ret; + } + return length; + } catch (ChecksumException ce) { + DFSClient.LOG.warn("Found Checksum error for " + + currentBlock + " from " + currentNode + + " at " + ce.getPos()); + // we want to remember which block replicas we have tried + addIntoCorruptedBlockMap(currentBlock, currentNode, + corruptedBlockMap); + throw ce; + } catch (IOException e) { + DFSClient.LOG.warn("Exception while reading from " + + currentBlock + " of " + src + " from " + + currentNode, e); + throw e; + } + } + + /** + * Seek to a new arbitrary location + */ + @Override + public synchronized void seek(long targetPos) throws IOException { + if (targetPos > getFileLength()) { + throw new EOFException("Cannot seek after EOF"); + } + if (targetPos < 0) { + throw new EOFException("Cannot seek to negative offset"); + } + if (closed.get()) { + throw new IOException("Stream is closed!"); + } + if (targetPos <= blockEnd) { + final long targetOffsetInBlk = getOffsetInBlockGroup(targetPos); + if (curStripeRange.include(targetOffsetInBlk)) { + int bufOffset = getStripedBufOffset(targetOffsetInBlk); + curStripeBuf.position(bufOffset); + pos = targetPos; + return; + } + } + pos = targetPos; + blockEnd = -1; + } + + private int getStripedBufOffset(long offsetInBlockGroup) { + final long stripeLen = cellSize * dataBlkNum; + // compute the position in the curStripeBuf based on "pos" + return (int) (offsetInBlockGroup % stripeLen); + } + + @Override + public synchronized boolean seekToNewSource(long targetPos) + throws IOException { + return false; + } + + @Override + protected synchronized int readWithStrategy(ReaderStrategy strategy, + int off, int len) throws IOException { + dfsClient.checkOpen(); + if (closed.get()) { + throw new IOException("Stream closed"); + } + Map> corruptedBlockMap = + new ConcurrentHashMap<>(); + if (pos < getFileLength()) { + try { + if (pos > blockEnd) { + blockSeekTo(pos); + } + int realLen = (int) Math.min(len, (blockEnd - pos + 1L)); + synchronized (infoLock) { + if (locatedBlocks.isLastBlockComplete()) { + realLen = (int) Math.min(realLen, + locatedBlocks.getFileLength() - pos); + } + } + + /** Number of bytes already read into buffer */ + int result = 0; + while (result < realLen) { + if (!curStripeRange.include(getOffsetInBlockGroup())) { + readOneStripe(corruptedBlockMap); + } + int ret = copyToTargetBuf(strategy, off + result, realLen - result); + result += ret; + pos += ret; + } + if (dfsClient.stats != null) { + dfsClient.stats.incrementBytesRead(result); + } + return result; + } finally { + // Check if need to report block replicas corruption either read + // was successful or ChecksumException occured. + reportCheckSumFailure(corruptedBlockMap, + currentLocatedBlock.getLocations().length); + } + } + return -1; + } + + /** + * Copy the data from {@link #curStripeBuf} into the given buffer + * @param strategy the ReaderStrategy containing the given buffer + * @param offset the offset of the given buffer. Used only when strategy is + * a ByteArrayStrategy + * @param length target length + * @return number of bytes copied + */ + private int copyToTargetBuf(ReaderStrategy strategy, int offset, int length) { + final long offsetInBlk = getOffsetInBlockGroup(); + int bufOffset = getStripedBufOffset(offsetInBlk); + curStripeBuf.position(bufOffset); + return strategy.copyFrom(curStripeBuf, offset, + Math.min(length, curStripeBuf.remaining())); + } + + /** + * The super method {@link DFSInputStream#refreshLocatedBlock} refreshes + * cached LocatedBlock by executing {@link DFSInputStream#getBlockAt} again. + * This method extends the logic by first remembering the index of the + * internal block, and re-parsing the refreshed block group with the same + * index. + */ + @Override + protected LocatedBlock refreshLocatedBlock(LocatedBlock block) + throws IOException { + int idx = StripedBlockUtil.getBlockIndex(block.getBlock().getLocalBlock()); + LocatedBlock lb = getBlockGroupAt(block.getStartOffset()); + // 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("refreshLocatedBlock for striped blocks, offset=" + + block.getStartOffset() + ". Obtained block " + lb + ", idx=" + idx); + } + return StripedBlockUtil.constructInternalBlock( + lsb, i, cellSize, dataBlkNum, idx); + } + + private LocatedStripedBlock getBlockGroupAt(long offset) throws IOException { + LocatedBlock lb = super.getBlockAt(offset); + assert lb instanceof LocatedStripedBlock : "NameNode" + + " should return a LocatedStripedBlock for a striped file"; + return (LocatedStripedBlock)lb; + } + + /** + * Real implementation of pread. + */ + @Override + protected void fetchBlockByteRange(LocatedBlock block, long start, + long end, byte[] buf, int offset, + Map> corruptedBlockMap) + throws IOException { + // Refresh the striped block group + LocatedStripedBlock blockGroup = getBlockGroupAt(block.getStartOffset()); + + AlignedStripe[] stripes = StripedBlockUtil.divideByteRangeIntoStripes( + ecPolicy, cellSize, blockGroup, start, end, buf, offset); + CompletionService readService = new ExecutorCompletionService<>( + dfsClient.getStripedReadsThreadPool()); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup( + blockGroup, cellSize, dataBlkNum, parityBlkNum); + final BlockReaderInfo[] preaderInfos = new BlockReaderInfo[groupSize]; + try { + for (AlignedStripe stripe : stripes) { + // Parse group to get chosen DN location + StripeReader preader = new PositionStripeReader(readService, stripe, + blks, preaderInfos, corruptedBlockMap); + preader.readStripe(); + } + } finally { + for (BlockReaderInfo preaderInfo : preaderInfos) { + closeReader(preaderInfo); + } + } + } + + @Override + protected void reportLostBlock(LocatedBlock lostBlock, + Collection ignoredNodes) { + DatanodeInfo[] nodes = lostBlock.getLocations(); + if (nodes != null && nodes.length > 0) { + List dnUUIDs = new ArrayList<>(); + for (DatanodeInfo node : nodes) { + dnUUIDs.add(node.getDatanodeUuid()); + } + if (!warnedNodes.containsAll(dnUUIDs)) { + DFSClient.LOG.warn(Arrays.toString(nodes) + " are unavailable and " + + "all striping blocks on them are lost. " + + "IgnoredNodes = " + ignoredNodes); + warnedNodes.addAll(dnUUIDs); + } + } else { + super.reportLostBlock(lostBlock, ignoredNodes); + } + } + + /** + * The reader for reading a complete {@link AlignedStripe}. Note that an + * {@link AlignedStripe} may cross multiple stripes with cellSize width. + */ + private abstract class StripeReader { + final Map, Integer> futures = new HashMap<>(); + final AlignedStripe alignedStripe; + final CompletionService service; + final LocatedBlock[] targetBlocks; + final Map> corruptedBlockMap; + final BlockReaderInfo[] readerInfos; + + StripeReader(CompletionService service, AlignedStripe alignedStripe, + LocatedBlock[] targetBlocks, BlockReaderInfo[] readerInfos, + Map> corruptedBlockMap) { + this.service = service; + this.alignedStripe = alignedStripe; + this.targetBlocks = targetBlocks; + this.readerInfos = readerInfos; + this.corruptedBlockMap = corruptedBlockMap; + } + + /** prepare all the data chunks */ + abstract void prepareDecodeInputs(); + + /** prepare the parity chunk and block reader if necessary */ + abstract boolean prepareParityChunk(int index) throws IOException; + + abstract void decode(); + + void updateState4SuccessRead(StripingChunkReadResult result) { + Preconditions.checkArgument( + result.state == StripingChunkReadResult.SUCCESSFUL); + readerInfos[result.index].setOffset(alignedStripe.getOffsetInBlock() + + alignedStripe.getSpanInBlock()); + } + + private void checkMissingBlocks() throws IOException { + if (alignedStripe.missingChunksNum > parityBlkNum) { + clearFutures(futures.keySet()); + throw new IOException(alignedStripe.missingChunksNum + + " missing blocks, the stripe is: " + alignedStripe); + } + } + + /** + * We need decoding. Thus go through all the data chunks and make sure we + * submit read requests for all of them. + */ + private void readDataForDecoding() throws IOException { + prepareDecodeInputs(); + for (int i = 0; i < dataBlkNum; i++) { + Preconditions.checkNotNull(alignedStripe.chunks[i]); + if (alignedStripe.chunks[i].state == StripingChunk.REQUESTED) { + if (!readChunk(targetBlocks[i], i)) { + alignedStripe.missingChunksNum++; + } + } + } + checkMissingBlocks(); + } + + void readParityChunks(int num) throws IOException { + for (int i = dataBlkNum, j = 0; i < dataBlkNum + parityBlkNum && j < num; + i++) { + if (alignedStripe.chunks[i] == null) { + if (prepareParityChunk(i) && readChunk(targetBlocks[i], i)) { + j++; + } else { + alignedStripe.missingChunksNum++; + } + } + } + checkMissingBlocks(); + } + + boolean createBlockReader(LocatedBlock block, int chunkIndex) + throws IOException { + BlockReader reader = null; + final ReaderRetryPolicy retry = new ReaderRetryPolicy(); + DNAddrPair dnInfo = new DNAddrPair(null, null, null); + + while(true) { + try { + // the cached block location might have been re-fetched, so always + // get it from cache. + block = refreshLocatedBlock(block); + targetBlocks[chunkIndex] = block; + + // internal block has one location, just rule out the deadNodes + dnInfo = getBestNodeDNAddrPair(block, null); + if (dnInfo == null) { + break; + } + reader = getBlockReader(block, alignedStripe.getOffsetInBlock(), + block.getBlockSize() - alignedStripe.getOffsetInBlock(), + dnInfo.addr, dnInfo.storageType, dnInfo.info); + } catch (IOException e) { + if (e instanceof InvalidEncryptionKeyException && + retry.shouldRefetchEncryptionKey()) { + DFSClient.LOG.info("Will fetch a new encryption key and retry, " + + "encryption key was invalid when connecting to " + dnInfo.addr + + " : " + e); + dfsClient.clearDataEncryptionKey(); + retry.refetchEncryptionKey(); + } else if (retry.shouldRefetchToken() && + tokenRefetchNeeded(e, dnInfo.addr)) { + fetchBlockAt(block.getStartOffset()); + retry.refetchToken(); + } else { + //TODO: handles connection issues + DFSClient.LOG.warn("Failed to connect to " + dnInfo.addr + " for " + + "block" + block.getBlock(), e); + // re-fetch the block in case the block has been moved + fetchBlockAt(block.getStartOffset()); + addToDeadNodes(dnInfo.info); + } + } + if (reader != null) { + readerInfos[chunkIndex] = new BlockReaderInfo(reader, dnInfo.info, + alignedStripe.getOffsetInBlock()); + return true; + } + } + return false; + } + + private ByteBufferStrategy[] getReadStrategies(StripingChunk chunk) { + if (chunk.byteBuffer != null) { + ByteBufferStrategy strategy = new ByteBufferStrategy(chunk.byteBuffer); + return new ByteBufferStrategy[]{strategy}; + } else { + ByteBufferStrategy[] strategies = + new ByteBufferStrategy[chunk.byteArray.getOffsets().length]; + for (int i = 0; i < strategies.length; i++) { + ByteBuffer buffer = ByteBuffer.wrap(chunk.byteArray.buf(), + chunk.byteArray.getOffsets()[i], chunk.byteArray.getLengths()[i]); + strategies[i] = new ByteBufferStrategy(buffer); + } + return strategies; + } + } + + boolean readChunk(final LocatedBlock block, int chunkIndex) + throws IOException { + final StripingChunk chunk = alignedStripe.chunks[chunkIndex]; + if (block == null) { + chunk.state = StripingChunk.MISSING; + return false; + } + if (readerInfos[chunkIndex] == null) { + if (!createBlockReader(block, chunkIndex)) { + chunk.state = StripingChunk.MISSING; + return false; + } + } else if (readerInfos[chunkIndex].shouldSkip) { + chunk.state = StripingChunk.MISSING; + return false; + } + + chunk.state = StripingChunk.PENDING; + Callable readCallable = readCells(readerInfos[chunkIndex].reader, + readerInfos[chunkIndex].datanode, + readerInfos[chunkIndex].blockReaderOffset, + alignedStripe.getOffsetInBlock(), getReadStrategies(chunk), + block.getBlock(), corruptedBlockMap); + + Future request = service.submit(readCallable); + futures.put(request, chunkIndex); + return true; + } + + /** read the whole stripe. do decoding if necessary */ + void readStripe() throws IOException { + for (int i = 0; i < dataBlkNum; i++) { + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state != StripingChunk.ALLZERO) { + if (!readChunk(targetBlocks[i], i)) { + alignedStripe.missingChunksNum++; + } + } + } + // There are missing block locations at this stage. Thus we need to read + // the full stripe and one more parity block. + if (alignedStripe.missingChunksNum > 0) { + checkMissingBlocks(); + readDataForDecoding(); + // read parity chunks + readParityChunks(alignedStripe.missingChunksNum); + } + // TODO: for a full stripe we can start reading (dataBlkNum + 1) chunks + + // Input buffers for potential decode operation, which remains null until + // first read failure + while (!futures.isEmpty()) { + try { + StripingChunkReadResult r = StripedBlockUtil + .getNextCompletedStripedRead(service, futures, 0); + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("Read task returned: " + r + ", for stripe " + + alignedStripe); + } + StripingChunk returnedChunk = alignedStripe.chunks[r.index]; + Preconditions.checkNotNull(returnedChunk); + Preconditions.checkState(returnedChunk.state == StripingChunk.PENDING); + + if (r.state == StripingChunkReadResult.SUCCESSFUL) { + returnedChunk.state = StripingChunk.FETCHED; + alignedStripe.fetchedChunksNum++; + updateState4SuccessRead(r); + if (alignedStripe.fetchedChunksNum == dataBlkNum) { + clearFutures(futures.keySet()); + break; + } + } else { + returnedChunk.state = StripingChunk.MISSING; + // close the corresponding reader + closeReader(readerInfos[r.index]); + + final int missing = alignedStripe.missingChunksNum; + alignedStripe.missingChunksNum++; + checkMissingBlocks(); + + readDataForDecoding(); + readParityChunks(alignedStripe.missingChunksNum - missing); + } + } catch (InterruptedException ie) { + String err = "Read request interrupted"; + DFSClient.LOG.error(err); + clearFutures(futures.keySet()); + // Don't decode if read interrupted + throw new InterruptedIOException(err); + } + } + + if (alignedStripe.missingChunksNum > 0) { + decode(); + } + } + } + + class PositionStripeReader extends StripeReader { + private byte[][] decodeInputs = null; + + PositionStripeReader(CompletionService service, + AlignedStripe alignedStripe, LocatedBlock[] targetBlocks, + BlockReaderInfo[] readerInfos, + Map> corruptedBlockMap) { + super(service, alignedStripe, targetBlocks, readerInfos, + corruptedBlockMap); + } + + @Override + void prepareDecodeInputs() { + if (decodeInputs == null) { + decodeInputs = StripedBlockUtil.initDecodeInputs(alignedStripe, + dataBlkNum, parityBlkNum); + } + } + + @Override + boolean prepareParityChunk(int index) { + Preconditions.checkState(index >= dataBlkNum && + alignedStripe.chunks[index] == null); + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index, + dataBlkNum, parityBlkNum); + alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]); + alignedStripe.chunks[index].addByteArraySlice(0, + (int) alignedStripe.getSpanInBlock()); + return true; + } + + @Override + void decode() { + StripedBlockUtil.finalizeDecodeInputs(decodeInputs, dataBlkNum, + parityBlkNum, alignedStripe); + StripedBlockUtil.decodeAndFillBuffer(decodeInputs, alignedStripe, + dataBlkNum, parityBlkNum, decoder); + } + } + + class StatefulStripeReader extends StripeReader { + ByteBuffer[] decodeInputs; + + StatefulStripeReader(CompletionService service, + AlignedStripe alignedStripe, LocatedBlock[] targetBlocks, + BlockReaderInfo[] readerInfos, + Map> corruptedBlockMap) { + super(service, alignedStripe, targetBlocks, readerInfos, + corruptedBlockMap); + } + + @Override + void prepareDecodeInputs() { + if (decodeInputs == null) { + decodeInputs = new ByteBuffer[dataBlkNum + parityBlkNum]; + final ByteBuffer cur; + synchronized (DFSStripedInputStream.this) { + cur = curStripeBuf.duplicate(); + } + StripedBlockUtil.VerticalRange range = alignedStripe.range; + for (int i = 0; i < dataBlkNum; i++) { + cur.limit(cur.capacity()); + int pos = (int) (range.offsetInBlock % cellSize + cellSize * i); + cur.position(pos); + cur.limit((int) (pos + range.spanInBlock)); + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i, + dataBlkNum, parityBlkNum); + decodeInputs[decodeIndex] = cur.slice(); + if (alignedStripe.chunks[i] == null) { + alignedStripe.chunks[i] = new StripingChunk( + decodeInputs[decodeIndex]); + } + } + } + } + + @Override + boolean prepareParityChunk(int index) throws IOException { + Preconditions.checkState(index >= dataBlkNum + && alignedStripe.chunks[index] == null); + if (blockReaders[index] != null && blockReaders[index].shouldSkip) { + alignedStripe.chunks[index] = new StripingChunk(StripingChunk.MISSING); + // we have failed the block reader before + return false; + } + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(index, + dataBlkNum, parityBlkNum); + ByteBuffer buf = getParityBuffer().duplicate(); + buf.position(cellSize * decodeIndex); + buf.limit(cellSize * decodeIndex + (int) alignedStripe.range.spanInBlock); + decodeInputs[decodeIndex] = buf.slice(); + alignedStripe.chunks[index] = new StripingChunk(decodeInputs[decodeIndex]); + return true; + } + + @Override + void decode() { + // TODO no copy for data chunks. this depends on HADOOP-12047 + final int span = (int) alignedStripe.getSpanInBlock(); + for (int i = 0; i < alignedStripe.chunks.length; i++) { + final int decodeIndex = StripedBlockUtil.convertIndex4Decode(i, + dataBlkNum, parityBlkNum); + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.ALLZERO) { + for (int j = 0; j < span; j++) { + decodeInputs[decodeIndex].put((byte) 0); + } + decodeInputs[decodeIndex].flip(); + } else if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.FETCHED) { + decodeInputs[decodeIndex].position(0); + decodeInputs[decodeIndex].limit(span); + } + } + int[] decodeIndices = new int[parityBlkNum]; + int pos = 0; + for (int i = 0; i < alignedStripe.chunks.length; i++) { + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.MISSING) { + int decodeIndex = StripedBlockUtil.convertIndex4Decode(i, + dataBlkNum, parityBlkNum); + if (i < dataBlkNum) { + decodeIndices[pos++] = decodeIndex; + } else { + decodeInputs[decodeIndex] = null; + } + } + } + decodeIndices = Arrays.copyOf(decodeIndices, pos); + + final int decodeChunkNum = decodeIndices.length; + ByteBuffer[] outputs = new ByteBuffer[decodeChunkNum]; + for (int i = 0; i < decodeChunkNum; i++) { + outputs[i] = decodeInputs[decodeIndices[i]]; + outputs[i].position(0); + outputs[i].limit((int) alignedStripe.range.spanInBlock); + decodeInputs[decodeIndices[i]] = null; + } + + decoder.decode(decodeInputs, decodeIndices, outputs); + } + } + + /** + * May need online read recovery, zero-copy read doesn't make + * sense, so don't support it. + */ + @Override + public synchronized ByteBuffer read(ByteBufferPool bufferPool, + int maxLength, EnumSet opts) + throws IOException, UnsupportedOperationException { + throw new UnsupportedOperationException( + "Not support enhanced byte buffer access."); + } + + @Override + public synchronized void releaseBuffer(ByteBuffer buffer) { + throw new UnsupportedOperationException( + "Not support enhanced byte buffer access."); + } + + /** A variation to {@link DFSInputStream#cancelAll} */ + private void clearFutures(Collection> futures) { + for (Future future : futures) { + future.cancel(false); + } + futures.clear(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java new file mode 100644 index 0000000000000..bf4e10e3527da --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSStripedOutputStream.java @@ -0,0 +1,953 @@ +/** + * 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.Arrays; +import java.util.Collections; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.HashSet; +import java.util.List; +import java.util.Map; +import java.util.Set; +import java.util.concurrent.BlockingQueue; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.CreateFlag; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +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.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.MultipleIOException; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; +import org.apache.hadoop.util.DataChecksum; +import org.apache.hadoop.util.Progressable; +import org.apache.hadoop.util.Time; + +import com.google.common.base.Preconditions; +import org.apache.htrace.core.TraceScope; + + +/** + * This class supports writing files in striped layout and erasure coded format. + * Each stripe contains a sequence of cells. + */ +@InterfaceAudience.Private +public class DFSStripedOutputStream extends DFSOutputStream { + static class MultipleBlockingQueue { + private final List> queues; + + MultipleBlockingQueue(int numQueue, int queueSize) { + List> list = new ArrayList<>(numQueue); + for (int i = 0; i < numQueue; i++) { + list.add(new LinkedBlockingQueue(queueSize)); + } + queues = Collections.synchronizedList(list); + } + + void offer(int i, T object) { + final boolean b = queues.get(i).offer(object); + Preconditions.checkState(b, "Failed to offer " + object + + " to queue, i=" + i); + } + + T take(int i) throws InterruptedIOException { + try { + return queues.get(i).take(); + } catch(InterruptedException ie) { + throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, ie); + } + } + + T takeWithTimeout(int i) throws InterruptedIOException { + try { + return queues.get(i).poll(100, TimeUnit.MILLISECONDS); + } catch (InterruptedException e) { + throw DFSUtilClient.toInterruptedIOException("take interrupted, i=" + i, e); + } + } + + T poll(int i) { + return queues.get(i).poll(); + } + + T peek(int i) { + return queues.get(i).peek(); + } + + void clear() { + for (BlockingQueue q : queues) { + q.clear(); + } + } + } + + /** Coordinate the communication between the streamers. */ + static class Coordinator { + /** + * The next internal block to write to for each streamers. The + * DFSStripedOutputStream makes the {@link ClientProtocol#addBlock} RPC to + * get a new block group. The block group is split to internal blocks, which + * are then distributed into the queue for streamers to retrieve. + */ + private final MultipleBlockingQueue followingBlocks; + /** + * Used to sync among all the streamers before allocating a new block. The + * DFSStripedOutputStream uses this to make sure every streamer has finished + * writing the previous block. + */ + private final MultipleBlockingQueue endBlocks; + + /** + * The following data structures are used for syncing while handling errors + */ + private final MultipleBlockingQueue newBlocks; + private final Map updateStreamerMap; + private final MultipleBlockingQueue streamerUpdateResult; + + Coordinator(final int numAllBlocks) { + followingBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); + endBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); + newBlocks = new MultipleBlockingQueue<>(numAllBlocks, 1); + updateStreamerMap = Collections.synchronizedMap( + new HashMap(numAllBlocks)); + streamerUpdateResult = new MultipleBlockingQueue<>(numAllBlocks, 1); + } + + MultipleBlockingQueue getFollowingBlocks() { + return followingBlocks; + } + + MultipleBlockingQueue getNewBlocks() { + return newBlocks; + } + + void offerEndBlock(int i, ExtendedBlock block) { + endBlocks.offer(i, block); + } + + void offerStreamerUpdateResult(int i, boolean success) { + streamerUpdateResult.offer(i, success); + } + + boolean takeStreamerUpdateResult(int i) throws InterruptedIOException { + return streamerUpdateResult.take(i); + } + + void updateStreamer(StripedDataStreamer streamer, + boolean success) { + assert !updateStreamerMap.containsKey(streamer); + updateStreamerMap.put(streamer, success); + } + + void clearFailureStates() { + newBlocks.clear(); + updateStreamerMap.clear(); + streamerUpdateResult.clear(); + } + } + + /** Buffers for writing the data and parity cells of a stripe. */ + class CellBuffers { + private final ByteBuffer[] buffers; + private final byte[][] checksumArrays; + + CellBuffers(int numParityBlocks) throws InterruptedException{ + if (cellSize % bytesPerChecksum != 0) { + throw new HadoopIllegalArgumentException("Invalid values: " + + HdfsClientConfigKeys.DFS_BYTES_PER_CHECKSUM_KEY + " (=" + + bytesPerChecksum + ") must divide cell size (=" + cellSize + ")."); + } + + checksumArrays = new byte[numParityBlocks][]; + final int size = getChecksumSize() * (cellSize / bytesPerChecksum); + for (int i = 0; i < checksumArrays.length; i++) { + checksumArrays[i] = new byte[size]; + } + + buffers = new ByteBuffer[numAllBlocks]; + for (int i = 0; i < buffers.length; i++) { + buffers[i] = ByteBuffer.wrap(byteArrayManager.newByteArray(cellSize)); + } + } + + private ByteBuffer[] getBuffers() { + return buffers; + } + + byte[] getChecksumArray(int i) { + return checksumArrays[i - numDataBlocks]; + } + + private int addTo(int i, byte[] b, int off, int len) { + final ByteBuffer buf = buffers[i]; + final int pos = buf.position() + len; + Preconditions.checkState(pos <= cellSize); + buf.put(b, off, len); + return pos; + } + + private void clear() { + for (int i = 0; i< numAllBlocks; i++) { + buffers[i].clear(); + if (i >= numDataBlocks) { + Arrays.fill(buffers[i].array(), (byte) 0); + } + } + } + + private void release() { + for (int i = 0; i < numAllBlocks; i++) { + byteArrayManager.release(buffers[i].array()); + } + } + + private void flipDataBuffers() { + for (int i = 0; i < numDataBlocks; i++) { + buffers[i].flip(); + } + } + } + + private final Coordinator coordinator; + private final CellBuffers cellBuffers; + private final RawErasureEncoder encoder; + private final List streamers; + private final DFSPacket[] currentPackets; // current Packet of each streamer + + /** Size of each striping cell, must be a multiple of bytesPerChecksum */ + private final int cellSize; + private final int numAllBlocks; + private final int numDataBlocks; + private ExtendedBlock currentBlockGroup; + private final String[] favoredNodes; + private final List failedStreamers; + + /** Construct a new output stream for creating a file. */ + DFSStripedOutputStream(DFSClient dfsClient, String src, HdfsFileStatus stat, + EnumSet flag, Progressable progress, + DataChecksum checksum, String[] favoredNodes) + throws IOException { + super(dfsClient, src, stat, flag, progress, checksum, favoredNodes, false); + if (LOG.isDebugEnabled()) { + LOG.debug("Creating DFSStripedOutputStream for " + src); + } + + final ErasureCodingPolicy ecPolicy = stat.getErasureCodingPolicy(); + final int numParityBlocks = ecPolicy.getNumParityUnits(); + cellSize = ecPolicy.getCellSize(); + numDataBlocks = ecPolicy.getNumDataUnits(); + numAllBlocks = numDataBlocks + numParityBlocks; + this.favoredNodes = favoredNodes; + failedStreamers = new ArrayList<>(); + + encoder = CodecUtil.createRSRawEncoder(dfsClient.getConfiguration(), + numDataBlocks, numParityBlocks); + + coordinator = new Coordinator(numAllBlocks); + try { + cellBuffers = new CellBuffers(numParityBlocks); + } catch (InterruptedException ie) { + throw DFSUtilClient.toInterruptedIOException( + "Failed to create cell buffers", ie); + } + + streamers = new ArrayList<>(numAllBlocks); + for (short i = 0; i < numAllBlocks; i++) { + StripedDataStreamer streamer = new StripedDataStreamer(stat, + dfsClient, src, progress, checksum, cachingStrategy, byteArrayManager, + favoredNodes, i, coordinator); + streamers.add(streamer); + } + currentPackets = new DFSPacket[streamers.size()]; + setCurrentStreamer(0); + } + + StripedDataStreamer getStripedDataStreamer(int i) { + return streamers.get(i); + } + + int getCurrentIndex() { + return getCurrentStreamer().getIndex(); + } + + private synchronized StripedDataStreamer getCurrentStreamer() { + return (StripedDataStreamer) streamer; + } + + private synchronized StripedDataStreamer setCurrentStreamer(int newIdx) { + // backup currentPacket for current streamer + if (streamer != null) { + int oldIdx = streamers.indexOf(getCurrentStreamer()); + if (oldIdx >= 0) { + currentPackets[oldIdx] = currentPacket; + } + } + + streamer = getStripedDataStreamer(newIdx); + currentPacket = currentPackets[newIdx]; + adjustChunkBoundary(); + + return getCurrentStreamer(); + } + + /** + * Encode the buffers, i.e. compute parities. + * + * @param buffers data buffers + parity buffers + */ + private static void encode(RawErasureEncoder encoder, int numData, + ByteBuffer[] buffers) { + final ByteBuffer[] dataBuffers = new ByteBuffer[numData]; + final ByteBuffer[] parityBuffers = new ByteBuffer[buffers.length - numData]; + System.arraycopy(buffers, 0, dataBuffers, 0, dataBuffers.length); + System.arraycopy(buffers, numData, parityBuffers, 0, parityBuffers.length); + + encoder.encode(dataBuffers, parityBuffers); + } + + /** + * check all the existing StripedDataStreamer and find newly failed streamers. + * @return The newly failed streamers. + * @throws IOException if less than {@link #numDataBlocks} streamers are still + * healthy. + */ + private Set checkStreamers() throws IOException { + Set newFailed = new HashSet<>(); + for(StripedDataStreamer s : streamers) { + if (!s.isHealthy() && !failedStreamers.contains(s)) { + newFailed.add(s); + } + } + + final int failCount = failedStreamers.size() + newFailed.size(); + if (LOG.isDebugEnabled()) { + LOG.debug("checkStreamers: " + streamers); + LOG.debug("healthy streamer count=" + (numAllBlocks - failCount)); + LOG.debug("original failed streamers: " + failedStreamers); + LOG.debug("newly failed streamers: " + newFailed); + } + if (failCount > (numAllBlocks - numDataBlocks)) { + throw new IOException("Failed: the number of failed blocks = " + + failCount + " > the number of data blocks = " + + (numAllBlocks - numDataBlocks)); + } + return newFailed; + } + + private void handleStreamerFailure(String err, Exception e) + throws IOException { + LOG.warn("Failed: " + err + ", " + this, e); + getCurrentStreamer().getErrorState().setInternalError(); + getCurrentStreamer().close(true); + checkStreamers(); + currentPacket = null; + } + + private void replaceFailedStreamers() { + assert streamers.size() == numAllBlocks; + for (short i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer oldStreamer = getStripedDataStreamer(i); + if (!oldStreamer.isHealthy()) { + StripedDataStreamer streamer = new StripedDataStreamer(oldStreamer.stat, + dfsClient, src, oldStreamer.progress, + oldStreamer.checksum4WriteBlock, cachingStrategy, byteArrayManager, + favoredNodes, i, coordinator); + streamers.set(i, streamer); + currentPackets[i] = null; + if (i == 0) { + this.streamer = streamer; + } + streamer.start(); + } + } + } + + private void waitEndBlocks(int i) throws IOException { + while (getStripedDataStreamer(i).isHealthy()) { + final ExtendedBlock b = coordinator.endBlocks.takeWithTimeout(i); + if (b != null) { + StripedBlockUtil.checkBlocks(currentBlockGroup, i, b); + return; + } + } + } + + private void allocateNewBlock() throws IOException { + if (currentBlockGroup != null) { + for (int i = 0; i < numAllBlocks; i++) { + // sync all the healthy streamers before writing to the new block + waitEndBlocks(i); + } + } + failedStreamers.clear(); + // replace failed streamers + replaceFailedStreamers(); + + if (LOG.isDebugEnabled()) { + LOG.debug("Allocating new block group. The previous block group: " + + currentBlockGroup); + } + + // TODO collect excludedNodes from all the data streamers + final LocatedBlock lb = addBlock(null, dfsClient, src, currentBlockGroup, + fileId, favoredNodes); + assert lb.isStriped(); + if (lb.getLocations().length < numDataBlocks) { + throw new IOException("Failed to get " + numDataBlocks + + " nodes from namenode: blockGroupSize= " + numAllBlocks + + ", blocks.length= " + lb.getLocations().length); + } + // assign the new block to the current block group + currentBlockGroup = lb.getBlock(); + + final LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) lb, cellSize, numDataBlocks, + numAllBlocks - numDataBlocks); + for (int i = 0; i < blocks.length; i++) { + StripedDataStreamer si = getStripedDataStreamer(i); + if (si.isHealthy()) { // skipping failed data streamer + if (blocks[i] == null) { + // Set exception and close streamer as there is no block locations + // found for the parity block. + LOG.warn("Failed to get block location for parity block, index=" + i); + si.getLastException().set( + new IOException("Failed to get following block, i=" + i)); + si.getErrorState().setInternalError(); + si.close(true); + } else { + coordinator.getFollowingBlocks().offer(i, blocks[i]); + } + } + } + } + + private boolean shouldEndBlockGroup() { + return currentBlockGroup != null && + currentBlockGroup.getNumBytes() == blockSize * numDataBlocks; + } + + @Override + protected synchronized void writeChunk(byte[] bytes, int offset, int len, + byte[] checksum, int ckoff, int cklen) throws IOException { + final int index = getCurrentIndex(); + final StripedDataStreamer current = getCurrentStreamer(); + final int pos = cellBuffers.addTo(index, bytes, offset, len); + final boolean cellFull = pos == cellSize; + + if (currentBlockGroup == null || shouldEndBlockGroup()) { + // the incoming data should belong to a new block. Allocate a new block. + allocateNewBlock(); + } + + currentBlockGroup.setNumBytes(currentBlockGroup.getNumBytes() + len); + if (current.isHealthy()) { + try { + super.writeChunk(bytes, offset, len, checksum, ckoff, cklen); + } catch(Exception e) { + handleStreamerFailure("offset=" + offset + ", length=" + len, e); + } + } + + // 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 (cellFull) { + int next = index + 1; + //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 (next == numDataBlocks) { + cellBuffers.flipDataBuffers(); + writeParityCells(); + next = 0; + // check failure state for all the streamers. Bump GS if necessary + checkStreamerFailures(); + + // if this is the end of the block group, end each internal block + if (shouldEndBlockGroup()) { + for (int i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer s = setCurrentStreamer(i); + if (s.isHealthy()) { + try { + endBlock(); + } catch (IOException ignored) {} + } + } + } + } + setCurrentStreamer(next); + } + } + + @Override + void enqueueCurrentPacketFull() throws IOException { + LOG.debug("enqueue full {}, src={}, bytesCurBlock={}, blockSize={}," + + " appendChunk={}, {}", currentPacket, src, getStreamer() + .getBytesCurBlock(), blockSize, getStreamer().getAppendChunk(), + getStreamer()); + enqueueCurrentPacket(); + adjustChunkBoundary(); + // no need to end block here + } + + private Set markExternalErrorOnStreamers() { + Set healthySet = new HashSet<>(); + for (StripedDataStreamer streamer : streamers) { + if (streamer.isHealthy() && + streamer.getStage() == BlockConstructionStage.DATA_STREAMING) { + streamer.setExternalError(); + healthySet.add(streamer); + } + } + return healthySet; + } + + /** + * Check and handle data streamer failures. This is called only when we have + * written a full stripe (i.e., enqueue all packets for a full stripe), or + * when we're closing the outputstream. + */ + private void checkStreamerFailures() throws IOException { + Set newFailed = checkStreamers(); + if (newFailed.size() > 0) { + // for healthy streamers, wait till all of them have fetched the new block + // and flushed out all the enqueued packets. + flushAllInternals(); + } + // get all the current failed streamers after the flush + newFailed = checkStreamers(); + while (newFailed.size() > 0) { + failedStreamers.addAll(newFailed); + coordinator.clearFailureStates(); + + // mark all the healthy streamers as external error + Set healthySet = markExternalErrorOnStreamers(); + + // we have newly failed streamers, update block for pipeline + final ExtendedBlock newBG = updateBlockForPipeline(healthySet); + + // wait till all the healthy streamers to + // 1) get the updated block info + // 2) create new block outputstream + newFailed = waitCreatingNewStreams(healthySet); + if (newFailed.size() + failedStreamers.size() > + numAllBlocks - numDataBlocks) { + throw new IOException( + "Data streamers failed while creating new block streams: " + + newFailed + ". There are not enough healthy streamers."); + } + for (StripedDataStreamer failedStreamer : newFailed) { + assert !failedStreamer.isHealthy(); + } + + // TODO we can also succeed if all the failed streamers have not taken + // the updated block + if (newFailed.size() == 0) { + // reset external error state of all the streamers + for (StripedDataStreamer streamer : healthySet) { + assert streamer.isHealthy(); + streamer.getErrorState().reset(); + } + updatePipeline(newBG); + } + for (int i = 0; i < numAllBlocks; i++) { + coordinator.offerStreamerUpdateResult(i, newFailed.size() == 0); + } + } + } + + private int checkStreamerUpdates(Set failed, + Set streamers) { + for (StripedDataStreamer streamer : streamers) { + if (!coordinator.updateStreamerMap.containsKey(streamer)) { + if (!streamer.isHealthy() && + coordinator.getNewBlocks().peek(streamer.getIndex()) != null) { + // this streamer had internal error before getting updated block + failed.add(streamer); + } + } + } + return coordinator.updateStreamerMap.size() + failed.size(); + } + + private Set waitCreatingNewStreams( + Set healthyStreamers) throws IOException { + Set failed = new HashSet<>(); + final int expectedNum = healthyStreamers.size(); + final long socketTimeout = dfsClient.getConf().getSocketTimeout(); + // the total wait time should be less than the socket timeout, otherwise + // a slow streamer may cause other streamers to timeout. here we wait for + // half of the socket timeout + long remaingTime = socketTimeout > 0 ? socketTimeout/2 : Long.MAX_VALUE; + final long waitInterval = 1000; + synchronized (coordinator) { + while (checkStreamerUpdates(failed, healthyStreamers) < expectedNum + && remaingTime > 0) { + try { + long start = Time.monotonicNow(); + coordinator.wait(waitInterval); + remaingTime -= Time.monotonicNow() - start; + } catch (InterruptedException e) { + throw DFSUtilClient.toInterruptedIOException("Interrupted when waiting" + + " for results of updating striped streamers", e); + } + } + } + synchronized (coordinator) { + for (StripedDataStreamer streamer : healthyStreamers) { + if (!coordinator.updateStreamerMap.containsKey(streamer)) { + // close the streamer if it is too slow to create new connection + streamer.setStreamerAsClosed(); + failed.add(streamer); + } + } + } + for (Map.Entry entry : + coordinator.updateStreamerMap.entrySet()) { + if (!entry.getValue()) { + failed.add(entry.getKey()); + } + } + for (StripedDataStreamer failedStreamer : failed) { + healthyStreamers.remove(failedStreamer); + } + return failed; + } + + /** + * Call {@link ClientProtocol#updateBlockForPipeline} and assign updated block + * to healthy streamers. + * @param healthyStreamers The healthy data streamers. These streamers join + * the failure handling. + */ + private ExtendedBlock updateBlockForPipeline( + Set healthyStreamers) throws IOException { + final LocatedBlock updated = dfsClient.namenode.updateBlockForPipeline( + currentBlockGroup, dfsClient.clientName); + final long newGS = updated.getBlock().getGenerationStamp(); + ExtendedBlock newBlock = new ExtendedBlock(currentBlockGroup); + newBlock.setGenerationStamp(newGS); + final LocatedBlock[] updatedBlks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) updated, cellSize, numDataBlocks, + numAllBlocks - numDataBlocks); + + for (int i = 0; i < numAllBlocks; i++) { + StripedDataStreamer si = getStripedDataStreamer(i); + if (healthyStreamers.contains(si)) { + final LocatedBlock lb = new LocatedBlock(new ExtendedBlock(newBlock), + null, null, null, -1, updated.isCorrupt(), null); + lb.setBlockToken(updatedBlks[i].getBlockToken()); + coordinator.getNewBlocks().offer(i, lb); + } + } + return newBlock; + } + + private void updatePipeline(ExtendedBlock newBG) throws IOException { + final DatanodeInfo[] newNodes = new DatanodeInfo[numAllBlocks]; + final String[] newStorageIDs = new String[numAllBlocks]; + for (int i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer streamer = getStripedDataStreamer(i); + final DatanodeInfo[] nodes = streamer.getNodes(); + final String[] storageIDs = streamer.getStorageIDs(); + if (streamer.isHealthy() && nodes != null && storageIDs != null) { + newNodes[i] = nodes[0]; + newStorageIDs[i] = storageIDs[0]; + } else { + newNodes[i] = new DatanodeInfo(DatanodeID.EMPTY_DATANODE_ID); + newStorageIDs[i] = ""; + } + } + dfsClient.namenode.updatePipeline(dfsClient.clientName, currentBlockGroup, + newBG, newNodes, newStorageIDs); + currentBlockGroup = newBG; + } + + private int stripeDataSize() { + return numDataBlocks * cellSize; + } + + @Override + public void hflush() { + throw new UnsupportedOperationException(); + } + + @Override + public void hsync() { + throw new UnsupportedOperationException(); + } + + @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.getLastException().set(new IOException("Lease timeout of " + + (dfsClient.getConf().getHdfsTimeout()/1000) + + " seconds expired.")); + } + closeThreads(true); + dfsClient.endFileLease(fileId); + } + + @Override + boolean isClosed() { + if (closed) { + return true; + } + for(StripedDataStreamer s : streamers) { + if (!s.streamerClosed()) { + return false; + } + } + return true; + } + + @Override + protected void closeThreads(boolean force) throws IOException { + final MultipleIOException.Builder b = new MultipleIOException.Builder(); + try { + for (StripedDataStreamer streamer : streamers) { + try { + streamer.close(force); + streamer.join(); + streamer.closeSocket(); + } catch (Exception e) { + try { + handleStreamerFailure("force=" + force, e); + } catch (IOException ioe) { + b.add(ioe); + } + } finally { + streamer.setSocketToNull(); + } + } + } finally { + setClosed(); + } + final IOException ioe = b.build(); + if (ioe != null) { + throw ioe; + } + } + + private boolean generateParityCellsForLastStripe() { + final long currentBlockGroupBytes = currentBlockGroup == null ? + 0 : currentBlockGroup.getNumBytes(); + final long lastStripeSize = currentBlockGroupBytes % stripeDataSize(); + if (lastStripeSize == 0) { + return false; + } + + final long parityCellSize = lastStripeSize < cellSize? + lastStripeSize : cellSize; + final ByteBuffer[] buffers = cellBuffers.getBuffers(); + + for (int i = 0; i < numAllBlocks; i++) { + // Pad zero bytes to make all cells exactly the size of parityCellSize + // If internal block is smaller than parity block, pad zero bytes. + // Also pad zero bytes to all parity cells + final int position = buffers[i].position(); + assert position <= parityCellSize : "If an internal block is smaller" + + " than parity block, then its last cell should be small than last" + + " parity cell"; + for (int j = 0; j < parityCellSize - position; j++) { + buffers[i].put((byte) 0); + } + buffers[i].flip(); + } + return true; + } + + void writeParityCells() throws IOException { + final ByteBuffer[] buffers = cellBuffers.getBuffers(); + //encode the data cells + encode(encoder, numDataBlocks, buffers); + for (int i = numDataBlocks; i < numAllBlocks; i++) { + writeParity(i, buffers[i], cellBuffers.getChecksumArray(i)); + } + cellBuffers.clear(); + } + + void writeParity(int index, ByteBuffer buffer, byte[] checksumBuf) + throws IOException { + final StripedDataStreamer current = setCurrentStreamer(index); + final int len = buffer.limit(); + + final long oldBytes = current.getBytesCurBlock(); + if (current.isHealthy()) { + try { + DataChecksum sum = getDataChecksum(); + sum.calculateChunkedSums(buffer.array(), 0, len, checksumBuf, 0); + for (int i = 0; i < len; i += sum.getBytesPerChecksum()) { + int chunkLen = Math.min(sum.getBytesPerChecksum(), len - i); + int ckOffset = i / sum.getBytesPerChecksum() * getChecksumSize(); + super.writeChunk(buffer.array(), i, chunkLen, checksumBuf, ckOffset, + getChecksumSize()); + } + } catch(Exception e) { + handleStreamerFailure("oldBytes=" + oldBytes + ", len=" + len, e); + } + } + } + + @Override + void setClosed() { + super.setClosed(); + for (int i = 0; i < numAllBlocks; i++) { + getStripedDataStreamer(i).release(); + } + cellBuffers.release(); + } + + @Override + protected synchronized void closeImpl() throws IOException { + if (isClosed()) { + final MultipleIOException.Builder b = new MultipleIOException.Builder(); + for(int i = 0; i < streamers.size(); i++) { + final StripedDataStreamer si = getStripedDataStreamer(i); + try { + si.getLastException().check(true); + } catch (IOException e) { + b.add(e); + } + } + final IOException ioe = b.build(); + if (ioe != null) { + throw ioe; + } + return; + } + + try { + // flush from all upper layers + flushBuffer(); + // if the last stripe is incomplete, generate and write parity cells + if (generateParityCellsForLastStripe()) { + writeParityCells(); + } + enqueueAllCurrentPackets(); + + // flush all the data packets + flushAllInternals(); + // check failures + checkStreamerFailures(); + + for (int i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer s = setCurrentStreamer(i); + if (s.isHealthy()) { + try { + if (s.getBytesCurBlock() > 0) { + setCurrentPacketToEmpty(); + } + // flush the last "close" packet to Datanode + flushInternal(); + } catch(Exception e) { + // TODO for both close and endBlock, we currently do not handle + // failures when sending the last packet. We actually do not need to + // bump GS for this kind of failure. Thus counting the total number + // of failures may be good enough. + } + } + } + + closeThreads(false); + TraceScope scope = dfsClient.getTracer().newScope("completeFile"); + try { + completeFile(currentBlockGroup); + } finally { + scope.close(); + } + dfsClient.endFileLease(fileId); + } catch (ClosedChannelException ignored) { + } finally { + setClosed(); + } + } + + private void enqueueAllCurrentPackets() throws IOException { + int idx = streamers.indexOf(getCurrentStreamer()); + for(int i = 0; i < streamers.size(); i++) { + final StripedDataStreamer si = setCurrentStreamer(i); + if (si.isHealthy() && currentPacket != null) { + try { + enqueueCurrentPacket(); + } catch (IOException e) { + handleStreamerFailure("enqueueAllCurrentPackets, i=" + i, e); + } + } + } + setCurrentStreamer(idx); + } + + void flushAllInternals() throws IOException { + int current = getCurrentIndex(); + + for (int i = 0; i < numAllBlocks; i++) { + final StripedDataStreamer s = setCurrentStreamer(i); + if (s.isHealthy()) { + try { + // flush all data to Datanode + flushInternal(); + } catch(Exception e) { + handleStreamerFailure("flushInternal " + s, e); + } + } + } + setCurrentStreamer(current); + } + + static void sleep(long ms, String op) throws InterruptedIOException { + try { + Thread.sleep(ms); + } catch(InterruptedException ie) { + throw DFSUtilClient.toInterruptedIOException( + "Sleep interrupted during " + op, ie); + } + } + + @Override + ExtendedBlock getBlock() { + return currentBlockGroup; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java index e275afb33e91f..f96ae65108bc0 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DFSUtilClient.java @@ -54,6 +54,7 @@ import javax.net.SocketFactory; import java.io.IOException; +import java.io.InterruptedIOException; import java.io.UnsupportedEncodingException; import java.net.InetAddress; import java.net.InetSocketAddress; @@ -652,4 +653,11 @@ public static URI getNNUri(InetSocketAddress namenode) { return URI.create(HdfsConstants.HDFS_URI_SCHEME + "://" + namenode.getHostName() + portString); } + + public static InterruptedIOException toInterruptedIOException(String message, + InterruptedException e) { + final InterruptedIOException iioe = new InterruptedIOException(message); + iioe.initCause(e); + return iioe; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java index d1d8d37f3dc80..af7a61e8b3763 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DataStreamer.java @@ -22,7 +22,6 @@ import java.io.BufferedOutputStream; import java.io.DataInputStream; import java.io.DataOutputStream; -import java.io.FileNotFoundException; import java.io.IOException; import java.io.InputStream; import java.io.InterruptedIOException; @@ -45,16 +44,12 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys.BlockWrite; import org.apache.hadoop.hdfs.client.impl.DfsClientConf; -import org.apache.hadoop.hdfs.protocol.DSQuotaExceededException; 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.protocol.NSQuotaExceededException; -import org.apache.hadoop.hdfs.protocol.QuotaByStorageTypeExceededException; import org.apache.hadoop.hdfs.protocol.QuotaExceededException; -import org.apache.hadoop.hdfs.protocol.UnresolvedPathException; import org.apache.hadoop.hdfs.protocol.datatransfer.BlockConstructionStage; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtoUtil; import org.apache.hadoop.hdfs.protocol.datatransfer.DataTransferProtocol; @@ -68,13 +63,10 @@ import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; -import org.apache.hadoop.hdfs.server.namenode.NotReplicatedYetException; import org.apache.hadoop.hdfs.util.ByteArrayManager; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.io.MultipleIOException; -import org.apache.hadoop.ipc.RemoteException; import org.apache.hadoop.net.NetUtils; -import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.DataChecksum; @@ -170,7 +162,7 @@ private static void releaseBuffer(List packets, ByteArrayManager bam) packets.clear(); } - static class LastExceptionInStreamer { + class LastExceptionInStreamer { private IOException thrown; synchronized void set(Throwable t) { @@ -188,7 +180,8 @@ synchronized void check(boolean resetToNull) throws IOException { if (thrown != null) { if (LOG.isTraceEnabled()) { // wrap and print the exception to know when the check is called - LOG.trace("Got Exception while checking", new Throwable(thrown)); + LOG.trace("Got Exception while checking, " + DataStreamer.this, + new Throwable(thrown)); } final IOException e = thrown; if (resetToNull) { @@ -204,8 +197,12 @@ synchronized void throwException4Close() throws IOException { } } + enum ErrorType { + NONE, INTERNAL, EXTERNAL + } + static class ErrorState { - private boolean error = false; + ErrorType error = ErrorType.NONE; private int badNodeIndex = -1; private int restartingNodeIndex = -1; private long restartingNodeDeadline = 0; @@ -215,23 +212,46 @@ static class ErrorState { this.datanodeRestartTimeout = datanodeRestartTimeout; } + synchronized void resetInternalError() { + if (hasInternalError()) { + error = ErrorType.NONE; + } + badNodeIndex = -1; + restartingNodeIndex = -1; + restartingNodeDeadline = 0; + } + synchronized void reset() { - error = false; + error = ErrorType.NONE; badNodeIndex = -1; restartingNodeIndex = -1; restartingNodeDeadline = 0; } + synchronized boolean hasInternalError() { + return error == ErrorType.INTERNAL; + } + + synchronized boolean hasExternalError() { + return error == ErrorType.EXTERNAL; + } + synchronized boolean hasError() { - return error; + return error != ErrorType.NONE; } synchronized boolean hasDatanodeError() { - return error && isNodeMarked(); + return error == ErrorType.INTERNAL && isNodeMarked(); } - synchronized void setError(boolean err) { - this.error = err; + synchronized void setInternalError() { + this.error = ErrorType.INTERNAL; + } + + synchronized void setExternalError() { + if (!hasInternalError()) { + this.error = ErrorType.EXTERNAL; + } } synchronized void setBadNodeIndex(int index) { @@ -294,14 +314,14 @@ synchronized void adjustState4RestartingNode() { } if (!isRestartingNode()) { - error = false; + error = ErrorType.NONE; } badNodeIndex = -1; } synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { if (restartingNodeIndex >= 0) { - if (!error) { + if (error == ErrorType.NONE) { throw new IllegalStateException("error=false while checking" + " restarting node deadline"); } @@ -332,8 +352,8 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { } private volatile boolean streamerClosed = false; - private ExtendedBlock block; // its length is number of bytes acked - private Token accessToken; + protected ExtendedBlock block; // its length is number of bytes acked + protected Token accessToken; private DataOutputStream blockStream; private DataInputStream blockReplyStream; private ResponseProcessor response = null; @@ -343,7 +363,7 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { private final ErrorState errorState; private BlockConstructionStage stage; // block construction stage - private long bytesSent = 0; // number of bytes that've been sent + protected long bytesSent = 0; // number of bytes that've been sent private final boolean isLazyPersistFile; /** Nodes have been used in the pipeline before and have failed. */ @@ -363,16 +383,16 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { private final LastExceptionInStreamer lastException = new LastExceptionInStreamer(); 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; + final DataChecksum checksum4WriteBlock; + final Progressable progress; + protected final HdfsFileStatus stat; // appending to existing partial block private volatile boolean appendChunk = false; // both dataQueue and ackQueue are protected by dataQueue lock - private final LinkedList dataQueue = new LinkedList<>(); + protected final LinkedList dataQueue = new LinkedList<>(); private final LinkedList ackQueue = new LinkedList<>(); private final AtomicReference cachingStrategy; private final ByteArrayManager byteArrayManager; @@ -389,14 +409,16 @@ synchronized void checkRestartingNodeDeadline(DatanodeInfo[] nodes) { CONGESTION_BACKOFF_MEAN_TIME_IN_MS * 10; private int lastCongestionBackoffTime; - private final LoadingCache excludedNodes; + protected final LoadingCache excludedNodes; private final String[] favoredNodes; - private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, + private DataStreamer(HdfsFileStatus stat, ExtendedBlock block, + DFSClient dfsClient, String src, Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, ByteArrayManager byteArrayManage, boolean isAppend, String[] favoredNodes) { + this.block = block; this.dfsClient = dfsClient; this.src = src; this.progress = progress; @@ -421,9 +443,8 @@ private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, String src, Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, ByteArrayManager byteArrayManage, String[] favoredNodes) { - this(stat, dfsClient, src, progress, checksum, cachingStrategy, + this(stat, block, dfsClient, src, progress, checksum, cachingStrategy, byteArrayManage, false, favoredNodes); - this.block = block; stage = BlockConstructionStage.PIPELINE_SETUP_CREATE; } @@ -437,10 +458,9 @@ private DataStreamer(HdfsFileStatus stat, DFSClient dfsClient, String src, String src, Progressable progress, DataChecksum checksum, AtomicReference cachingStrategy, ByteArrayManager byteArrayManage) throws IOException { - this(stat, dfsClient, src, progress, checksum, cachingStrategy, + this(stat, lastBlock.getBlock(), dfsClient, src, progress, checksum, cachingStrategy, byteArrayManage, true, null); stage = BlockConstructionStage.PIPELINE_SETUP_APPEND; - block = lastBlock.getBlock(); bytesSent = block.getNumBytes(); accessToken = lastBlock.getBlockToken(); } @@ -461,6 +481,10 @@ void setPipelineInConstruction(LocatedBlock lastBlock) throws IOException{ } } + void setAccessToken(Token t) { + this.accessToken = t; + } + private void setPipeline(LocatedBlock lb) { setPipeline(lb.getLocations(), lb.getStorageTypes(), lb.getStorageIDs()); } @@ -483,7 +507,7 @@ private void initDataStreaming() { stage = BlockConstructionStage.DATA_STREAMING; } - private void endBlock() { + protected void endBlock() { if(LOG.isDebugEnabled()) { LOG.debug("Closing old block " + block); } @@ -521,7 +545,7 @@ public void run() { DFSPacket one; try { // process datanode IO errors if any - boolean doSleep = processDatanodeError(); + boolean doSleep = processDatanodeOrExternalError(); final int halfSocketTimeout = dfsClient.getConf().getSocketTimeout()/2; synchronized (dataQueue) { @@ -566,16 +590,13 @@ public void run() { } // get new block from namenode. + if (LOG.isDebugEnabled()) { + LOG.debug("stage=" + stage + ", " + this); + } if (stage == BlockConstructionStage.PIPELINE_SETUP_CREATE) { - if(LOG.isDebugEnabled()) { - LOG.debug("Allocating new block"); - } setPipeline(nextBlockOutputStream()); initDataStreaming(); } else if (stage == BlockConstructionStage.PIPELINE_SETUP_APPEND) { - if(LOG.isDebugEnabled()) { - LOG.debug("Append to block " + block); - } setupPipelineForAppendOrRecovery(); if (streamerClosed) { continue; @@ -586,10 +607,7 @@ public void run() { long lastByteOffsetInBlock = one.getLastByteOffsetBlock(); if (lastByteOffsetInBlock > stat.getBlockSize()) { throw new IOException("BlockSize " + stat.getBlockSize() + - " is smaller than data size. " + - " Offset of packet in block " + - lastByteOffsetInBlock + - " Aborting file " + src); + " < lastByteOffsetInBlock, " + this + ", " + one); } if (one.isLastPacketInBlock()) { @@ -628,8 +646,7 @@ public void run() { } if (LOG.isDebugEnabled()) { - LOG.debug("DataStreamer block " + block + - " sending packet " + one); + LOG.debug(this + " sending " + one); } // write out data to remote datanode @@ -695,7 +712,7 @@ public void run() { } lastException.set(e); assert !(e instanceof NullPointerException); - errorState.setError(true); + errorState.setInternalError(); if (!errorState.isNodeMarked()) { // Not a datanode issue streamerClosed = true; @@ -840,6 +857,9 @@ void close(boolean force) { } } + void setStreamerAsClosed() { + streamerClosed = true; + } private void checkClosed() throws IOException { if (streamerClosed) { @@ -860,7 +880,7 @@ private void closeResponder() { } } - private void closeStream() { + void closeStream() { final MultipleIOException.Builder b = new MultipleIOException.Builder(); if (blockStream != null) { @@ -1043,7 +1063,7 @@ public void run() { } catch (Exception e) { if (!responderClosed) { lastException.set(e); - errorState.setError(true); + errorState.setInternalError(); errorState.markFirstNodeIfNotMarked(); synchronized (dataQueue) { dataQueue.notifyAll(); @@ -1068,14 +1088,18 @@ void close() { } } + private boolean shouldHandleExternalError(){ + return errorState.hasExternalError() && blockStream != null; + } + /** * If this stream has encountered any errors, shutdown threads * and mark the stream as closed. * * @return true if it should sleep for a while after returning. */ - private boolean processDatanodeError() throws IOException { - if (!errorState.hasDatanodeError()) { + private boolean processDatanodeOrExternalError() throws IOException { + if (!errorState.hasDatanodeError() && !shouldHandleExternalError()) { return false; } if (response != null) { @@ -1108,7 +1132,8 @@ private boolean processDatanodeError() throws IOException { return false; } } - boolean doSleep = setupPipelineForAppendOrRecovery(); + + setupPipelineForAppendOrRecovery(); if (!streamerClosed && dfsClient.clientRunning) { if (stage == BlockConstructionStage.PIPELINE_CLOSE) { @@ -1141,7 +1166,7 @@ private boolean processDatanodeError() throws IOException { } } - return doSleep; + return false; } void setHflush() { @@ -1300,7 +1325,7 @@ private void transfer(final DatanodeInfo src, final DatanodeInfo[] targets, * This happens when a file is appended or data streaming fails * It keeps on trying until a pipeline is setup */ - private boolean setupPipelineForAppendOrRecovery() throws IOException { + private void setupPipelineForAppendOrRecovery() throws IOException { // check number of datanodes if (nodes == null || nodes.length == 0) { String msg = "Could not get block locations. " + "Source file \"" @@ -1308,19 +1333,23 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException { LOG.warn(msg); lastException.set(new IOException(msg)); streamerClosed = true; - return false; + return; } + setupPipelineInternal(nodes, storageTypes); + } + protected void setupPipelineInternal(DatanodeInfo[] datanodes, + StorageType[] nodeStorageTypes) throws IOException { boolean success = false; long newGS = 0L; while (!success && !streamerClosed && dfsClient.clientRunning) { if (!handleRestartingDatanode()) { - return false; + return; } - final boolean isRecovery = errorState.hasError(); + final boolean isRecovery = errorState.hasInternalError(); if (!handleBadDatanode()) { - return false; + return; } handleDatanodeReplacement(); @@ -1341,7 +1370,6 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException { if (success) { block = updatePipeline(newGS); } - return false; // do not sleep, continue processing } /** @@ -1349,7 +1377,7 @@ private boolean setupPipelineForAppendOrRecovery() throws IOException { * This process is repeated until the deadline or the node starts back up. * @return true if it should continue. */ - private boolean handleRestartingDatanode() { + boolean handleRestartingDatanode() { if (errorState.isRestartingNode()) { // 4 seconds or the configured deadline period, whichever is shorter. // This is the retry interval and recovery will be retried in this @@ -1372,7 +1400,7 @@ private boolean handleRestartingDatanode() { * Remove bad node from list of nodes if badNodeIndex was set. * @return true if it should continue. */ - private boolean handleBadDatanode() { + boolean handleBadDatanode() { final int badNodeIndex = errorState.getBadNodeIndex(); if (badNodeIndex >= 0) { if (nodes.length <= 1) { @@ -1422,7 +1450,7 @@ private void handleDatanodeReplacement() throws IOException { } } - private void failPacket4Testing() { + void failPacket4Testing() { if (failPacket) { // for testing failPacket = false; try { @@ -1434,37 +1462,43 @@ private void failPacket4Testing() { } } - LocatedBlock updateBlockForPipeline() throws IOException { - return dfsClient.namenode.updateBlockForPipeline( - block, dfsClient.clientName); + private LocatedBlock updateBlockForPipeline() throws IOException { + return dfsClient.namenode.updateBlockForPipeline(block, dfsClient.clientName); + } + + static ExtendedBlock newBlock(ExtendedBlock b, final long newGS) { + return new ExtendedBlock(b.getBlockPoolId(), b.getBlockId(), + b.getNumBytes(), newGS); } /** update pipeline at the namenode */ ExtendedBlock updatePipeline(long newGS) throws IOException { - final ExtendedBlock newBlock = new ExtendedBlock( - block.getBlockPoolId(), block.getBlockId(), block.getNumBytes(), newGS); + final ExtendedBlock newBlock = newBlock(block, newGS); dfsClient.namenode.updatePipeline(dfsClient.clientName, block, newBlock, nodes, storageIDs); return newBlock; } + private int getNumBlockWriteRetry() { + return dfsClient.getConf().getNumBlockWriteRetry(); + } + /** * Open a DataStreamer to a DataNode so that it can be written to. * This happens when a file is created and each time a new block is allocated. * Must get block ID and the IDs of the destinations from the namenode. * Returns the list of target datanodes. */ - private LocatedBlock nextBlockOutputStream() throws IOException { + protected LocatedBlock nextBlockOutputStream() throws IOException { LocatedBlock lb = null; DatanodeInfo[] nodes = null; StorageType[] storageTypes = null; - int count = dfsClient.getConf().getNumBlockWriteRetry(); + int count = getNumBlockWriteRetry(); boolean success = false; ExtendedBlock oldBlock = block; do { - errorState.reset(); + errorState.resetInternalError(); lastException.clear(); - success = false; DatanodeInfo[] excluded = excludedNodes.getAllPresent(excludedNodes.asMap().keySet()) @@ -1504,7 +1538,7 @@ private LocatedBlock nextBlockOutputStream() throws IOException { // connects to the first datanode in the pipeline // Returns true if success, otherwise return failure. // - private boolean createBlockOutputStream(DatanodeInfo[] nodes, + boolean createBlockOutputStream(DatanodeInfo[] nodes, StorageType[] nodeStorageTypes, long newGS, boolean recoveryFlag) { if (nodes.length == 0) { LOG.info("nodes are empty for write pipeline of " + block); @@ -1514,7 +1548,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, String firstBadLink = ""; boolean checkRestart = false; if (LOG.isDebugEnabled()) { - LOG.debug("pipeline = " + Arrays.asList(nodes)); + LOG.debug("pipeline = " + Arrays.toString(nodes) + ", " + this); } // persist blocks on namenode on next flush @@ -1583,10 +1617,10 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, assert null == blockStream : "Previous blockStream unclosed"; blockStream = out; result = true; // success - errorState.reset(); + errorState.resetInternalError(); } catch (IOException ie) { if (!errorState.isRestartingNode()) { - LOG.info("Exception in createBlockOutputStream", ie); + LOG.info("Exception in createBlockOutputStream " + this, ie); } if (ie instanceof InvalidEncryptionKeyException && refetchEncryptionKey > 0) { LOG.info("Will fetch a new encryption key and retry, " @@ -1619,7 +1653,7 @@ private boolean createBlockOutputStream(DatanodeInfo[] nodes, if (checkRestart && shouldWaitForRestart(i)) { errorState.initRestartingNode(i, "Datanode " + i + " is restarting: " + nodes[i]); } - errorState.setError(true); + errorState.setInternalError(); lastException.set(ie); result = false; // error } finally { @@ -1661,58 +1695,10 @@ private boolean[] getPinnings(DatanodeInfo[] nodes, boolean shouldLog) { } } - protected LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) + private LocatedBlock locateFollowingBlock(DatanodeInfo[] excludedNodes) throws IOException { - final DfsClientConf conf = dfsClient.getConf(); - int retries = conf.getNumBlockWriteLocateFollowingRetry(); - long sleeptime = conf.getBlockWriteLocateFollowingInitialDelayMs(); - while (true) { - long localstart = Time.monotonicNow(); - while (true) { - try { - return dfsClient.namenode.addBlock(src, dfsClient.clientName, - block, excludedNodes, stat.getFileId(), favoredNodes); - } catch (RemoteException e) { - IOException ue = - e.unwrapRemoteException(FileNotFoundException.class, - AccessControlException.class, - NSQuotaExceededException.class, - DSQuotaExceededException.class, - QuotaByStorageTypeExceededException.class, - UnresolvedPathException.class); - if (ue != e) { - throw ue; // no need to retry these exceptions - } - - - if (NotReplicatedYetException.class.getName(). - equals(e.getClassName())) { - if (retries == 0) { - throw e; - } else { - --retries; - LOG.info("Exception while adding a block", e); - long elapsed = Time.monotonicNow() - localstart; - if (elapsed > 5000) { - LOG.info("Waiting for replication for " - + (elapsed / 1000) + " seconds"); - } - try { - LOG.warn("NotReplicatedYetException sleeping " + src - + " retries left " + retries); - Thread.sleep(sleeptime); - sleeptime *= 2; - } catch (InterruptedException ie) { - LOG.warn("Caught exception", ie); - } - } - } else { - throw e; - } - - } - } - } + return DFSOutputStream.addBlock(excludedNodes, dfsClient, src, block, + stat.getFileId(), favoredNodes); } /** @@ -1767,6 +1753,14 @@ DatanodeInfo[] getNodes() { return nodes; } + String[] getStorageIDs() { + return storageIDs; + } + + BlockConstructionStage getStage() { + return stage; + } + /** * return the token of the block * @@ -1776,6 +1770,10 @@ Token getBlockToken() { return accessToken; } + ErrorState getErrorState() { + return errorState; + } + /** * Put a packet to the data queue * @@ -1788,7 +1786,7 @@ void queuePacket(DFSPacket packet) { dataQueue.addLast(packet); lastQueuedSeqno = packet.getSeqno(); if (LOG.isDebugEnabled()) { - LOG.debug("Queued packet " + packet.getSeqno()); + LOG.debug("Queued " + packet + ", " + this); } dataQueue.notifyAll(); } @@ -1941,7 +1939,6 @@ void closeSocket() throws IOException { @Override public String toString() { - return (block == null? null: block.getLocalBlock()) - + "@" + Arrays.toString(getNodes()); + return block == null? "block==null": "" + block.getLocalBlock(); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java index ea86d2dc775a5..18cc124655cc6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/DistributedFileSystem.java @@ -85,6 +85,7 @@ import org.apache.hadoop.hdfs.protocol.SnapshottableDirectoryStatus; import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.io.Text; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.net.NetUtils; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.security.Credentials; @@ -2256,4 +2257,79 @@ public DFSInotifyEventInputStream getInotifyEventStream(long lastReadTxid) throws IOException { return dfs.getInotifyEventStream(lastReadTxid); } + + /** + * Set the source path to the specified erasure coding policy. + * + * @param path The directory to set the policy + * @param ecPolicy The erasure coding policy. If not specified default will be used. + * @throws IOException + */ + public void setErasureCodingPolicy(final Path path, final ErasureCodingPolicy ecPolicy) + throws IOException { + Path absF = fixRelativePart(path); + new FileSystemLinkResolver() { + @Override + public Void doCall(final Path p) throws IOException, + UnresolvedLinkException { + dfs.setErasureCodingPolicy(getPathName(p), ecPolicy); + return null; + } + + @Override + public Void next(final FileSystem fs, final Path p) throws IOException { + if (fs instanceof DistributedFileSystem) { + DistributedFileSystem myDfs = (DistributedFileSystem) fs; + myDfs.setErasureCodingPolicy(p, ecPolicy); + return null; + } + throw new UnsupportedOperationException( + "Cannot setErasureCodingPolicy through a symlink to a " + + "non-DistributedFileSystem: " + path + " -> " + p); + } + }.resolve(this, absF); + } + + /** + * Get erasure coding policy information for the specified path + * + * @param path The path of the file or directory + * @return Returns the policy information if file or directory on the path + * is erasure coded, null otherwise + * @throws IOException + */ + public ErasureCodingPolicy getErasureCodingPolicy(final Path path) + throws IOException { + Path absF = fixRelativePart(path); + return new FileSystemLinkResolver() { + @Override + public ErasureCodingPolicy doCall(final Path p) throws IOException, + UnresolvedLinkException { + return dfs.getErasureCodingPolicy(getPathName(p)); + } + + @Override + public ErasureCodingPolicy next(final FileSystem fs, final Path p) + throws IOException { + if (fs instanceof DistributedFileSystem) { + DistributedFileSystem myDfs = (DistributedFileSystem) fs; + return myDfs.getErasureCodingPolicy(p); + } + throw new UnsupportedOperationException( + "Cannot getErasureCodingPolicy through a symlink to a " + + "non-DistributedFileSystem: " + path + " -> " + p); + } + }.resolve(this, absF); + } + + /** + * Retrieve all the erasure coding policies supported by this file system. + * + * @return all erasure coding policies supported by this file system. + * @throws IOException + */ + public Collection getAllErasureCodingPolicies() + throws IOException { + return Arrays.asList(dfs.getErasureCodingPolicies()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java index 2eb9d526f089e..015e5cbaafb41 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/ExternalBlockReader.java @@ -24,6 +24,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.ReadOption; import org.apache.hadoop.hdfs.shortcircuit.ClientMmap; +import org.apache.hadoop.util.DataChecksum; /** * An ExternalBlockReader uses pluggable ReplicaAccessor objects to read from @@ -123,4 +124,9 @@ public ClientMmap getClientMmap(EnumSet opts) { // For now, pluggable ReplicaAccessors do not support zero-copy. return null; } + + @Override + public DataChecksum getDataChecksum() { + return null; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java index 15a5bee7dc50f..5255862b7f506 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader.java @@ -511,4 +511,9 @@ public boolean isShortCircuit() { public ClientMmap getClientMmap(EnumSet opts) { return null; } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java index 7a7932d1b6a4f..10f310d8ce788 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/RemoteBlockReader2.java @@ -480,4 +480,9 @@ public boolean isShortCircuit() { public ClientMmap getClientMmap(EnumSet opts) { return null; } + + @Override + public DataChecksum getDataChecksum() { + return checksum; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java new file mode 100644 index 0000000000000..a313ecb18949f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/StripedDataStreamer.java @@ -0,0 +1,187 @@ +/** + * 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.util.concurrent.atomic.AtomicReference; + +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.DFSStripedOutputStream.Coordinator; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.hdfs.util.ByteArrayManager; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.apache.hadoop.util.DataChecksum; +import org.apache.hadoop.util.Progressable; + +import com.google.common.annotations.VisibleForTesting; + +/** + * This class extends {@link DataStreamer} to support writing striped blocks + * to datanodes. + * A {@link DFSStripedOutputStream} has multiple {@link StripedDataStreamer}s. + * Whenever the streamers need to talk the namenode, only the fastest streamer + * sends an rpc call to the namenode and then populates the result for the + * other streamers. + */ +public class StripedDataStreamer extends DataStreamer { + private final Coordinator coordinator; + private final int index; + + StripedDataStreamer(HdfsFileStatus stat, + DFSClient dfsClient, String src, + Progressable progress, DataChecksum checksum, + AtomicReference cachingStrategy, + ByteArrayManager byteArrayManage, String[] favoredNodes, + short index, Coordinator coordinator) { + super(stat, null, dfsClient, src, progress, checksum, cachingStrategy, + byteArrayManage, favoredNodes); + this.index = index; + this.coordinator = coordinator; + } + + int getIndex() { + return index; + } + + boolean isHealthy() { + return !streamerClosed() && !getErrorState().hasInternalError(); + } + + @Override + protected void endBlock() { + coordinator.offerEndBlock(index, block); + super.endBlock(); + } + + /** + * The upper level DFSStripedOutputStream will allocate the new block group. + * All the striped data streamer only needs to fetch from the queue, which + * should be already be ready. + */ + private LocatedBlock getFollowingBlock() throws IOException { + if (!this.isHealthy()) { + // No internal block for this streamer, maybe no enough healthy DN. + // Throw the exception which has been set by the StripedOutputStream. + this.getLastException().check(false); + } + return coordinator.getFollowingBlocks().poll(index); + } + + @Override + protected LocatedBlock nextBlockOutputStream() throws IOException { + boolean success; + LocatedBlock lb = getFollowingBlock(); + block = lb.getBlock(); + block.setNumBytes(0); + bytesSent = 0; + accessToken = lb.getBlockToken(); + + DatanodeInfo[] nodes = lb.getLocations(); + StorageType[] storageTypes = lb.getStorageTypes(); + + // Connect to the DataNode. If fail the internal error state will be set. + success = createBlockOutputStream(nodes, storageTypes, 0L, false); + + if (!success) { + block = null; + final DatanodeInfo badNode = nodes[getErrorState().getBadNodeIndex()]; + LOG.info("Excluding datanode " + badNode); + excludedNodes.put(badNode, badNode); + throw new IOException("Unable to create new block."); + } + return lb; + } + + @VisibleForTesting + LocatedBlock peekFollowingBlock() { + return coordinator.getFollowingBlocks().peek(index); + } + + @Override + protected void setupPipelineInternal(DatanodeInfo[] nodes, + StorageType[] nodeStorageTypes) throws IOException { + boolean success = false; + while (!success && !streamerClosed() && dfsClient.clientRunning) { + if (!handleRestartingDatanode()) { + return; + } + if (!handleBadDatanode()) { + // for striped streamer if it is datanode error then close the stream + // and return. no need to replace datanode + return; + } + + // get a new generation stamp and an access token + final LocatedBlock lb = coordinator.getNewBlocks().take(index); + long newGS = lb.getBlock().getGenerationStamp(); + setAccessToken(lb.getBlockToken()); + + // set up the pipeline again with the remaining nodes. when a striped + // data streamer comes here, it must be in external error state. + assert getErrorState().hasExternalError(); + success = createBlockOutputStream(nodes, nodeStorageTypes, newGS, true); + + failPacket4Testing(); + getErrorState().checkRestartingNodeDeadline(nodes); + + // notify coordinator the result of createBlockOutputStream + synchronized (coordinator) { + if (!streamerClosed()) { + coordinator.updateStreamer(this, success); + coordinator.notify(); + } else { + success = false; + } + } + + if (success) { + // wait for results of other streamers + success = coordinator.takeStreamerUpdateResult(index); + if (success) { + // if all succeeded, update its block using the new GS + block = newBlock(block, newGS); + } else { + // otherwise close the block stream and restart the recovery process + closeStream(); + } + } else { + // if fail, close the stream. The internal error state and last + // exception have already been set in createBlockOutputStream + // TODO: wait for restarting DataNodes during RollingUpgrade + closeStream(); + setStreamerAsClosed(); + } + } // while + } + + void setExternalError() { + getErrorState().setExternalError(); + synchronized (dataQueue) { + dataQueue.notifyAll(); + } + } + + @Override + public String toString() { + return "#" + index + ": " + (!isHealthy() ? "failed, ": "") + super.toString(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java index fe31531674510..11be51f2a07bb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/HdfsClientConfigKeys.java @@ -291,6 +291,18 @@ interface HedgedRead { int THREADPOOL_SIZE_DEFAULT = 0; } + /** dfs.client.read.striped configuration properties */ + interface StripedRead { + String PREFIX = Read.PREFIX + "striped."; + + String THREADPOOL_SIZE_KEY = PREFIX + "threadpool.size"; + /** + * With default RS-6-3-64k erasure coding policy, each normal read could span + * 6 DNs, so this default value accommodates 3 read streams + */ + int THREADPOOL_SIZE_DEFAULT = 18; + } + /** dfs.http.client configuration properties */ interface HttpClient { String PREFIX = "dfs.http.client."; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java index d46ab47f5be23..b99e3bab287cb 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/client/impl/DfsClientConf.java @@ -18,6 +18,7 @@ package org.apache.hadoop.hdfs.client.impl; import com.google.common.annotations.VisibleForTesting; +import com.google.common.base.Preconditions; import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.CommonConfigurationKeysPublic; @@ -130,6 +131,9 @@ public class DfsClientConf { private final List> replicaAccessorBuilderClasses; + private final int stripedReadThreadpoolSize; + + public DfsClientConf(Configuration conf) { // The hdfsTimeout is currently the same as the ipc timeout hdfsTimeout = Client.getTimeout(conf); @@ -237,6 +241,12 @@ public DfsClientConf(Configuration conf) { HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_KEY, HdfsClientConfigKeys.HedgedRead.THREADPOOL_SIZE_DEFAULT); + stripedReadThreadpoolSize = conf.getInt( + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY, + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_DEFAULT); + Preconditions.checkArgument(stripedReadThreadpoolSize > 0, "The value of " + + HdfsClientConfigKeys.StripedRead.THREADPOOL_SIZE_KEY + + " must be greater than 0."); replicaAccessorBuilderClasses = loadReplicaAccessorBuilderClasses(conf); } @@ -518,6 +528,13 @@ public int getHedgedReadThreadpoolSize() { return hedgedReadThreadpoolSize; } + /** + * @return the stripedReadThreadpoolSize + */ + public int getStripedReadThreadpoolSize() { + return stripedReadThreadpoolSize; + } + /** * @return the replicaAccessorBuilderClasses */ diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java index 852899973536a..a0504a8fba39e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ClientProtocol.java @@ -1483,4 +1483,31 @@ List listXAttrs(String src) */ @Idempotent EventBatchList getEditsFromTxid(long txid) throws IOException; + + /** + * Set an erasure coding policy on a specified path. + * @param src The path to set policy on. + * @param ecPolicy The erasure coding policy. If null, default policy will + * be used + */ + @AtMostOnce + void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) + throws IOException; + + /** + * Get the erasure coding policies loaded in Namenode + * + * @throws IOException + */ + @Idempotent + ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException; + + /** + * Get the information about the EC policy for the path + * + * @param src path to get the info for + * @throws IOException + */ + @Idempotent + ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java index 6d72285426106..c709cbdd47135 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/DatanodeID.java @@ -38,6 +38,8 @@ @InterfaceStability.Evolving public class DatanodeID implements Comparable { public static final DatanodeID[] EMPTY_ARRAY = {}; + public static final DatanodeID EMPTY_DATANODE_ID = new DatanodeID("null", + "null", "null", 0, 0, 0, 0); private String ipAddr; // IP address private String hostName; // hostname claimed by datanode diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.java new file mode 100644 index 0000000000000..e5dfdff5070d3 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/ErasureCodingPolicy.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.hdfs.protocol; + +import org.apache.hadoop.io.erasurecode.ECSchema; + +import java.util.Map; + +/** + * A policy about how to write/read/code an erasure coding file. + */ +public final class ErasureCodingPolicy { + + private final String name; + private final ECSchema schema; + private final int cellSize; + + public ErasureCodingPolicy(String name, ECSchema schema, int cellSize){ + this.name = name; + this.schema = schema; + this.cellSize = cellSize; + } + + public String getName() { + return name; + } + + public ECSchema getSchema() { + return schema; + } + + public int getCellSize() { + return cellSize; + } + + public int getNumDataUnits() { + return schema.getNumDataUnits(); + } + + public int getNumParityUnits() { + return schema.getNumParityUnits(); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + ErasureCodingPolicy that = (ErasureCodingPolicy) o; + + if (that.getName().equals(name) && that.getCellSize() == cellSize + && that.getSchema().equals(schema)) { + return true; + } + return false; + } + + @Override + public int hashCode() { + int result = name.hashCode(); + result = 31 * result + schema.hashCode(); + result = 31 * result + cellSize; + return result; + } + + @Override + public String toString() { + StringBuilder sb = new StringBuilder("ErasureCodingPolicy=["); + sb.append("Name=" + name + ", "); + sb.append("Schema=[" + schema.toString() + "], "); + sb.append("CellSize=" + cellSize + " "); + sb.append("]"); + return sb.toString(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java index 34f429a21b517..6e05ce086f732 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsFileStatus.java @@ -48,6 +48,8 @@ public class HdfsFileStatus { private final FileEncryptionInfo feInfo; + private final ErasureCodingPolicy ecPolicy; + // Used by dir, not including dot and dotdot. Always zero for a regular file. private final int childrenNum; private final byte storagePolicy; @@ -73,7 +75,7 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, long blocksize, long modification_time, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, int childrenNum, FileEncryptionInfo feInfo, - byte storagePolicy) { + byte storagePolicy, ErasureCodingPolicy ecPolicy) { this.length = length; this.isdir = isdir; this.block_replication = (short)block_replication; @@ -93,6 +95,7 @@ public HdfsFileStatus(long length, boolean isdir, int block_replication, this.childrenNum = childrenNum; this.feInfo = feInfo; this.storagePolicy = storagePolicy; + this.ecPolicy = ecPolicy; } /** @@ -250,6 +253,10 @@ public final FileEncryptionInfo getFileEncryptionInfo() { return feInfo; } + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; + } + public final int getChildrenNum() { return childrenNum; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java index 23e8f57839bf1..2121dcf23bca9 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/HdfsLocatedFileStatus.java @@ -58,10 +58,11 @@ public HdfsLocatedFileStatus(long length, boolean isdir, int block_replication, long blocksize, long modification_time, long access_time, FsPermission permission, String owner, String group, byte[] symlink, byte[] path, long fileId, LocatedBlocks locations, - int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy) { + int childrenNum, FileEncryptionInfo feInfo, byte storagePolicy, + ErasureCodingPolicy ecPolicy) { super(length, isdir, block_replication, blocksize, modification_time, access_time, permission, owner, group, symlink, path, fileId, - childrenNum, feInfo, storagePolicy); + childrenNum, feInfo, storagePolicy, ecPolicy); this.locations = locations; } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java index cc13f10605e8f..a9596bf67dadd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlock.java @@ -49,14 +49,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 blockToken = new Token(); + private Token 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) { @@ -203,4 +203,8 @@ public String toString() { + "; locs=" + Arrays.asList(locs) + "}"; } + + public boolean isStriped() { + return false; + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java index e48969776a6ca..6e01bbe82f168 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedBlocks.java @@ -37,6 +37,7 @@ public class LocatedBlocks { private final LocatedBlock lastLocatedBlock; private final boolean isLastBlockComplete; private final FileEncryptionInfo fileEncryptionInfo; + private final ErasureCodingPolicy ecPolicy; public LocatedBlocks() { fileLength = 0; @@ -45,17 +46,20 @@ public LocatedBlocks() { lastLocatedBlock = null; isLastBlockComplete = false; fileEncryptionInfo = null; + ecPolicy = null; } public LocatedBlocks(long flength, boolean isUnderConstuction, - List blks, LocatedBlock lastBlock, - boolean isLastBlockCompleted, FileEncryptionInfo feInfo) { + List blks, LocatedBlock lastBlock, + boolean isLastBlockCompleted, FileEncryptionInfo feInfo, + ErasureCodingPolicy ecPolicy) { fileLength = flength; blocks = blks; underConstruction = isUnderConstuction; this.lastLocatedBlock = lastBlock; this.isLastBlockComplete = isLastBlockCompleted; this.fileEncryptionInfo = feInfo; + this.ecPolicy = ecPolicy; } /** @@ -111,6 +115,13 @@ public FileEncryptionInfo getFileEncryptionInfo() { return fileEncryptionInfo; } + /** + * @return The ECPolicy for ErasureCoded file, null otherwise. + */ + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; + } + /** * Find block containing specified offset. * diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java new file mode 100644 index 0000000000000..a9a80c20c32f6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/LocatedStripedBlock.java @@ -0,0 +1,86 @@ +/** + * 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.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.security.token.Token; + +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 static final int[] EMPTY_INDICES = {}; + private static final Token EMPTY_TOKEN = new Token<>(); + + private int[] blockIndices; + private Token[] blockTokens; + + 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); + + if (indices == null) { + this.blockIndices = EMPTY_INDICES; + } else { + this.blockIndices = new int[indices.length]; + System.arraycopy(indices, 0, blockIndices, 0, indices.length); + } + blockTokens = new Token[blockIndices.length]; + for (int i = 0; i < blockIndices.length; i++) { + blockTokens[i] = EMPTY_TOKEN; + } + } + + @Override + public String toString() { + return getClass().getSimpleName() + "{" + getBlock() + + "; getBlockSize()=" + getBlockSize() + + "; corrupt=" + isCorrupt() + + "; offset=" + getStartOffset() + + "; locs=" + Arrays.asList(getLocations()) + + "; indices=" + Arrays.toString(blockIndices) + + "}"; + } + + public int[] getBlockIndices() { + return this.blockIndices; + } + + @Override + public boolean isStriped() { + return true; + } + + public Token[] getBlockTokens() { + return blockTokens; + } + + public void setBlockTokens(Token[] tokens) { + this.blockTokens = tokens; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java index ac19d44cd4253..813ea266cd07f 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocol/SnapshottableDirectoryStatus.java @@ -61,7 +61,7 @@ public SnapshottableDirectoryStatus(long modification_time, long access_time, int snapshotNumber, int snapshotQuota, byte[] parentFullPath) { this.dirStatus = new HdfsFileStatus(0, true, 0, 0, modification_time, access_time, permission, owner, group, null, localName, inodeId, - childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED); + childrenNum, null, HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); this.snapshotNumber = snapshotNumber; this.snapshotQuota = snapshotQuota; this.parentFullPath = parentFullPath; diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java index f4ce46d896c9d..7b02691b677cf 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/ClientNamenodeProtocolTranslatorPB.java @@ -161,10 +161,16 @@ 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.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.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; 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; @@ -176,6 +182,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.ipc.ProtobufHelper; import org.apache.hadoop.ipc.ProtocolMetaInterface; import org.apache.hadoop.ipc.ProtocolTranslator; @@ -233,6 +240,10 @@ public class ClientNamenodeProtocolTranslatorPB implements VOID_GET_STORAGE_POLICIES_REQUEST = GetStoragePoliciesRequestProto.newBuilder().build(); + private final static GetErasureCodingPoliciesRequestProto + VOID_GET_EC_POLICIES_REQUEST = GetErasureCodingPoliciesRequestProto + .newBuilder().build(); + public ClientNamenodeProtocolTranslatorPB(ClientNamenodeProtocolPB proxy) { rpcProxy = proxy; } @@ -328,7 +339,7 @@ public LastBlockWithStatus append(String src, String clientName, try { AppendResponseProto res = rpcProxy.append(null, req); LocatedBlock lastBlock = res.hasBlock() ? PBHelperClient - .convert(res.getBlock()) : null; + .convertLocatedBlockProto(res.getBlock()) : null; HdfsFileStatus stat = (res.hasStat()) ? PBHelperClient.convert(res.getStat()) : null; return new LastBlockWithStatus(lastBlock, stat); @@ -416,7 +427,8 @@ public LocatedBlock addBlock(String src, String clientName, req.addAllFavoredNodes(Arrays.asList(favoredNodes)); } try { - return PBHelperClient.convert(rpcProxy.addBlock(null, req.build()).getBlock()); + return PBHelperClient.convertLocatedBlockProto( + rpcProxy.addBlock(null, req.build()).getBlock()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } @@ -441,8 +453,8 @@ public LocatedBlock getAdditionalDatanode(String src, long fileId, .setClientName(clientName) .build(); try { - return PBHelperClient.convert(rpcProxy.getAdditionalDatanode(null, req) - .getBlock()); + return PBHelperClient.convertLocatedBlockProto( + rpcProxy.getAdditionalDatanode(null, req).getBlock()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); } @@ -469,7 +481,7 @@ public boolean complete(String src, String clientName, @Override public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { ReportBadBlocksRequestProto req = ReportBadBlocksRequestProto.newBuilder() - .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlock(blocks))) + .addAllBlocks(Arrays.asList(PBHelperClient.convertLocatedBlocks(blocks))) .build(); try { rpcProxy.reportBadBlocks(null, req); @@ -901,7 +913,7 @@ public LocatedBlock updateBlockForPipeline(ExtendedBlock block, .setClientName(clientName) .build(); try { - return PBHelperClient.convert( + return PBHelperClient.convertLocatedBlockProto( rpcProxy.updateBlockForPipeline(null, req).getBlock()); } catch (ServiceException e) { throw ProtobufHelper.getRemoteException(e); @@ -1406,6 +1418,23 @@ public BatchedEntries listEncryptionZones(long id) } } + @Override + public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) + throws IOException { + final SetErasureCodingPolicyRequestProto.Builder builder = + SetErasureCodingPolicyRequestProto.newBuilder(); + builder.setSrc(src); + if (ecPolicy != null) { + builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy)); + } + SetErasureCodingPolicyRequestProto req = builder.build(); + try { + rpcProxy.setErasureCodingPolicy(null, req); + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + @Override public void setXAttr(String src, XAttr xAttr, EnumSet flag) throws IOException { @@ -1528,4 +1557,37 @@ public EventBatchList getEditsFromTxid(long txid) throws IOException { throw ProtobufHelper.getRemoteException(e); } } + + @Override + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { + try { + GetErasureCodingPoliciesResponseProto response = rpcProxy + .getErasureCodingPolicies(null, VOID_GET_EC_POLICIES_REQUEST); + ErasureCodingPolicy[] ecPolicies = + new ErasureCodingPolicy[response.getEcPoliciesCount()]; + int i = 0; + for (ErasureCodingPolicyProto ecPolicyProto : response.getEcPoliciesList()) { + ecPolicies[i++] = PBHelperClient.convertErasureCodingPolicy(ecPolicyProto); + } + return ecPolicies; + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } + + @Override + public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException { + GetErasureCodingPolicyRequestProto req = GetErasureCodingPolicyRequestProto.newBuilder() + .setSrc(src).build(); + try { + GetErasureCodingPolicyResponseProto response = rpcProxy.getErasureCodingPolicy( + null, req); + if (response.hasEcPolicy()) { + return PBHelperClient.convertErasureCodingPolicy(response.getEcPolicy()); + } + return null; + } catch (ServiceException e) { + throw ProtobufHelper.getRemoteException(e); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java index ae0a3f6a1de99..a7cacc7c19a3a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/protocolPB/PBHelperClient.java @@ -23,7 +23,10 @@ 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.Set; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; @@ -31,8 +34,6 @@ import com.google.protobuf.ByteString; import com.google.protobuf.CodedInputStream; -import static com.google.common.base.Preconditions.checkNotNull; - import org.apache.hadoop.crypto.CipherOption; import org.apache.hadoop.crypto.CipherSuite; import org.apache.hadoop.crypto.CryptoProtocolVersion; @@ -71,6 +72,7 @@ import org.apache.hadoop.hdfs.protocol.DatanodeLocalInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; import org.apache.hadoop.hdfs.protocol.EncryptionZone; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -81,6 +83,7 @@ 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; @@ -112,6 +115,8 @@ 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.EncryptionZonesProtos.EncryptionZoneProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.BlockECRecoveryInfoProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto; @@ -128,6 +133,7 @@ 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.ExtendedBlockProto; +import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.ErasureCodingPolicyProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsPermissionProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.FsServerDefaultsProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.HdfsFileStatusProto; @@ -161,6 +167,7 @@ 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; @@ -497,7 +504,7 @@ public static List convertCipherOptionProtos( return null; } - public static LocatedBlock convert(LocatedBlockProto proto) { + public static LocatedBlock convertLocatedBlockProto(LocatedBlockProto proto) { if (proto == null) return null; List locs = proto.getLocsList(); DatanodeInfo[] targets = new DatanodeInfo[locs.size()]; @@ -517,8 +524,17 @@ public static LocatedBlock convert(LocatedBlockProto proto) { 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 cachedLocs = new ArrayList<>(locs.size()); + List cachedLocs = new ArrayList(locs.size()); List isCachedList = proto.getIsCachedList(); for (int i=0; i tokenProtos = proto.getBlockTokensList(); + Token[] blockTokens = new Token[indices.length]; + for (int i = 0; i < indices.length; i++) { + blockTokens[i] = convert(tokenProtos.get(i)); + } + ((LocatedStripedBlock) lb).setBlockTokens(blockTokens); + } lb.setBlockToken(convert(proto.getBlockToken())); return lb; @@ -591,11 +621,12 @@ public static AdminStates convert(AdminState adminState) { public static LocatedBlocks convert(LocatedBlocksProto lb) { return new LocatedBlocks( lb.getFileLength(), lb.getUnderConstruction(), - convertLocatedBlock(lb.getBlocksList()), - lb.hasLastBlock() ? convert(lb.getLastBlock()) : null, + convertLocatedBlocks(lb.getBlocksList()), + lb.hasLastBlock() ? + convertLocatedBlockProto(lb.getLastBlock()) : null, lb.getIsLastBlockComplete(), - lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : - null); + lb.hasFileEncryptionInfo() ? convert(lb.getFileEncryptionInfo()) : null, + lb.hasEcPolicy() ? convertErasureCodingPolicy(lb.getEcPolicy()) : null); } public static BlockStoragePolicy[] convertStoragePolicies( @@ -716,23 +747,41 @@ public static EventBatchList convert(GetEditsFromTxidResponseProto resp) throws } // Located Block Arrays and Lists - public static LocatedBlockProto[] convertLocatedBlock(LocatedBlock[] lb) { + public static LocatedBlockProto[] convertLocatedBlocks(LocatedBlock[] lb) { + if (lb == null) return null; + return convertLocatedBlocks2(Arrays.asList(lb)) + .toArray(new LocatedBlockProto[lb.length]); + } + + public static LocatedBlock[] convertLocatedBlocks(LocatedBlockProto[] lb) { + if (lb == null) return null; + return convertLocatedBlocks(Arrays.asList(lb)) + .toArray(new LocatedBlock[lb.length]); + } + + public static List convertLocatedBlocks( + List lb) { if (lb == null) return null; - return convertLocatedBlock2(Arrays.asList(lb)).toArray( - new LocatedBlockProto[lb.length]); + final int len = lb.size(); + List result = new ArrayList<>(len); + for (LocatedBlockProto aLb : lb) { + result.add(convertLocatedBlockProto(aLb)); + } + return result; } - public static List convertLocatedBlock2(List lb) { + public static List convertLocatedBlocks2( + List lb) { if (lb == null) return null; final int len = lb.size(); List result = new ArrayList<>(len); - for (int i = 0; i < len; ++i) { - result.add(convert(lb.get(i))); + for (LocatedBlock aLb : lb) { + result.add(convertLocatedBlock(aLb)); } return result; } - 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(); @@ -740,7 +789,7 @@ public static LocatedBlockProto convert(LocatedBlock b) { Lists.newLinkedList(Arrays.asList(b.getCachedLocations())); for (int i = 0; i < locs.length; i++) { DatanodeInfo loc = locs[i]; - builder.addLocs(i, convert(loc)); + builder.addLocs(i, PBHelperClient.convert(loc)); boolean locIsCached = cachedLocs.contains(loc); builder.addIsCached(locIsCached); if (locIsCached) { @@ -749,21 +798,30 @@ public static LocatedBlockProto convert(LocatedBlock b) { } Preconditions.checkArgument(cachedLocs.size() == 0, "Found additional cached replica locations that are not in the set of" - + " storage-backed locations!"); + + " storage-backed locations!"); StorageType[] storageTypes = b.getStorageTypes(); if (storageTypes != null) { - for (int i = 0; i < storageTypes.length; ++i) { - builder.addStorageTypes(convertStorageType(storageTypes[i])); + for (StorageType storageType : storageTypes) { + builder.addStorageTypes(convertStorageType(storageType)); } } final String[] storageIDs = b.getStorageIDs(); if (storageIDs != null) { builder.addAllStorageIDs(Arrays.asList(storageIDs)); } + if (b instanceof LocatedStripedBlock) { + LocatedStripedBlock sb = (LocatedStripedBlock) b; + int[] indices = sb.getBlockIndices(); + Token[] blockTokens = sb.getBlockTokens(); + for (int i = 0; i < indices.length; i++) { + builder.addBlockIndex(indices[i]); + builder.addBlockTokens(PBHelperClient.convert(blockTokens[i])); + } + } - return builder.setB(convert(b.getBlock())) - .setBlockToken(convert(b.getBlockToken())) + return builder.setB(PBHelperClient.convert(b.getBlock())) + .setBlockToken(PBHelperClient.convert(b.getBlockToken())) .setCorrupt(b.isCorrupt()).setOffset(b.getStartOffset()).build(); } @@ -897,7 +955,7 @@ public static List convertLocatedBlock( final int len = lb.size(); List result = new ArrayList<>(len); for (int i = 0; i < len; ++i) { - result.add(convert(lb.get(i))); + result.add(convertLocatedBlockProto(lb.get(i))); } return result; } @@ -1087,7 +1145,7 @@ public static CachePoolEntry convert(CachePoolEntryProto proto) { public static CachePoolInfo convert (CachePoolInfoProto proto) { // Pool name is a required field, the rest are optional - String poolName = checkNotNull(proto.getPoolName()); + String poolName = Preconditions.checkNotNull(proto.getPoolName()); CachePoolInfo info = new CachePoolInfo(poolName); if (proto.hasOwnerName()) { info.setOwnerName(proto.getOwnerName()); @@ -1341,7 +1399,8 @@ public static HdfsFileStatus convert(HdfsFileStatusProto fs) { fs.hasChildrenNum() ? fs.getChildrenNum() : -1, fs.hasFileEncryptionInfo() ? convert(fs.getFileEncryptionInfo()) : null, fs.hasStoragePolicy() ? (byte) fs.getStoragePolicy() - : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED); + : HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, + fs.hasEcPolicy() ? convertErasureCodingPolicy(fs.getEcPolicy()) : null); } public static CorruptFileBlocks convert(CorruptFileBlocksProto c) { @@ -1655,14 +1714,18 @@ public static LocatedBlocksProto convert(LocatedBlocks lb) { LocatedBlocksProto.Builder builder = LocatedBlocksProto.newBuilder(); if (lb.getLastLocatedBlock() != null) { - builder.setLastBlock(convert(lb.getLastLocatedBlock())); + builder.setLastBlock( + convertLocatedBlock(lb.getLastLocatedBlock())); } if (lb.getFileEncryptionInfo() != null) { builder.setFileEncryptionInfo(convert(lb.getFileEncryptionInfo())); } + if (lb.getErasureCodingPolicy() != null) { + builder.setEcPolicy(convertErasureCodingPolicy(lb.getErasureCodingPolicy())); + } return builder.setFileLength(lb.getFileLength()) .setUnderConstruction(lb.isUnderConstruction()) - .addAllBlocks(convertLocatedBlock2(lb.getLocatedBlocks())) + .addAllBlocks(convertLocatedBlocks2(lb.getLocatedBlocks())) .setIsLastBlockComplete(lb.isLastBlockComplete()).build(); } @@ -1763,6 +1826,9 @@ public static HdfsFileStatusProto convert(HdfsFileStatus fs) { builder.setLocations(convert(locations)); } } + if(fs.getErasureCodingPolicy() != null) { + builder.setEcPolicy(convertErasureCodingPolicy(fs.getErasureCodingPolicy())); + } return builder.build(); } @@ -2326,4 +2392,44 @@ static List convert(DatanodeInfo[][] targets) { } return Arrays.asList(ret); } + + public static ECSchema convertECSchema(HdfsProtos.ECSchemaProto schema) { + List optionsList = schema.getOptionsList(); + Map options = new HashMap<>(optionsList.size()); + for (HdfsProtos.ECSchemaOptionEntryProto option : optionsList) { + options.put(option.getKey(), option.getValue()); + } + return new ECSchema(schema.getCodecName(), schema.getDataUnits(), + schema.getParityUnits(), options); + } + + public static HdfsProtos.ECSchemaProto convertECSchema(ECSchema schema) { + HdfsProtos.ECSchemaProto.Builder builder = HdfsProtos.ECSchemaProto.newBuilder() + .setCodecName(schema.getCodecName()) + .setDataUnits(schema.getNumDataUnits()) + .setParityUnits(schema.getNumParityUnits()); + Set> entrySet = schema.getExtraOptions().entrySet(); + for (Map.Entry entry : entrySet) { + builder.addOptions(HdfsProtos.ECSchemaOptionEntryProto.newBuilder() + .setKey(entry.getKey()).setValue(entry.getValue()).build()); + } + return builder.build(); + } + + public static ErasureCodingPolicy convertErasureCodingPolicy( + ErasureCodingPolicyProto policy) { + return new ErasureCodingPolicy(policy.getName(), + convertECSchema(policy.getSchema()), + policy.getCellSize()); + } + + public static ErasureCodingPolicyProto convertErasureCodingPolicy( + ErasureCodingPolicy policy) { + ErasureCodingPolicyProto.Builder builder = ErasureCodingPolicyProto + .newBuilder() + .setName(policy.getName()) + .setSchema(convertECSchema(policy.getSchema())) + .setCellSize(policy.getCellSize()); + return builder.build(); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java new file mode 100644 index 0000000000000..264c532243dd7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/util/StripedBlockUtil.java @@ -0,0 +1,952 @@ +/** + * 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.util; + +import com.google.common.annotations.VisibleForTesting; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.DFSClient; +import org.apache.hadoop.hdfs.DFSStripedOutputStream; +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.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.security.token.Token; + +import java.nio.ByteBuffer; +import java.util.*; +import java.io.IOException; +import java.util.concurrent.CancellationException; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutionException; +import java.util.concurrent.Future; +import java.util.concurrent.TimeUnit; + +/** + * When accessing a file in striped layout, operations on logical byte ranges + * in the file need to be mapped to physical byte ranges on block files stored + * on DataNodes. This utility class facilities this mapping by defining and + * exposing a number of striping-related concepts. The most basic ones are + * illustrated in the following diagram. Unless otherwise specified, all + * range-related calculations are inclusive (the end offset of the previous + * range should be 1 byte lower than the start offset of the next one). + * + * | <---- Block Group ----> | <- Block Group: logical unit composing + * | | striped HDFS files. + * blk_0 blk_1 blk_2 <- Internal Blocks: each internal block + * | | | represents a physically stored local + * v v v block file + * +------+ +------+ +------+ + * |cell_0| |cell_1| |cell_2| <- {@link StripingCell} represents the + * +------+ +------+ +------+ logical order that a Block Group should + * |cell_3| |cell_4| |cell_5| be accessed: cell_0, cell_1, ... + * +------+ +------+ +------+ + * |cell_6| |cell_7| |cell_8| + * +------+ +------+ +------+ + * |cell_9| + * +------+ <- A cell contains cellSize bytes of data + */ +@InterfaceAudience.Private +public class StripedBlockUtil { + + /** + * This method parses a striped block group into individual blocks. + * + * @param bg The striped block group + * @param cellSize The size of a striping cell + * @param dataBlkNum The number of data blocks + * @return An array containing the blocks in the group + */ + public static LocatedBlock[] parseStripedBlockGroup(LocatedStripedBlock bg, + int cellSize, int dataBlkNum, int parityBlkNum) { + int locatedBGSize = bg.getBlockIndices().length; + LocatedBlock[] lbs = new LocatedBlock[dataBlkNum + parityBlkNum]; + for (short i = 0; i < locatedBGSize; i++) { + final int idx = bg.getBlockIndices()[i]; + // for now we do not use redundant replica of an internal block + if (idx < (dataBlkNum + parityBlkNum) && lbs[idx] == null) { + lbs[idx] = constructInternalBlock(bg, i, cellSize, + dataBlkNum, idx); + } + } + return lbs; + } + + /** + * This method creates an internal block at the given index of a block group + * + * @param idxInReturnedLocs The index in the stored locations in the + * {@link LocatedStripedBlock} object + * @param idxInBlockGroup The logical index in the striped block group + * @return The constructed internal block + */ + public static LocatedBlock constructInternalBlock(LocatedStripedBlock bg, + int idxInReturnedLocs, int cellSize, int dataBlkNum, + int idxInBlockGroup) { + final ExtendedBlock blk = constructInternalBlock( + bg.getBlock(), cellSize, dataBlkNum, idxInBlockGroup); + final LocatedBlock locatedBlock; + if (idxInReturnedLocs < bg.getLocations().length) { + locatedBlock = new LocatedBlock(blk, + new DatanodeInfo[]{bg.getLocations()[idxInReturnedLocs]}, + new String[]{bg.getStorageIDs()[idxInReturnedLocs]}, + new StorageType[]{bg.getStorageTypes()[idxInReturnedLocs]}, + bg.getStartOffset(), bg.isCorrupt(), null); + } else { + locatedBlock = new LocatedBlock(blk, null, null, null, + bg.getStartOffset(), bg.isCorrupt(), null); + } + Token[] blockTokens = bg.getBlockTokens(); + if (idxInReturnedLocs < blockTokens.length) { + locatedBlock.setBlockToken(blockTokens[idxInReturnedLocs]); + } + return locatedBlock; + } + + /** + * This method creates an internal {@link ExtendedBlock} at the given index + * of a block group. + */ + public static ExtendedBlock constructInternalBlock(ExtendedBlock blockGroup, + int cellSize, int dataBlkNum, int idxInBlockGroup) { + ExtendedBlock block = new ExtendedBlock(blockGroup); + block.setBlockId(blockGroup.getBlockId() + idxInBlockGroup); + block.setNumBytes(getInternalBlockLength(blockGroup.getNumBytes(), + cellSize, dataBlkNum, idxInBlockGroup)); + return block; + } + + /** + * Get the size of an internal block at the given index of a block group + * + * @param dataSize Size of the block group only counting data blocks + * @param cellSize The size of a striping cell + * @param numDataBlocks The number of data blocks + * @param i The logical index in the striped block group + * @return The size of the internal block at the specified index + */ + public static long getInternalBlockLength(long dataSize, + int cellSize, int numDataBlocks, int i) { + Preconditions.checkArgument(dataSize >= 0); + Preconditions.checkArgument(cellSize > 0); + Preconditions.checkArgument(numDataBlocks > 0); + Preconditions.checkArgument(i >= 0); + // Size of each stripe (only counting data blocks) + final int stripeSize = cellSize * numDataBlocks; + // If block group ends at stripe boundary, each internal block has an equal + // share of the group + final int lastStripeDataLen = (int)(dataSize % stripeSize); + if (lastStripeDataLen == 0) { + return dataSize / numDataBlocks; + } + + final int numStripes = (int) ((dataSize - 1) / stripeSize + 1); + return (numStripes - 1L)*cellSize + + lastCellSize(lastStripeDataLen, cellSize, numDataBlocks, i); + } + + private static int lastCellSize(int size, int cellSize, int numDataBlocks, + int i) { + if (i < numDataBlocks) { + // parity block size (i.e. i >= numDataBlocks) is the same as + // the first data block size (i.e. i = 0). + size -= i*cellSize; + if (size < 0) { + size = 0; + } + } + return size > cellSize? cellSize: size; + } + + /** + * Given a byte's offset in an internal block, calculate the offset in + * the block group + */ + public static long offsetInBlkToOffsetInBG(int cellSize, int dataBlkNum, + long offsetInBlk, int idxInBlockGroup) { + int cellIdxInBlk = (int) (offsetInBlk / cellSize); + return cellIdxInBlk * cellSize * dataBlkNum // n full stripes before offset + + idxInBlockGroup * cellSize // m full cells before offset + + offsetInBlk % cellSize; // partial cell + } + + /** + * Get the next completed striped read task + * + * @return {@link StripingChunkReadResult} indicating the status of the read task + * succeeded, and the block index of the task. If the method times + * out without getting any completed read tasks, -1 is returned as + * block index. + * @throws InterruptedException + */ + public static StripingChunkReadResult getNextCompletedStripedRead( + CompletionService readService, Map, Integer> futures, + final long timeoutMillis) throws InterruptedException { + Preconditions.checkArgument(!futures.isEmpty()); + Future future = null; + try { + if (timeoutMillis > 0) { + future = readService.poll(timeoutMillis, TimeUnit.MILLISECONDS); + } else { + future = readService.take(); + } + if (future != null) { + future.get(); + return new StripingChunkReadResult(futures.remove(future), + StripingChunkReadResult.SUCCESSFUL); + } else { + return new StripingChunkReadResult(StripingChunkReadResult.TIMEOUT); + } + } catch (ExecutionException e) { + if (DFSClient.LOG.isDebugEnabled()) { + DFSClient.LOG.debug("ExecutionException " + e); + } + return new StripingChunkReadResult(futures.remove(future), + StripingChunkReadResult.FAILED); + } catch (CancellationException e) { + return new StripingChunkReadResult(futures.remove(future), + StripingChunkReadResult.CANCELLED); + } + } + + /** + * Get the total usage of the striped blocks, which is the total of data + * blocks and parity blocks + * + * @param numDataBlkBytes + * Size of the block group only counting data blocks + * @param dataBlkNum + * The number of data blocks + * @param parityBlkNum + * The number of parity blocks + * @param cellSize + * The size of a striping cell + * @return The total usage of data blocks and parity blocks + */ + public static long spaceConsumedByStripedBlock(long numDataBlkBytes, + int dataBlkNum, int parityBlkNum, int cellSize) { + int parityIndex = dataBlkNum + 1; + long numParityBlkBytes = getInternalBlockLength(numDataBlkBytes, cellSize, + dataBlkNum, parityIndex) * parityBlkNum; + return numDataBlkBytes + numParityBlkBytes; + } + + /** + * Initialize the decoding input buffers based on the chunk states in an + * {@link AlignedStripe}. For each chunk that was not initially requested, + * schedule a new fetch request with the decoding input buffer as transfer + * destination. + */ + public static byte[][] initDecodeInputs(AlignedStripe alignedStripe, + int dataBlkNum, int parityBlkNum) { + byte[][] decodeInputs = + new byte[dataBlkNum + parityBlkNum][(int) alignedStripe.getSpanInBlock()]; + // read the full data aligned stripe + for (int i = 0; i < dataBlkNum; i++) { + if (alignedStripe.chunks[i] == null) { + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + alignedStripe.chunks[i] = new StripingChunk(decodeInputs[decodeIndex]); + alignedStripe.chunks[i].addByteArraySlice(0, + (int) alignedStripe.getSpanInBlock()); + } + } + return decodeInputs; + } + + /** + * Some fetched {@link StripingChunk} might be stored in original application + * buffer instead of prepared decode input buffers. Some others are beyond + * the range of the internal blocks and should correspond to all zero bytes. + * When all pending requests have returned, this method should be called to + * finalize decode input buffers. + */ + public static void finalizeDecodeInputs(final byte[][] decodeInputs, + int dataBlkNum, int parityBlkNum, AlignedStripe alignedStripe) { + for (int i = 0; i < alignedStripe.chunks.length; i++) { + final StripingChunk chunk = alignedStripe.chunks[i]; + final int decodeIndex = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + if (chunk != null && chunk.state == StripingChunk.FETCHED) { + chunk.copyTo(decodeInputs[decodeIndex]); + } else if (chunk != null && chunk.state == StripingChunk.ALLZERO) { + Arrays.fill(decodeInputs[decodeIndex], (byte) 0); + } else { + decodeInputs[decodeIndex] = null; + } + } + } + + /** + * Currently decoding requires parity chunks are before data chunks. + * The indices are opposite to what we store in NN. In future we may + * improve the decoding to make the indices order the same as in NN. + * + * @param index The index to convert + * @param dataBlkNum The number of data blocks + * @param parityBlkNum The number of parity blocks + * @return converted index + */ + public static int convertIndex4Decode(int index, int dataBlkNum, + int parityBlkNum) { + return index < dataBlkNum ? index + parityBlkNum : index - dataBlkNum; + } + + public static int convertDecodeIndexBack(int index, int dataBlkNum, + int parityBlkNum) { + return index < parityBlkNum ? index + dataBlkNum : index - parityBlkNum; + } + + /** + * Decode based on the given input buffers and erasure coding policy. + */ + public static void decodeAndFillBuffer(final byte[][] decodeInputs, + AlignedStripe alignedStripe, int dataBlkNum, int parityBlkNum, + RawErasureDecoder decoder) { + // Step 1: prepare indices and output buffers for missing data units + int[] decodeIndices = new int[parityBlkNum]; + int pos = 0; + for (int i = 0; i < dataBlkNum; i++) { + if (alignedStripe.chunks[i] != null && + alignedStripe.chunks[i].state == StripingChunk.MISSING){ + decodeIndices[pos++] = convertIndex4Decode(i, dataBlkNum, parityBlkNum); + } + } + decodeIndices = Arrays.copyOf(decodeIndices, pos); + byte[][] decodeOutputs = + new byte[decodeIndices.length][(int) alignedStripe.getSpanInBlock()]; + + // Step 2: decode into prepared output buffers + decoder.decode(decodeInputs, decodeIndices, decodeOutputs); + + // Step 3: fill original application buffer with decoded data + for (int i = 0; i < decodeIndices.length; i++) { + int missingBlkIdx = convertDecodeIndexBack(decodeIndices[i], + dataBlkNum, parityBlkNum); + StripingChunk chunk = alignedStripe.chunks[missingBlkIdx]; + if (chunk.state == StripingChunk.MISSING) { + chunk.copyFrom(decodeOutputs[i]); + } + } + } + + /** + * Similar functionality with {@link #divideByteRangeIntoStripes}, but is used + * by stateful read and uses ByteBuffer as reading target buffer. Besides the + * read range is within a single stripe thus the calculation logic is simpler. + */ + public static AlignedStripe[] divideOneStripe(ErasureCodingPolicy ecPolicy, + int cellSize, LocatedStripedBlock blockGroup, long rangeStartInBlockGroup, + long rangeEndInBlockGroup, ByteBuffer buf) { + final int dataBlkNum = ecPolicy.getNumDataUnits(); + // Step 1: map the byte range to StripingCells + StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize, + blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup); + + // Step 2: get the unmerged ranges on each internal block + VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize, + cells); + + // Step 3: merge into stripes + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges); + + // Step 4: calculate each chunk's position in destination buffer. Since the + // whole read range is within a single stripe, the logic is simpler here. + int bufOffset = (int) (rangeStartInBlockGroup % ((long) cellSize * dataBlkNum)); + for (StripingCell cell : cells) { + long cellStart = cell.idxInInternalBlk * cellSize + cell.offset; + long cellEnd = cellStart + cell.size - 1; + for (AlignedStripe s : stripes) { + long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1; + long overlapStart = Math.max(cellStart, s.getOffsetInBlock()); + long overlapEnd = Math.min(cellEnd, stripeEnd); + int overLapLen = (int) (overlapEnd - overlapStart + 1); + if (overLapLen > 0) { + Preconditions.checkState(s.chunks[cell.idxInStripe] == null); + final int pos = (int) (bufOffset + overlapStart - cellStart); + buf.position(pos); + buf.limit(pos + overLapLen); + s.chunks[cell.idxInStripe] = new StripingChunk(buf.slice()); + } + } + bufOffset += cell.size; + } + + // Step 5: prepare ALLZERO blocks + prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum); + return stripes; + } + + /** + * This method divides a requested byte range into an array of inclusive + * {@link AlignedStripe}. + * @param ecPolicy The codec policy for the file, which carries the numbers + * of data / parity blocks + * @param cellSize Cell size of stripe + * @param blockGroup The striped block group + * @param rangeStartInBlockGroup The byte range's start offset in block group + * @param rangeEndInBlockGroup The byte range's end offset in block group + * @param buf Destination buffer of the read operation for the byte range + * @param offsetInBuf Start offset into the destination buffer + * + * At most 5 stripes will be generated from each logical range, as + * demonstrated in the header of {@link AlignedStripe}. + */ + public static AlignedStripe[] divideByteRangeIntoStripes(ErasureCodingPolicy ecPolicy, + int cellSize, LocatedStripedBlock blockGroup, + long rangeStartInBlockGroup, long rangeEndInBlockGroup, byte[] buf, + int offsetInBuf) { + + // Step 0: analyze range and calculate basic parameters + final int dataBlkNum = ecPolicy.getNumDataUnits(); + + // Step 1: map the byte range to StripingCells + StripingCell[] cells = getStripingCellsOfByteRange(ecPolicy, cellSize, + blockGroup, rangeStartInBlockGroup, rangeEndInBlockGroup); + + // Step 2: get the unmerged ranges on each internal block + VerticalRange[] ranges = getRangesForInternalBlocks(ecPolicy, cellSize, + cells); + + // Step 3: merge into at most 5 stripes + AlignedStripe[] stripes = mergeRangesForInternalBlocks(ecPolicy, ranges); + + // Step 4: calculate each chunk's position in destination buffer + calcualteChunkPositionsInBuf(cellSize, stripes, cells, buf, offsetInBuf); + + // Step 5: prepare ALLZERO blocks + prepareAllZeroChunks(blockGroup, stripes, cellSize, dataBlkNum); + + return stripes; + } + + /** + * Map the logical byte range to a set of inclusive {@link StripingCell} + * instances, each representing the overlap of the byte range to a cell + * used by {@link DFSStripedOutputStream} in encoding + */ + @VisibleForTesting + private static StripingCell[] getStripingCellsOfByteRange(ErasureCodingPolicy ecPolicy, + int cellSize, LocatedStripedBlock blockGroup, + long rangeStartInBlockGroup, long rangeEndInBlockGroup) { + Preconditions.checkArgument( + rangeStartInBlockGroup <= rangeEndInBlockGroup && + rangeEndInBlockGroup < blockGroup.getBlockSize()); + long len = rangeEndInBlockGroup - rangeStartInBlockGroup + 1; + int firstCellIdxInBG = (int) (rangeStartInBlockGroup / cellSize); + int lastCellIdxInBG = (int) (rangeEndInBlockGroup / cellSize); + int numCells = lastCellIdxInBG - firstCellIdxInBG + 1; + StripingCell[] cells = new StripingCell[numCells]; + + final int firstCellOffset = (int) (rangeStartInBlockGroup % cellSize); + final int firstCellSize = + (int) Math.min(cellSize - (rangeStartInBlockGroup % cellSize), len); + cells[0] = new StripingCell(ecPolicy, firstCellSize, firstCellIdxInBG, + firstCellOffset); + if (lastCellIdxInBG != firstCellIdxInBG) { + final int lastCellSize = (int) (rangeEndInBlockGroup % cellSize) + 1; + cells[numCells - 1] = new StripingCell(ecPolicy, lastCellSize, + lastCellIdxInBG, 0); + } + + for (int i = 1; i < numCells - 1; i++) { + cells[i] = new StripingCell(ecPolicy, cellSize, i + firstCellIdxInBG, 0); + } + + return cells; + } + + /** + * Given a logical byte range, mapped to each {@link StripingCell}, calculate + * the physical byte range (inclusive) on each stored internal block. + */ + @VisibleForTesting + private static VerticalRange[] getRangesForInternalBlocks(ErasureCodingPolicy ecPolicy, + int cellSize, StripingCell[] cells) { + int dataBlkNum = ecPolicy.getNumDataUnits(); + int parityBlkNum = ecPolicy.getNumParityUnits(); + + VerticalRange ranges[] = new VerticalRange[dataBlkNum + parityBlkNum]; + + long earliestStart = Long.MAX_VALUE; + long latestEnd = -1; + for (StripingCell cell : cells) { + // iterate through all cells and update the list of StripeRanges + if (ranges[cell.idxInStripe] == null) { + ranges[cell.idxInStripe] = new VerticalRange( + cell.idxInInternalBlk * cellSize + cell.offset, cell.size); + } else { + ranges[cell.idxInStripe].spanInBlock += cell.size; + } + VerticalRange range = ranges[cell.idxInStripe]; + if (range.offsetInBlock < earliestStart) { + earliestStart = range.offsetInBlock; + } + if (range.offsetInBlock + range.spanInBlock - 1 > latestEnd) { + latestEnd = range.offsetInBlock + range.spanInBlock - 1; + } + } + + // Each parity block should be fetched at maximum range of all data blocks + for (int i = dataBlkNum; i < dataBlkNum + parityBlkNum; i++) { + ranges[i] = new VerticalRange(earliestStart, + latestEnd - earliestStart + 1); + } + + return ranges; + } + + /** + * Merge byte ranges on each internal block into a set of inclusive + * {@link AlignedStripe} instances. + */ + private static AlignedStripe[] mergeRangesForInternalBlocks( + ErasureCodingPolicy ecPolicy, VerticalRange[] ranges) { + int dataBlkNum = ecPolicy.getNumDataUnits(); + int parityBlkNum = ecPolicy.getNumParityUnits(); + List stripes = new ArrayList<>(); + SortedSet stripePoints = new TreeSet<>(); + for (VerticalRange r : ranges) { + if (r != null) { + stripePoints.add(r.offsetInBlock); + stripePoints.add(r.offsetInBlock + r.spanInBlock); + } + } + + long prev = -1; + for (long point : stripePoints) { + if (prev >= 0) { + stripes.add(new AlignedStripe(prev, point - prev, + dataBlkNum + parityBlkNum)); + } + prev = point; + } + return stripes.toArray(new AlignedStripe[stripes.size()]); + } + + private static void calcualteChunkPositionsInBuf(int cellSize, + AlignedStripe[] stripes, StripingCell[] cells, byte[] buf, + int offsetInBuf) { + /** + * | <--------------- AlignedStripe --------------->| + * + * |<- length_0 ->|<-- length_1 -->|<- length_2 ->| + * +------------------+------------------+----------------+ + * | cell_0_0_0 | cell_3_1_0 | cell_6_2_0 | <- blk_0 + * +------------------+------------------+----------------+ + * _/ \_______________________ + * | | + * v offset_0 v offset_1 + * +----------------------------------------------------------+ + * | cell_0_0_0 | cell_1_0_1 and cell_2_0_2 |cell_3_1_0 ...| <- buf + * | (partial) | (from blk_1 and blk_2) | | + * +----------------------------------------------------------+ + * + * Cell indexing convention defined in {@link StripingCell} + */ + int done = 0; + for (StripingCell cell : cells) { + long cellStart = cell.idxInInternalBlk * cellSize + cell.offset; + long cellEnd = cellStart + cell.size - 1; + for (AlignedStripe s : stripes) { + long stripeEnd = s.getOffsetInBlock() + s.getSpanInBlock() - 1; + long overlapStart = Math.max(cellStart, s.getOffsetInBlock()); + long overlapEnd = Math.min(cellEnd, stripeEnd); + int overLapLen = (int) (overlapEnd - overlapStart + 1); + if (overLapLen <= 0) { + continue; + } + if (s.chunks[cell.idxInStripe] == null) { + s.chunks[cell.idxInStripe] = new StripingChunk(buf); + } + s.chunks[cell.idxInStripe].addByteArraySlice( + (int)(offsetInBuf + done + overlapStart - cellStart), overLapLen); + } + done += cell.size; + } + } + + /** + * If a {@link StripingChunk} maps to a byte range beyond an internal block's + * size, the chunk should be treated as zero bytes in decoding. + */ + private static void prepareAllZeroChunks(LocatedStripedBlock blockGroup, + AlignedStripe[] stripes, int cellSize, int dataBlkNum) { + for (AlignedStripe s : stripes) { + for (int i = 0; i < dataBlkNum; i++) { + long internalBlkLen = getInternalBlockLength(blockGroup.getBlockSize(), + cellSize, dataBlkNum, i); + if (internalBlkLen <= s.getOffsetInBlock()) { + Preconditions.checkState(s.chunks[i] == null); + s.chunks[i] = new StripingChunk(StripingChunk.ALLZERO); + } + } + } + } + + /** + * Cell is the unit of encoding used in {@link DFSStripedOutputStream}. This + * size impacts how a logical offset in the file or block group translates + * to physical byte offset in a stored internal block. The StripingCell util + * class facilitates this calculation. Each StripingCell is inclusive with + * its start and end offsets -- e.g., the end logical offset of cell_0_0_0 + * should be 1 byte lower than the start logical offset of cell_1_0_1. + * + * | <------- Striped Block Group -------> | + * blk_0 blk_1 blk_2 + * | | | + * v v v + * +----------+ +----------+ +----------+ + * |cell_0_0_0| |cell_1_0_1| |cell_2_0_2| + * +----------+ +----------+ +----------+ + * |cell_3_1_0| |cell_4_1_1| |cell_5_1_2| <- {@link #idxInBlkGroup} = 5 + * +----------+ +----------+ +----------+ {@link #idxInInternalBlk} = 1 + * {@link #idxInStripe} = 2 + * A StripingCell is a special instance of {@link StripingChunk} whose offset + * and size align with the cell used when writing data. + * TODO: consider parity cells + */ + @VisibleForTesting + static class StripingCell { + final ErasureCodingPolicy ecPolicy; + /** Logical order in a block group, used when doing I/O to a block group */ + final int idxInBlkGroup; + final int idxInInternalBlk; + final int idxInStripe; + /** + * When a logical byte range is mapped to a set of cells, it might + * partially overlap with the first and last cells. This field and the + * {@link #size} variable represent the start offset and size of the + * overlap. + */ + final int offset; + final int size; + + StripingCell(ErasureCodingPolicy ecPolicy, int cellSize, int idxInBlkGroup, + int offset) { + this.ecPolicy = ecPolicy; + this.idxInBlkGroup = idxInBlkGroup; + this.idxInInternalBlk = idxInBlkGroup / ecPolicy.getNumDataUnits(); + this.idxInStripe = idxInBlkGroup - + this.idxInInternalBlk * ecPolicy.getNumDataUnits(); + this.offset = offset; + this.size = cellSize; + } + } + + /** + * Given a requested byte range on a striped block group, an AlignedStripe + * represents an inclusive {@link VerticalRange} that is aligned with both + * the byte range and boundaries of all internal blocks. As illustrated in + * the diagram, any given byte range on a block group leads to 1~5 + * AlignedStripe's. + * + * |<-------- Striped Block Group -------->| + * blk_0 blk_1 blk_2 blk_3 blk_4 + * +----+ | +----+ +----+ + * |full| | | | | | <- AlignedStripe0: + * +----+ |~~~~| | |~~~~| |~~~~| 1st cell is partial + * |part| | | | | | | | <- AlignedStripe1: byte range + * +----+ +----+ +----+ | |~~~~| |~~~~| doesn't start at 1st block + * |full| |full| |full| | | | | | + * |cell| |cell| |cell| | | | | | <- AlignedStripe2 (full stripe) + * | | | | | | | | | | | + * +----+ +----+ +----+ | |~~~~| |~~~~| + * |full| |part| | | | | | <- AlignedStripe3: byte range + * |~~~~| +----+ | |~~~~| |~~~~| doesn't end at last block + * | | | | | | | <- AlignedStripe4: + * +----+ | +----+ +----+ last cell is partial + * | + * <---- data blocks ----> | <--- parity ---> + * + * An AlignedStripe is the basic unit of reading from a striped block group, + * because within the AlignedStripe, all internal blocks can be processed in + * a uniform manner. + * + * The coverage of an AlignedStripe on an internal block is represented as a + * {@link StripingChunk}. + * + * To simplify the logic of reading a logical byte range from a block group, + * a StripingChunk is either completely in the requested byte range or + * completely outside the requested byte range. + */ + public static class AlignedStripe { + public VerticalRange range; + /** status of each chunk in the stripe */ + public final StripingChunk[] chunks; + public int fetchedChunksNum = 0; + public int missingChunksNum = 0; + + public AlignedStripe(long offsetInBlock, long length, int width) { + Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0); + this.range = new VerticalRange(offsetInBlock, length); + this.chunks = new StripingChunk[width]; + } + + public boolean include(long pos) { + return range.include(pos); + } + + public long getOffsetInBlock() { + return range.offsetInBlock; + } + + public long getSpanInBlock() { + return range.spanInBlock; + } + + @Override + public String toString() { + return "Offset=" + range.offsetInBlock + ", length=" + range.spanInBlock + + ", fetchedChunksNum=" + fetchedChunksNum + + ", missingChunksNum=" + missingChunksNum; + } + } + + /** + * A simple utility class representing an arbitrary vertical inclusive range + * starting at {@link #offsetInBlock} and lasting for {@link #spanInBlock} + * bytes in an internal block. Note that VerticalRange doesn't necessarily + * align with {@link StripingCell}. + * + * |<- Striped Block Group ->| + * blk_0 + * | + * v + * +-----+ + * |~~~~~| <-- {@link #offsetInBlock} + * | | ^ + * | | | + * | | | {@link #spanInBlock} + * | | v + * |~~~~~| --- + * | | + * +-----+ + */ + public static class VerticalRange { + /** start offset in the block group (inclusive) */ + public long offsetInBlock; + /** length of the stripe range */ + public long spanInBlock; + + public VerticalRange(long offsetInBlock, long length) { + Preconditions.checkArgument(offsetInBlock >= 0 && length >= 0); + this.offsetInBlock = offsetInBlock; + this.spanInBlock = length; + } + + /** whether a position is in the range */ + public boolean include(long pos) { + return pos >= offsetInBlock && pos < offsetInBlock + spanInBlock; + } + } + + /** + * Indicates the coverage of an {@link AlignedStripe} on an internal block, + * and the state of the chunk in the context of the read request. + * + * |<---------------- Striped Block Group --------------->| + * blk_0 blk_1 blk_2 blk_3 blk_4 + * +---------+ | +----+ +----+ + * null null |REQUESTED| | |null| |null| <- AlignedStripe0 + * +---------+ |---------| | |----| |----| + * null |REQUESTED| |REQUESTED| | |null| |null| <- AlignedStripe1 + * +---------+ +---------+ +---------+ | +----+ +----+ + * |REQUESTED| |REQUESTED| ALLZERO | |null| |null| <- AlignedStripe2 + * +---------+ +---------+ | +----+ +----+ + * <----------- data blocks ------------> | <--- parity ---> + */ + public static class StripingChunk { + /** Chunk has been successfully fetched */ + public static final int FETCHED = 0x01; + /** Chunk has encountered failed when being fetched */ + public static final int MISSING = 0x02; + /** Chunk being fetched (fetching task is in-flight) */ + public static final int PENDING = 0x04; + /** + * Chunk is requested either by application or for decoding, need to + * schedule read task + */ + public static final int REQUESTED = 0X08; + /** + * Internal block is short and has no overlap with chunk. Chunk considered + * all-zero bytes in codec calculations. + */ + public static final int ALLZERO = 0X0f; + + /** + * If a chunk is completely in requested range, the state transition is: + * REQUESTED (when AlignedStripe created) -> PENDING -> {FETCHED | MISSING} + * If a chunk is completely outside requested range (including parity + * chunks), state transition is: + * null (AlignedStripe created) -> REQUESTED (upon failure) -> PENDING ... + */ + public int state = REQUESTED; + + public final ChunkByteArray byteArray; + public final ByteBuffer byteBuffer; + + public StripingChunk(byte[] buf) { + this.byteArray = new ChunkByteArray(buf); + byteBuffer = null; + } + + public StripingChunk(ByteBuffer buf) { + this.byteArray = null; + this.byteBuffer = buf; + } + + public StripingChunk(int state) { + this.byteArray = null; + this.byteBuffer = null; + this.state = state; + } + + public void addByteArraySlice(int offset, int length) { + assert byteArray != null; + byteArray.offsetsInBuf.add(offset); + byteArray.lengthsInBuf.add(length); + } + + void copyTo(byte[] target) { + assert byteArray != null; + byteArray.copyTo(target); + } + + void copyFrom(byte[] src) { + assert byteArray != null; + byteArray.copyFrom(src); + } + } + + public static class ChunkByteArray { + private final byte[] buf; + private final List offsetsInBuf; + private final List lengthsInBuf; + + ChunkByteArray(byte[] buf) { + this.buf = buf; + this.offsetsInBuf = new ArrayList<>(); + this.lengthsInBuf = new ArrayList<>(); + } + + public int[] getOffsets() { + int[] offsets = new int[offsetsInBuf.size()]; + for (int i = 0; i < offsets.length; i++) { + offsets[i] = offsetsInBuf.get(i); + } + return offsets; + } + + public int[] getLengths() { + int[] lens = new int[this.lengthsInBuf.size()]; + for (int i = 0; i < lens.length; i++) { + lens[i] = this.lengthsInBuf.get(i); + } + return lens; + } + + public byte[] buf() { + return buf; + } + + void copyTo(byte[] target) { + int posInBuf = 0; + for (int i = 0; i < offsetsInBuf.size(); i++) { + System.arraycopy(buf, offsetsInBuf.get(i), + target, posInBuf, lengthsInBuf.get(i)); + posInBuf += lengthsInBuf.get(i); + } + } + + void copyFrom(byte[] src) { + int srcPos = 0; + for (int j = 0; j < offsetsInBuf.size(); j++) { + System.arraycopy(src, srcPos, buf, offsetsInBuf.get(j), + lengthsInBuf.get(j)); + srcPos += lengthsInBuf.get(j); + } + } + } + + /** + * This class represents result from a striped read request. + * If the task was successful or the internal computation failed, + * an index is also returned. + */ + public static class StripingChunkReadResult { + public static final int SUCCESSFUL = 0x01; + public static final int FAILED = 0x02; + public static final int TIMEOUT = 0x04; + public static final int CANCELLED = 0x08; + + public final int index; + public final int state; + + public StripingChunkReadResult(int state) { + Preconditions.checkArgument(state == TIMEOUT, + "Only timeout result should return negative index."); + this.index = -1; + this.state = state; + } + + public StripingChunkReadResult(int index, int state) { + Preconditions.checkArgument(state != TIMEOUT, + "Timeout result should return negative index."); + this.index = index; + this.state = state; + } + + @Override + public String toString() { + return "(index=" + index + ", state =" + state + ")"; + } + } + + /** + * Check if the information such as IDs and generation stamps in block-i + * match the block group. + */ + public static void checkBlocks(ExtendedBlock blockGroup, + int i, ExtendedBlock blocki) throws IOException { + if (!blocki.getBlockPoolId().equals(blockGroup.getBlockPoolId())) { + throw new IOException("Block pool IDs mismatched: block" + i + "=" + + blocki + ", expected block group=" + blockGroup); + } + if (blocki.getBlockId() - i != blockGroup.getBlockId()) { + throw new IOException("Block IDs mismatched: block" + i + "=" + + blocki + ", expected block group=" + blockGroup); + } + if (blocki.getGenerationStamp() != blockGroup.getGenerationStamp()) { + throw new IOException("Generation stamps mismatched: block" + i + "=" + + blocki + ", expected block group=" + blockGroup); + } + } + + public static int getBlockIndex(Block reportedBlock) { + long BLOCK_GROUP_INDEX_MASK = 15; + return (int) (reportedBlock.getBlockId() & + BLOCK_GROUP_INDEX_MASK); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java index 3f85814219862..d938997ab3dfc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/java/org/apache/hadoop/hdfs/web/JsonUtilClient.java @@ -132,7 +132,7 @@ static HdfsFileStatus toFileStatus(final Map json, boolean includesType) { blockSize, mTime, aTime, permission, owner, group, symlink, DFSUtilClient.string2Bytes(localName), fileId, childrenNum, null, - storagePolicy); + storagePolicy, null); } /** Convert a Json map to an ExtendedBlock object. */ @@ -504,7 +504,7 @@ static LocatedBlocks toLocatedBlocks( (Map) m.get("lastLocatedBlock")); final boolean isLastBlockComplete = (Boolean)m.get("isLastBlockComplete"); return new LocatedBlocks(fileLength, isUnderConstruction, locatedBlocks, - lastLocatedBlock, isLastBlockComplete, null); + lastLocatedBlock, isLastBlockComplete, null, null); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto index 7d3256887db4b..6a140ebe7ef59 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/ClientNamenodeProtocol.proto @@ -34,6 +34,7 @@ import "acl.proto"; import "xattr.proto"; import "encryption.proto"; import "inotify.proto"; +import "erasurecoding.proto"; /** * The ClientNamenodeProtocol Service defines the interface between a client @@ -866,8 +867,14 @@ service ClientNamenodeProtocol { returns(ListEncryptionZonesResponseProto); rpc getEZForPath(GetEZForPathRequestProto) returns(GetEZForPathResponseProto); + rpc setErasureCodingPolicy(SetErasureCodingPolicyRequestProto) + returns(SetErasureCodingPolicyResponseProto); rpc getCurrentEditLogTxid(GetCurrentEditLogTxidRequestProto) returns(GetCurrentEditLogTxidResponseProto); rpc getEditsFromTxid(GetEditsFromTxidRequestProto) returns(GetEditsFromTxidResponseProto); + rpc getErasureCodingPolicies(GetErasureCodingPoliciesRequestProto) + returns(GetErasureCodingPoliciesResponseProto); + rpc getErasureCodingPolicy(GetErasureCodingPolicyRequestProto) + returns(GetErasureCodingPolicyResponseProto); } diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto new file mode 100644 index 0000000000000..fa24aefa11dae --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/erasurecoding.proto @@ -0,0 +1,60 @@ +/** + * 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. + */ + +option java_package = "org.apache.hadoop.hdfs.protocol.proto"; +option java_outer_classname = "ErasureCodingProtos"; +option java_generate_equals_and_hash = true; +package hadoop.hdfs; + +import "hdfs.proto"; + +message SetErasureCodingPolicyRequestProto { + required string src = 1; + optional ErasureCodingPolicyProto ecPolicy = 2; +} + +message SetErasureCodingPolicyResponseProto { +} + +message GetErasureCodingPoliciesRequestProto { // void request +} + +message GetErasureCodingPoliciesResponseProto { + repeated ErasureCodingPolicyProto ecPolicies = 1; +} + +message GetErasureCodingPolicyRequestProto { + required string src = 1; // path to get the policy info +} + +message GetErasureCodingPolicyResponseProto { + optional ErasureCodingPolicyProto ecPolicy = 1; +} + +/** + * Block erasure coding recovery info + */ +message BlockECRecoveryInfoProto { + required ExtendedBlockProto block = 1; + required DatanodeInfosProto sourceDnInfos = 2; + required DatanodeInfosProto targetDnInfos = 3; + required StorageUuidsProto targetStorageUuids = 4; + required StorageTypesProto targetStorageTypes = 5; + repeated uint32 liveBlockIndices = 6; + required ErasureCodingPolicyProto ecPolicy = 7; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto index 0e2d541869980..d35fb57feb785 100644 --- a/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto +++ b/hadoop-hdfs-project/hadoop-hdfs-client/src/main/proto/hdfs.proto @@ -215,6 +215,10 @@ message LocatedBlockProto { repeated bool isCached = 6 [packed=true]; // if a location in locs is cached repeated StorageTypeProto storageTypes = 7; repeated string storageIDs = 8; + + // striped block related fields + repeated uint32 blockIndex = 9; // used for striped block to indicate block index for each storage + repeated hadoop.common.TokenProto blockTokens = 10; // each internal block has a block token } message DataEncryptionKeyProto { @@ -295,6 +299,33 @@ message LocatedBlocksProto { optional LocatedBlockProto lastBlock = 4; required bool isLastBlockComplete = 5; optional FileEncryptionInfoProto fileEncryptionInfo = 6; + + // Optional field for erasure coding + optional ErasureCodingPolicyProto ecPolicy = 7; +} + +/** + * ECSchema options entry + */ +message ECSchemaOptionEntryProto { + required string key = 1; + required string value = 2; +} + +/** + * ECSchema for erasurecoding + */ +message ECSchemaProto { + required string codecName = 1; + required uint32 dataUnits = 2; + required uint32 parityUnits = 3; + repeated ECSchemaOptionEntryProto options = 4; +} + +message ErasureCodingPolicyProto { + required string name = 1; + required ECSchemaProto schema = 2; + required uint32 cellSize = 3; } /** @@ -331,7 +362,10 @@ message HdfsFileStatusProto { optional FileEncryptionInfoProto fileEncryptionInfo = 15; optional uint32 storagePolicy = 16 [default = 0]; // block storage policy id -} + + // Optional field for erasure coding + optional ErasureCodingPolicyProto ecPolicy = 17; +} /** * Checksum algorithms/types used in HDFS @@ -444,3 +478,11 @@ message RollingUpgradeStatusProto { required string blockPoolId = 1; optional bool finalized = 2 [default = false]; } + + +/** + * A list of storage IDs. + */ +message StorageUuidsProto { + repeated string storageUuids = 1; +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt new file mode 100755 index 0000000000000..6a01d616f9503 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/CHANGES-HDFS-EC-7285.txt @@ -0,0 +1,455 @@ + BREAKDOWN OF HDFS-7285 SUBTASKS AND RELATED JIRAS + + HDFS-7347. Configurable erasure coding policy for individual files and + directories ( Zhe Zhang via vinayakumarb ) + + 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) + + HDFS-7617. Add unit tests for editlog transactions for EC + (Hui Zheng via Zhe Zhang) + + HDFS-7839. Erasure coding: implement facilities in NameNode to create and + manage EC zones (Zhe Zhang) + + HDFS-7969. Erasure coding: NameNode support for lease recovery of striped + block groups. (Zhe Zhang) + + HDFS-7782. Erasure coding: pread from files in striped layout. + (Zhe Zhang and Jing Zhao via Zhe Zhang) + + 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) + + HDFS-8077. Erasure coding: fix bugs in EC zone and symlinks. + (Jing Zhao and Zhe Zhang via Jing Zhao) + + 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) + + HDFS-8090. Erasure Coding: Add RPC to client-namenode to list all + ECSchemas loaded in Namenode. (vinayakumarb) + + HDFS-8122. Erasure Coding: Support specifying ECSchema during creation of ECZone. + (Vinayakumar B via Zhe Zhang) + + HDFS-8114. Erasure coding: Add auditlog FSNamesystem#createErasureCodingZone if this + operation fails. (Rakesh R via Zhe Zhang) + + HDFS-8123. Erasure Coding: Better to move EC related proto messages to a + separate erasurecoding proto file (Rakesh R via vinayakumarb) + + HDFS-7349. Support DFS command for the EC encoding (vinayakumarb) + + HDFS-8120. Erasure coding: created util class to analyze striped block groups. + (Contributed by Zhe Zhang and Li Bo via Jing Zhao) + + HDFS-7994. Detect if resevered EC Block ID is already used during namenode + startup. (Hui Zheng via szetszwo) + + HDFS-8167. BlockManager.addBlockCollectionWithCheck should check if the block is a striped block. (Hui Zheng via zhz). + + HDFS-8166. DFSStripedOutputStream should not create empty blocks. (Jing Zhao) + + HDFS-7937. Erasure Coding: INodeFile quota computation unit tests. + (Kai Sasaki via Jing Zhao) + + HDFS-8145. Fix the editlog corruption exposed by failed TestAddStripedBlocks. + (Jing Zhao) + + HDFS-8146. Protobuf changes for BlockECRecoveryCommand and its fields for + making it ready for transfer to DN (Uma Maheswara Rao G via vinayakumarb) + + HDFS-8181. createErasureCodingZone sets retryCache state as false always + (Uma Maheswara Rao G via vinayakumarb) + + HDFS-8190. StripedBlockUtil.getInternalBlockLength may have overflow error. + (szetszwo) + + HDFS-8216. TestDFSStripedOutputStream should use BlockReaderTestUtil to + create BlockReader. (szetszwo via Zhe Zhang) + + HDFS-8212. DistributedFileSystem.createErasureCodingZone should pass schema + in FileSystemLinkResolver. (szetszwo via Zhe Zhang) + + HDFS-8024. Erasure Coding: ECworker frame, basics, bootstraping and configuration. + (umamahesh) + + HDFS-8156. Add/implement necessary APIs even we just have the system default + schema. (Kai Zheng via Zhe Zhang) + + HDFS-8136. Client gets and uses EC schema when reads and writes a stripping + file. (Kai Sasaki via Kai Zheng) + + HDFS-8233. Fix DFSStripedOutputStream#getCurrentBlockGroupBytes when the last + stripe is at the block group boundary. (jing9) + + HDFS-8223. Should calculate checksum for parity blocks in DFSStripedOutputStream. + (Yi Liu via jing9) + + HDFS-8228. Erasure Coding: SequentialBlockGroupIdGenerator#nextValue may cause + block id conflicts (Jing Zhao via Zhe Zhang) + + HDFS-8033. Erasure coding: stateful (non-positional) read from files in + striped layout (Zhe Zhang) + + HDFS-8230. Erasure Coding: Ignore DatanodeProtocol#DNA_ERASURE_CODING_RECOVERY + commands from standbynode if any (vinayakumarb) + + HDFS-8189. ClientProtocol#createErasureCodingZone API was wrongly annotated + as Idempotent (vinayakumarb) + + HDFS-8235. Erasure Coding: Create DFSStripedInputStream in DFSClient#open. + (Kai Sasaki via jing9) + + HDFS-8272. Erasure Coding: simplify the retry logic in DFSStripedInputStream + (stateful read). (Jing Zhao via Zhe Zhang) + + HDFS-8282. Erasure coding: move striped reading logic to StripedBlockUtil. + (Zhe Zhang) + + HDFS-8183. Erasure Coding: Improve DFSStripedOutputStream closing of + datastreamer threads. (Rakesh R via Zhe Zhang) + + HDFS-8308. Erasure Coding: NameNode may get blocked in waitForLoadingFSImage() + when loading editlog. (jing9) + + HDFS-7949. WebImageViewer need support file size calculation with striped + blocks. (Rakesh R via Zhe Zhang) + + HDFS-8316. Erasure coding: refactor EC constants to be consistent with HDFS-8249. + (Zhe Zhang via jing9) + + HDFS-8281. Erasure Coding: implement parallel stateful reading for striped layout. + (jing9) + + HDFS-8137. Send the EC schema to DataNode via EC encoding/recovering command(umamahesh) + + HDFS-8242. Erasure Coding: XML based end-to-end test for ECCli commands + (Rakesh R via vinayakumarb) + + HDFS-8324. Add trace info to DFSClient#getErasureCodingZoneInfo(..) (vinayakumarb via + umamahesh) + + HDFS-7672. Handle write failure for stripping blocks and refactor the + existing code in DFSStripedOutputStream and StripedDataStreamer. (szetszwo) + + HDFS-7348. Erasure Coding: DataNode reconstruct striped blocks. + (Yi Liu via Zhe Zhang) + + HADOOP-11921. Enhance tests for erasure coders. (Kai Zheng) + + HDFS-8334. Erasure coding: rename DFSStripedInputStream related test + classes. (Zhe Zhang) + + HDFS-8129. Erasure Coding: Maintain consistent naming for Erasure Coding related classes - EC/ErasureCoding + (umamahesh) + + HDFS-8203. Erasure Coding: Seek and other Ops in DFSStripedInputStream. + (Yi Liu via jing9) + + HDFS-8289. Erasure Coding: add ECSchema to HdfsFileStatus. (Yong Zhang via + jing9) + + HDFS-8355. Erasure Coding: Refactor BlockInfo and BlockInfoUnderConstruction. + (Tsz Wo Nicholas Sze via jing9) + + HDFS-7678. Erasure coding: DFSInputStream with decode functionality (pread). + (Zhe Zhang) + + HDFS-8372. Erasure coding: compute storage type quotas for striped files, + to be consistent with HDFS-8327. (Zhe Zhang via jing9) + + HDFS-8368. Erasure Coding: DFS opening a non-existent file need to be + handled properly (Rakesh R via zhz) + + HDFS-8363. Erasure Coding: DFSStripedInputStream#seekToNewSource. (yliu) + + HDFS-8195. Erasure coding: Fix file quota change when we complete/commit + the striped blocks. (Takuya Fukudome via zhz) + + HDFS-8364. Erasure coding: fix some minor bugs in EC CLI + (Walter Su via vinayakumarb) + + HDFS-8391. NN should consider current EC tasks handling count from DN while + assigning new tasks. (umamahesh) + + HDFS-8367. BlockInfoStriped uses EC schema. (Kai Sasaki via Kai Zheng) + + HDFS-8352. Erasure Coding: test webhdfs read write stripe file. (waltersu4549) + + HDFS-8417. Erasure Coding: Pread failed to read data starting from not-first stripe. + (Walter Su via jing9) + + HDFS-8418. Fix the isNeededReplication calculation for Striped block in NN. + (Yi Liu via jing9) + + HDFS-8320. Erasure coding: consolidate striping-related terminologies. (zhz) + + HDFS-8366. Erasure Coding: Make the timeout parameter of polling blocking queue + configurable in DFSStripedOutputStream. (Li Bo) + + HDFS-8378. Erasure Coding: Few improvements for the erasure coding worker. + (Rakesh R via waltersu4549) + + HDFS-8375. Add cellSize as an XAttr to ECZone. ( Vinayakumar B via zhz). + + HDFS-8428. Erasure Coding: Fix the NullPointerException when deleting file. + (Yi Liu via zhz). + + HDFS-8323. Bump GenerationStamp for write faliure in DFSStripedOutputStream. + (Tsz Wo Nicholas Sze via jing9) + + HDFS-8427. Remove dataBlockNum and parityBlockNum from BlockInfoStriped. + (Kai Sasaki via jing9) + + HDFS-8186. Erasure coding: Make block placement policy for EC file configurable. + (Walter Su via zhz) + + HDFS-8294. Erasure Coding: Fix Findbug warnings present in erasure coding. + (Rakesh R via zhz) + + HDFS-8441. Erasure Coding: make condition check earlier for setReplication. + (waltersu4549) + + HDFS-7768. Change fsck to support EC files. (Takanobu Asanuma via szetszwo) + + HDFS-8382. Remove chunkSize and initialize from erasure coder. (Kai Zheng) + + HDFS-8408. Revisit and refactor ErasureCodingInfo (vinayakumarb) + + HDFS-8479. Erasure coding: fix striping related logic in FSDirWriteFileOp to + sync with HDFS-8421. (Zhe Zhang via jing9) + + HDFS-8481. Erasure coding: remove workarounds in client side stripped blocks + recovering. (zhz) + + HDFS-8336. Expose some administrative erasure coding operations to HdfsAdmin + (Uma Maheswara Rao G via vinayakumarb) + + HDFS-8444. Erasure Coding: fix cannot rename a zone dir + (Walter Su via vinayakumarb) + + HDFS-8517. Fix a decoding issue in stripped block recovering in client side. + (Kai Zheng via jing9) + + HDFS-8453. Erasure coding: properly handle start offset for internal blocks + in a block group. (Zhe Zhang via jing9) + + HDFS-7621. Erasure Coding: update the Balancer/Mover data migration logic. + (Walter Su via zhz) + + HDFS-8328. Follow-on to update decode for DataNode striped blocks + reconstruction. (yliu) + + HDFS-8319. Erasure Coding: support decoding for stateful read. + (Jing Zhao via zhz) + + HDFS-8460. Erasure Coding: stateful read result doesn't match data + occasionally because of flawed test. (Walter Su via zhz) + + HDFS-8556. Erasure Coding: Fix usage of 'createZone' (vinayakumarb) + + HDFS-8571. Fix TestErasureCodingCli test (Vinayakumar B via waltersu4549) + + HDFS-8450. Erasure Coding: Consolidate erasure coding zone related + implementation into a single class (Rakesh R via vinayakumarb) + + HDFS-8585. Erasure Coding: Remove dataBlockNum and parityBlockNum from + StripedBlockProto. (Yi Liu via jing9) + + HDFS-8559. Erasure Coding: fix non-protobuf fsimage for striped blocks. + (Jing Zhao via yliu) + + HDFS-8580. Erasure coding: Persist cellSize in BlockInfoStriped and + StripedBlocksFeature. (Walter Su via jing9) + + HDFS-8466. Refactor BlockInfoContiguous and fix NPE in + TestBlockInfo#testCopyConstructor() (vinayakumarb) + + HDFS-8254. Avoid assigning a leading streamer in StripedDataStreamer to + tolerate datanode failure. (Tsz Wo Nicholas Sze via jing9) + + HDFS-8543. Erasure Coding: processOverReplicatedBlock() handles striped block. + (Walter Su via jing9) + + HDFS-8602. Erasure Coding: Client can't read(decode) the EC files which have + corrupt blocks. (jing9 and Kai Sasaki) + + HDFS-8567. Erasure Coding: SafeMode handles file smaller than a full stripe. + (Walter Su via jing9) + + HDFS-8253. DFSStripedOutputStream.closeThreads releases cellBuffers + multiple times. (Kai Sasaki via szetszwo) + + HDFS-8468. 2 RPC calls for every file read in DFSClient#open(..) resulting in + double Audit log entries (vinayakumarb) + + HDFS-8684. Erasure Coding: fix some block number calculation for striped + block. (yliu) + + HDFS-8461. Erasure coding: fix priority level of UnderReplicatedBlocks for + striped block. (Walter Su via jing9) + + HDFS-8719. Erasure Coding: client generates too many small packets when + writing parity data. (Li Bo via waltersu4549) + + HDFS-8563. Erasure Coding: fsck handles file smaller than a full stripe. + (Walter Su via jing9) + + HDFS-8484. Erasure coding: Two contiguous blocks occupy IDs belong to same + striped group. (Walter Su via jing9) + + HDFS-8744. Erasure Coding: the number of chunks in packet is not updated + when writing parity data. (Li Bo) + + HDFS-8669. Erasure Coding: handle missing internal block locations in + DFSStripedInputStream. (jing9) + + HDFS-8702. Erasure coding: update BlockManager.blockHasEnoughRacks(..) logic + for striped block. (Kai Sasaki via jing9) + + HDFS-8734. Erasure Coding: fix one cell need two packets. (Walter Su via + jing9) + + HDFS-8619. Erasure Coding: revisit replica counting for striped blocks. + (Jing Zhao via yliu) + + HDFS-8058. Erasure coding: use BlockInfo[] for both striped and contiguous + blocks in INodeFile. (Zhe Zhang and Yi Liu via zhz) + + HDFS-8787. Erasure coding: rename BlockInfoContiguousUC and BlockInfoStripedUC + to be consistent with trunk. (zhz) + + HDFS-8433. Erasure coding: set blockToken in LocatedStripedBlock.(waltersu4549) + + HDFS-8760. Erasure Coding: reuse BlockReader when reading the same block in pread. + (jing9) + + HDFS-8781. Erasure Coding: Correctly handle BlockManager#InvalidateBlocks for + striped block. (Yi Liu via jing9) + + HDFS-8813. Erasure Coding: Client no need to decode missing parity blocks. + (Walter Su via jing9) + + HDFS-8798. Erasure Coding: fix DFSStripedInputStream/DFSStripedOutputStream + re-fetch token when expired. (Walter Su via jing9) + + HDFS-8769. Erasure coding: unit test for SequentialBlockGroupIdGenerator. + (Rakesh R via waltersu4549) + + HDFS-8202. Improve end to end stirpping file test to add erasure recovering + test. (Xinwei Qin via zhz) + + HDFS-8804. Erasure Coding: use DirectBufferPool in DFSStripedInputStream for + buffer allocation. (jing9) + + HDFS-8399. Erasure Coding: unit test the behaviour of BlockManager recovery + work for the deleted blocks. (Rakesh R via zhz) + + HDFS-8857. Erasure Coding: Fix ArrayIndexOutOfBoundsException in + TestWriteStripedFileWithFailure. (Li Bo) + + HDFS-8827. Erasure Coding: Fix NPE when NameNode processes over-replicated + striped blocks. (Walter Su and Takuya Fukudome via jing9) + + HDFS-8854. Erasure coding: add ECPolicy to replace schema+cellSize in + hadoop-hdfs. (Walter Su via zhz) + + HDFS-8220. Erasure Coding: StripedDataStreamer fails to handle the + blocklocations which doesn't satisfy BlockGroupSize. (Rakesh R via zhz) + + HDFS-8838. Erasure Coding: Tolerate datanode failures in DFSStripedOutputStream + when the data length is small. (szetszwo via waltersu4549) + + HDFS-8909. Erasure coding: update BlockInfoContiguousUC and BlockInfoStripedUC + to use BlockUnderConstructionFeature. (Jing Zhao via waltersu4549) + + HDFS-8937. Erasure coding: do not throw exception when setting replication on + EC file. (Gao Rui via jing9) + + HDFS-8978. Erasure coding: fix 2 failed tests of DFSStripedOutputStream. + (Walter Su via jing9) + + HDFS-8833. Erasure coding: store EC schema and cell size in INodeFile and + eliminate notion of EC zones. (zhz) + + HDFS-8853. Erasure Coding: Provide ECSchema validation when setting EC + policy. (andreina via zhz) + + HDFS-8975. Erasure coding : Fix random failure in TestSafeModeWithStripedFile + (J.Andreina via vinayakumarb) + + HDFS-7351. Document the HDFS Erasure Coding feature. + (umamahesh and Zhe Zhang via wang) + + HDFS-8899. Erasure Coding: use threadpool for EC recovery tasks on DataNode. + (Rakesh R via zhz) + + HDFS-9088. Cleanup erasure coding documentation. (wang via zhz) + + HDFS-9097. Erasure coding: update EC command "-s" flag to "-p" when + specifying policy. (zhz) + + HDFS-9086. Rename dfs.datanode.stripedread.threshold.millis to + dfs.datanode.stripedread.timeout.millis. (wang via zhz) + + HDFS-8550. Erasure Coding: Fix FindBugs Multithreaded correctness Warning. + (Rakesh R via zhz) + + HDFS-9113. ErasureCodingWorker#processErasureCodingTasks should not fail to process + remaining tasks due to one invalid ECTask (umamahesh) + + HDFS-9091. Erasure Coding: Provide DistributedFilesystem API to + getAllErasureCodingPolicies. (Rakesh R via zhz) + + HDFS-8920. Erasure Coding: when recovering lost blocks, logs can be too + verbose and hurt performance. (Rui Li via Kai Zheng) + + HDFS-8882. Erasure Coding: Use datablocks, parityblocks and cell size from + ErasureCodingPolicy (Vinayakumar B via zhz) + + HDFS-9040. Erasure coding: coordinate data streamers in + DFSStripedOutputStream. (jing9 and Walter Su) diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs index 852b040622986..8b1ede82887db 100755 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/bin/hdfs @@ -34,6 +34,7 @@ function hadoop_usage hadoop_add_subcommand "debug" "run a Debug Admin to execute HDFS debug commands" hadoop_add_subcommand "dfs" "run a filesystem command on the file system" hadoop_add_subcommand "dfsadmin" "run a DFS admin client" + hadoop_add_subcommand "erasurecode" "run a HDFS ErasureCoding CLI" hadoop_add_subcommand "fetchdt" "fetch a delegation token from the NameNode" hadoop_add_subcommand "fsck" "run a DFS filesystem checking utility" hadoop_add_subcommand "getconf" "get config values from configuration" @@ -133,6 +134,11 @@ case ${COMMAND} in hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" ;; + erasurecode) + CLASS=org.apache.hadoop.hdfs.tools.erasurecode.ECCli + hadoop_debug "Appending HADOOP_CLIENT_OPTS onto HADOOP_OPTS" + HADOOP_OPTS="${HADOOP_OPTS} ${HADOOP_CLIENT_OPTS}" + ;; fetchdt) CLASS=org.apache.hadoop.hdfs.tools.DelegationTokenFetcher ;; 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 b63195537bbb7..0166029ad4683 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 @@ -23,6 +23,7 @@ import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyRackFaultTolerant; import org.apache.hadoop.hdfs.server.datanode.fsdataset.impl.RamDiskReplicaLruTracker; import org.apache.hadoop.http.HttpConfig; @@ -169,6 +170,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final int DFS_NAMENODE_REPLICATION_INTERVAL_DEFAULT = 3; public static final String DFS_NAMENODE_REPLICATION_MIN_KEY = "dfs.namenode.replication.min"; public static final int DFS_NAMENODE_REPLICATION_MIN_DEFAULT = 1; + public static final String DFS_NAMENODE_STRIPE_MIN_KEY = "dfs.namenode.stripe.min"; + public static final int DFS_NAMENODE_STRIPE_MIN_DEFAULT = 1; public static final String DFS_NAMENODE_SAFEMODE_REPLICATION_MIN_KEY = "dfs.namenode.safemode.replication.min"; public static final String DFS_NAMENODE_REPLICATION_PENDING_TIMEOUT_SEC_KEY = "dfs.namenode.replication.pending.timeout-sec"; @@ -397,6 +400,14 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final int DFS_DATANODE_DIRECTORYSCAN_INTERVAL_DEFAULT = 21600; public static final String DFS_DATANODE_DIRECTORYSCAN_THREADS_KEY = "dfs.datanode.directoryscan.threads"; public static final int DFS_DATANODE_DIRECTORYSCAN_THREADS_DEFAULT = 1; + public static final String DFS_DATANODE_STRIPED_READ_THREADS_KEY = "dfs.datanode.stripedread.threads"; + public static final int DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT = 20; + public static final String DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY = "dfs.datanode.stripedread.buffer.size"; + public static final int DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT = 64 * 1024; + public static final String DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY = "dfs.datanode.stripedread.timeout.millis"; + public static final int DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT = 5000; //5s + public static final String DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY = "dfs.datanode.striped.blockrecovery.threads.size"; + public static final int DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT = 8; public static final String DFS_DATANODE_DIRECTORYSCAN_THROTTLE_LIMIT_MS_PER_SEC_KEY = "dfs.datanode.directoryscan.throttle.limit.ms.per.sec"; @@ -461,6 +472,8 @@ public class DFSConfigKeys extends CommonConfigurationKeys { public static final Class DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT = BlockPlacementPolicyDefault.class; public static final String DFS_REPLICATION_MAX_KEY = "dfs.replication.max"; public static final int DFS_REPLICATION_MAX_DEFAULT = 512; + public static final String DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY = "dfs.block.placement.ec.classname"; + public static final Class DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT = BlockPlacementPolicyRackFaultTolerant.class; public static final String DFS_DF_INTERVAL_KEY = "dfs.df.interval"; public static final int DFS_DF_INTERVAL_DEFAULT = 60000; @@ -735,7 +748,7 @@ public class DFSConfigKeys extends CommonConfigurationKeys { "dfs.namenode.reject-unresolved-dn-topology-mapping"; public static final boolean DFS_REJECT_UNRESOLVED_DN_TOPOLOGY_MAPPING_DEFAULT = false; - + // 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"; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java index 5b11ac277f61b..b0ea7ce90d6e5 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/DFSUtil.java @@ -36,6 +36,7 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_SERVER_HTTPS_TRUSTSTORE_PASSWORD_KEY; import java.io.IOException; +import java.io.InterruptedIOException; import java.io.PrintStream; import java.io.UnsupportedEncodingException; import java.net.InetAddress; @@ -88,6 +89,7 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import com.google.protobuf.BlockingService; @InterfaceAudience.Private @@ -1439,5 +1441,4 @@ public static KeyProviderCryptoExtension createKeyProviderCryptoExtension( .createKeyProviderCryptoExtension(keyProvider); return cryptoProvider; } - } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java index 84499bb3d4d98..7b055fc8f6458 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/client/HdfsAdmin.java @@ -40,6 +40,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.security.AccessControlException; import org.apache.hadoop.hdfs.tools.DFSAdmin; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; /** * The public API for performing administrative functions on HDFS. Those writing @@ -363,4 +364,39 @@ public void setStoragePolicy(final Path src, final String policyName) throws IOException { dfs.setStoragePolicy(src, policyName); } + + /** + * Set the source path to the specified erasure coding policy. + * + * @param path The source path referring to a directory. + * @param ecPolicy The erasure coding policy for the directory. + * If null, the default will be used. + * @throws IOException + */ + public void setErasureCodingPolicy(final Path path, + final ErasureCodingPolicy ecPolicy) throws IOException { + dfs.setErasureCodingPolicy(path, ecPolicy); + } + + /** + * Get the erasure coding policy information for the specified path + * + * @param path + * @return Returns the policy information if file or directory on the path is + * erasure coded. Null otherwise. + * @throws IOException + */ + public ErasureCodingPolicy getErasureCodingPolicy(final Path path) + throws IOException { + return dfs.getErasureCodingPolicy(path); + } + + /** + * Get the Erasure coding policies supported. + * + * @throws IOException + */ + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { + return dfs.getClient().getErasureCodingPolicies(); + } } 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 d93277cd77a34..1e4b8995deeff 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 @@ -200,6 +200,12 @@ import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.GetEZForPathRequestProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesResponseProto; import org.apache.hadoop.hdfs.protocol.proto.EncryptionZonesProtos.ListEncryptionZonesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPoliciesResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.GetErasureCodingPolicyResponseProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyRequestProto; +import org.apache.hadoop.hdfs.protocol.proto.ErasureCodingProtos.SetErasureCodingPolicyResponseProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockStoragePolicyProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeIDProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.DatanodeInfoProto; @@ -216,6 +222,7 @@ import org.apache.hadoop.hdfs.security.token.delegation.DelegationTokenIdentifier; import org.apache.hadoop.io.EnumSetWritable; import org.apache.hadoop.io.Text; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenRequestProto; import org.apache.hadoop.security.proto.SecurityProtos.CancelDelegationTokenResponseProto; import org.apache.hadoop.security.proto.SecurityProtos.GetDelegationTokenRequestProto; @@ -422,7 +429,8 @@ public AppendResponseProto append(RpcController controller, req.getClientName(), flags); AppendResponseProto.Builder builder = AppendResponseProto.newBuilder(); if (result.getLastBlock() != null) { - builder.setBlock(PBHelperClient.convert(result.getLastBlock())); + builder.setBlock(PBHelperClient.convertLocatedBlock( + result.getLastBlock())); } if (result.getFileStatus() != null) { builder.setStat(PBHelperClient.convert(result.getFileStatus())); @@ -498,7 +506,7 @@ public AddBlockResponseProto addBlock(RpcController controller, (favor == null || favor.size() == 0) ? null : favor .toArray(new String[favor.size()])); return AddBlockResponseProto.newBuilder() - .setBlock(PBHelperClient.convert(result)).build(); + .setBlock(PBHelperClient.convertLocatedBlock(result)).build(); } catch (IOException e) { throw new ServiceException(e); } @@ -522,7 +530,7 @@ public GetAdditionalDatanodeResponseProto getAdditionalDatanode( new DatanodeInfoProto[excludesList.size()])), req.getNumAdditionalNodes(), req.getClientName()); return GetAdditionalDatanodeResponseProto.newBuilder().setBlock( - PBHelperClient.convert(result)) + PBHelperClient.convertLocatedBlock(result)) .build(); } catch (IOException e) { throw new ServiceException(e); @@ -548,7 +556,7 @@ public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller, ReportBadBlocksRequestProto req) throws ServiceException { try { List bl = req.getBlocksList(); - server.reportBadBlocks(PBHelperClient.convertLocatedBlock( + server.reportBadBlocks(PBHelperClient.convertLocatedBlocks( bl.toArray(new LocatedBlockProto[bl.size()]))); } catch (IOException e) { throw new ServiceException(e); @@ -953,8 +961,8 @@ public UpdateBlockForPipelineResponseProto updateBlockForPipeline( RpcController controller, UpdateBlockForPipelineRequestProto req) throws ServiceException { try { - LocatedBlockProto result = PBHelperClient.convert(server - .updateBlockForPipeline(PBHelperClient.convert(req.getBlock()), + LocatedBlockProto result = PBHelperClient.convertLocatedBlock( + server.updateBlockForPipeline(PBHelperClient.convert(req.getBlock()), req.getClientName())); return UpdateBlockForPipelineResponseProto.newBuilder().setBlock(result) .build(); @@ -1393,6 +1401,20 @@ public ListEncryptionZonesResponseProto listEncryptionZones( } } + @Override + public SetErasureCodingPolicyResponseProto setErasureCodingPolicy( + RpcController controller, SetErasureCodingPolicyRequestProto req) + throws ServiceException { + try { + ErasureCodingPolicy ecPolicy = req.hasEcPolicy() ? + PBHelperClient.convertErasureCodingPolicy(req.getEcPolicy()) : null; + server.setErasureCodingPolicy(req.getSrc(), ecPolicy); + return SetErasureCodingPolicyResponseProto.newBuilder().build(); + } catch (IOException e) { + throw new ServiceException(e); + } + } + @Override public SetXAttrResponseProto setXAttr(RpcController controller, SetXAttrRequestProto req) throws ServiceException { @@ -1514,4 +1536,35 @@ public GetEditsFromTxidResponseProto getEditsFromTxid(RpcController controller, throw new ServiceException(e); } } + + @Override + public GetErasureCodingPoliciesResponseProto getErasureCodingPolicies(RpcController controller, + GetErasureCodingPoliciesRequestProto request) throws ServiceException { + try { + ErasureCodingPolicy[] ecPolicies = server.getErasureCodingPolicies(); + GetErasureCodingPoliciesResponseProto.Builder resBuilder = GetErasureCodingPoliciesResponseProto + .newBuilder(); + for (ErasureCodingPolicy ecPolicy : ecPolicies) { + resBuilder.addEcPolicies(PBHelperClient.convertErasureCodingPolicy(ecPolicy)); + } + return resBuilder.build(); + } catch (IOException e) { + throw new ServiceException(e); + } + } + + @Override + public GetErasureCodingPolicyResponseProto getErasureCodingPolicy(RpcController controller, + GetErasureCodingPolicyRequestProto request) throws ServiceException { + try { + ErasureCodingPolicy ecPolicy = server.getErasureCodingPolicy(request.getSrc()); + GetErasureCodingPolicyResponseProto.Builder builder = GetErasureCodingPolicyResponseProto.newBuilder(); + if (ecPolicy != null) { + builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy(ecPolicy)); + } + return builder.build(); + } catch (IOException e) { + throw new ServiceException(e); + } + } } 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 705d573194974..6b20af54655fe 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 @@ -281,7 +281,7 @@ public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { ReportBadBlocksRequestProto.Builder builder = ReportBadBlocksRequestProto .newBuilder(); for (int i = 0; i < blocks.length; i++) { - builder.addBlocks(i, PBHelperClient.convert(blocks[i])); + builder.addBlocks(i, PBHelperClient.convertLocatedBlock(blocks[i])); } ReportBadBlocksRequestProto req = builder.build(); try { 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 4b9f7c4b5fe3e..93c11ba7d8557 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 @@ -259,7 +259,7 @@ public ReportBadBlocksResponseProto reportBadBlocks(RpcController controller, List lbps = request.getBlocksList(); LocatedBlock [] blocks = new LocatedBlock [lbps.size()]; for(int i=0; i datanodeUuids = b.getDatanodeUuidsList(); final List storageUuids = b.getStorageUuidsList(); final List storageTypes = b.getStorageTypesList(); - return new BlockWithLocations(PBHelperClient.convert(b.getBlock()), + BlockWithLocations blk = new BlockWithLocations(PBHelperClient. + convert(b.getBlock()), datanodeUuids.toArray(new String[datanodeUuids.size()]), storageUuids.toArray(new String[storageUuids.size()]), PBHelperClient.convertStorageTypes(storageTypes, storageUuids.size())); + if (b.hasIndices()) { + blk = new StripedBlockWithLocations(blk, b.getIndices().toByteArray(), + (short) b.getDataBlockNum(), b.getCellSize()); + } + return blk; } public static BlocksWithLocationsProto convert(BlocksWithLocations blks) { @@ -333,7 +358,7 @@ public static RecoveringBlockProto convert(RecoveringBlock b) { if (b == null) { return null; } - LocatedBlockProto lb = PBHelperClient.convert((LocatedBlock) b); + LocatedBlockProto lb = PBHelperClient.convertLocatedBlock(b); RecoveringBlockProto.Builder builder = RecoveringBlockProto.newBuilder(); builder.setBlock(lb).setNewGenStamp(b.getNewGenerationStamp()); if(b.getNewBlock() != null) @@ -413,6 +438,8 @@ public static DatanodeCommand convert(DatanodeCommandProto proto) { return REG_CMD; case BlockIdCommand: return PBHelper.convert(proto.getBlkIdCmd()); + case BlockECRecoveryCommand: + return PBHelper.convert(proto.getBlkECRecoveryCmd()); default: return null; } @@ -542,6 +569,11 @@ public static DatanodeCommandProto convert(DatanodeCommand datanodeCommand) { builder.setCmdType(DatanodeCommandProto.Type.BlockIdCommand). setBlkIdCmd(PBHelper.convert((BlockIdCommand) datanodeCommand)); break; + case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY: + builder.setCmdType(DatanodeCommandProto.Type.BlockECRecoveryCommand) + .setBlkECRecoveryCmd( + convert((BlockECRecoveryCommand) datanodeCommand)); + break; case DatanodeProtocol.DNA_UNKNOWN: //Not expected default: builder.setCmdType(DatanodeCommandProto.Type.NullDatanodeCommand); @@ -705,7 +737,6 @@ public static NamespaceInfoProto convert(NamespaceInfo info) { .build(); } - public static NNHAStatusHeartbeat convert(NNHAStatusHeartbeatProto s) { if (s == null) return null; switch (s.getState()) { @@ -790,4 +821,140 @@ public static BlockReportContextProto convert(BlockReportContext context) { setLeaseId(context.getLeaseId()). build(); } + + private static List convertIntArray(short[] liveBlockIndices) { + List liveBlockIndicesList = new ArrayList<>(); + for (short s : liveBlockIndices) { + liveBlockIndicesList.add((int) s); + } + return liveBlockIndicesList; + } + + private static StorageTypesProto convertStorageTypesProto( + StorageType[] targetStorageTypes) { + StorageTypesProto.Builder builder = StorageTypesProto.newBuilder(); + for (StorageType storageType : targetStorageTypes) { + builder.addStorageTypes(PBHelperClient.convertStorageType(storageType)); + } + return builder.build(); + } + + private static HdfsProtos.StorageUuidsProto convertStorageIDs(String[] targetStorageIDs) { + HdfsProtos.StorageUuidsProto.Builder builder = HdfsProtos.StorageUuidsProto.newBuilder(); + for (String storageUuid : targetStorageIDs) { + builder.addStorageUuids(storageUuid); + } + return builder.build(); + } + + private static DatanodeInfosProto convertToDnInfosProto(DatanodeInfo[] dnInfos) { + DatanodeInfosProto.Builder builder = DatanodeInfosProto.newBuilder(); + for (DatanodeInfo datanodeInfo : dnInfos) { + builder.addDatanodes(PBHelperClient.convert(datanodeInfo)); + } + return builder.build(); + } + + private static String[] convert(HdfsProtos.StorageUuidsProto targetStorageUuidsProto) { + List storageUuidsList = targetStorageUuidsProto + .getStorageUuidsList(); + String[] storageUuids = new String[storageUuidsList.size()]; + for (int i = 0; i < storageUuidsList.size(); i++) { + storageUuids[i] = storageUuidsList.get(i); + } + return storageUuids; + } + + public static BlockECRecoveryInfo convertBlockECRecoveryInfo( + BlockECRecoveryInfoProto blockEcRecoveryInfoProto) { + ExtendedBlockProto blockProto = blockEcRecoveryInfoProto.getBlock(); + ExtendedBlock block = PBHelperClient.convert(blockProto); + + DatanodeInfosProto sourceDnInfosProto = blockEcRecoveryInfoProto + .getSourceDnInfos(); + DatanodeInfo[] sourceDnInfos = PBHelperClient.convert(sourceDnInfosProto); + + DatanodeInfosProto targetDnInfosProto = blockEcRecoveryInfoProto + .getTargetDnInfos(); + DatanodeInfo[] targetDnInfos = PBHelperClient.convert(targetDnInfosProto); + + HdfsProtos.StorageUuidsProto targetStorageUuidsProto = blockEcRecoveryInfoProto + .getTargetStorageUuids(); + String[] targetStorageUuids = convert(targetStorageUuidsProto); + + StorageTypesProto targetStorageTypesProto = blockEcRecoveryInfoProto + .getTargetStorageTypes(); + StorageType[] convertStorageTypes = PBHelperClient.convertStorageTypes( + targetStorageTypesProto.getStorageTypesList(), targetStorageTypesProto + .getStorageTypesList().size()); + + List liveBlockIndicesList = blockEcRecoveryInfoProto + .getLiveBlockIndicesList(); + short[] liveBlkIndices = new short[liveBlockIndicesList.size()]; + for (int i = 0; i < liveBlockIndicesList.size(); i++) { + liveBlkIndices[i] = liveBlockIndicesList.get(i).shortValue(); + } + + ErasureCodingPolicy ecPolicy = + PBHelperClient.convertErasureCodingPolicy( + blockEcRecoveryInfoProto.getEcPolicy()); + + return new BlockECRecoveryInfo(block, sourceDnInfos, targetDnInfos, + targetStorageUuids, convertStorageTypes, liveBlkIndices, ecPolicy); + } + + public static BlockECRecoveryInfoProto convertBlockECRecoveryInfo( + BlockECRecoveryInfo blockEcRecoveryInfo) { + BlockECRecoveryInfoProto.Builder builder = BlockECRecoveryInfoProto + .newBuilder(); + builder.setBlock(PBHelperClient.convert( + blockEcRecoveryInfo.getExtendedBlock())); + + DatanodeInfo[] sourceDnInfos = blockEcRecoveryInfo.getSourceDnInfos(); + builder.setSourceDnInfos(convertToDnInfosProto(sourceDnInfos)); + + DatanodeInfo[] targetDnInfos = blockEcRecoveryInfo.getTargetDnInfos(); + builder.setTargetDnInfos(convertToDnInfosProto(targetDnInfos)); + + String[] targetStorageIDs = blockEcRecoveryInfo.getTargetStorageIDs(); + builder.setTargetStorageUuids(convertStorageIDs(targetStorageIDs)); + + StorageType[] targetStorageTypes = blockEcRecoveryInfo + .getTargetStorageTypes(); + builder.setTargetStorageTypes(convertStorageTypesProto(targetStorageTypes)); + + short[] liveBlockIndices = blockEcRecoveryInfo.getLiveBlockIndices(); + builder.addAllLiveBlockIndices(convertIntArray(liveBlockIndices)); + + builder.setEcPolicy(PBHelperClient.convertErasureCodingPolicy( + blockEcRecoveryInfo.getErasureCodingPolicy())); + + return builder.build(); + } + + public static BlockECRecoveryCommandProto convert( + BlockECRecoveryCommand blkECRecoveryCmd) { + BlockECRecoveryCommandProto.Builder builder = BlockECRecoveryCommandProto + .newBuilder(); + Collection blockECRecoveryInfos = blkECRecoveryCmd + .getECTasks(); + for (BlockECRecoveryInfo blkECRecoveryInfo : blockECRecoveryInfos) { + builder + .addBlockECRecoveryinfo(convertBlockECRecoveryInfo(blkECRecoveryInfo)); + } + return builder.build(); + } + + public static BlockECRecoveryCommand convert( + BlockECRecoveryCommandProto blkECRecoveryCmdProto) { + Collection blkECRecoveryInfos = new ArrayList<>(); + List blockECRecoveryinfoList = blkECRecoveryCmdProto + .getBlockECRecoveryinfoList(); + for (BlockECRecoveryInfoProto blockECRecoveryInfoProto : blockECRecoveryinfoList) { + blkECRecoveryInfos + .add(convertBlockECRecoveryInfo(blockECRecoveryInfoProto)); + } + return new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, + blkECRecoveryInfos); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java index f3f3d6fc2a522..2bd28ab83ba5e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Balancer.java @@ -49,8 +49,8 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Source; import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Task; import org.apache.hadoop.hdfs.server.balancer.Dispatcher.Util; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicyDefault; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies; import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport; @@ -211,7 +211,9 @@ public class Balancer { */ private static void checkReplicationPolicyCompatibility(Configuration conf ) throws UnsupportedActionException { - if (!(BlockPlacementPolicy.getInstance(conf, null, null, null) instanceof + BlockPlacementPolicies placementPolicies = + new BlockPlacementPolicies(conf, null, null, null); + if (!(placementPolicies.getPolicy(false) instanceof BlockPlacementPolicyDefault)) { throw new UnsupportedActionException( "Balancer without BlockPlacementPolicyDefault"); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java index ff2d7622fd874..3b36555b122e8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/balancer/Dispatcher.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.balancer; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; import static org.apache.hadoop.hdfs.protocolPB.PBHelperClient.vintPrefixed; import java.io.BufferedInputStream; @@ -65,6 +66,7 @@ import org.apache.hadoop.hdfs.server.balancer.Dispatcher.DDatanode.StorageGroup; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.io.IOUtils; import org.apache.hadoop.net.NetUtils; @@ -148,18 +150,17 @@ private static class GlobalBlockMap { private final Map map = new HashMap(); /** - * Get the block from the map; - * if the block is not found, create a new block and put it in the map. + * Put block in the map if it's not found + * @return the block which be put in the map the first time */ - private DBlock get(Block b) { - DBlock block = map.get(b); - if (block == null) { - block = new DBlock(b); - map.put(b, block); + private DBlock putIfAbsent(Block blk, DBlock dblk) { + if (!map.containsKey(blk)) { + map.put(blk, dblk); + return dblk; } - return block; + return map.get(blk); } - + /** Remove all blocks except for the moved blocks. */ private void removeAllButRetain(MovedBlocks movedBlocks) { for (Iterator i = map.keySet().iterator(); i.hasNext();) { @@ -200,9 +201,9 @@ public Collection values() { } } - /** This class keeps track of a scheduled block move */ + /** This class keeps track of a scheduled reportedBlock move */ public class PendingMove { - private DBlock block; + private DBlock reportedBlock; private Source source; private DDatanode proxySource; private StorageGroup target; @@ -214,7 +215,7 @@ private PendingMove(Source source, StorageGroup target) { @Override public String toString() { - final Block b = block != null ? block.getBlock() : null; + final Block b = reportedBlock != null ? reportedBlock.getBlock() : null; String bStr = b != null ? (b + " with size=" + b.getNumBytes() + " ") : " "; return bStr + "from " + source.getDisplayName() + " to " + target @@ -223,8 +224,8 @@ public String toString() { } /** - * Choose a block & a proxy source for this pendingMove whose source & - * target have already been chosen. + * Choose a good block/blockGroup from source & Get reportedBlock from + * the block & Choose a proxy source for the reportedBlock. * * @return true if a block and its proxy are chosen; false otherwise */ @@ -248,7 +249,11 @@ private boolean markMovedIfGoodBlock(DBlock block, StorageType targetStorageType synchronized (block) { synchronized (movedBlocks) { if (isGoodBlockCandidate(source, target, targetStorageType, block)) { - this.block = block; + if (block instanceof DBlockStriped) { + reportedBlock = ((DBlockStriped) block).getInternalBlock(source); + } else { + reportedBlock = block; + } if (chooseProxySource()) { movedBlocks.put(block); if (LOG.isDebugEnabled()) { @@ -275,7 +280,7 @@ private boolean chooseProxySource() { } // if node group is supported, first try add nodes in the same node group if (cluster.isNodeGroupAware()) { - for (StorageGroup loc : block.getLocations()) { + for (StorageGroup loc : reportedBlock.getLocations()) { if (cluster.isOnSameNodeGroup(loc.getDatanodeInfo(), targetDN) && addTo(loc)) { return true; @@ -283,13 +288,13 @@ && addTo(loc)) { } } // check if there is replica which is on the same rack with the target - for (StorageGroup loc : block.getLocations()) { + for (StorageGroup loc : reportedBlock.getLocations()) { if (cluster.isOnSameRack(loc.getDatanodeInfo(), targetDN) && addTo(loc)) { return true; } } // find out a non-busy replica - for (StorageGroup loc : block.getLocations()) { + for (StorageGroup loc : reportedBlock.getLocations()) { if (addTo(loc)) { return true; } @@ -297,7 +302,7 @@ && addTo(loc)) { return false; } - /** add to a proxy source for specific block movement */ + /** add to a proxy source for specific reportedBlock movement */ private boolean addTo(StorageGroup g) { final DDatanode dn = g.getDDatanode(); if (dn.addPendingBlock(this)) { @@ -310,6 +315,7 @@ private boolean addTo(StorageGroup g) { /** Dispatch the move to the proxy source & wait for the response. */ private void dispatch() { LOG.info("Start moving " + this); + assert !(reportedBlock instanceof DBlockStriped); Socket sock = new Socket(); DataOutputStream out = null; @@ -324,7 +330,7 @@ private void dispatch() { OutputStream unbufOut = sock.getOutputStream(); InputStream unbufIn = sock.getInputStream(); ExtendedBlock eb = new ExtendedBlock(nnc.getBlockpoolID(), - block.getBlock()); + reportedBlock.getBlock()); final KeyManager km = nnc.getKeyManager(); Token accessToken = km.getAccessToken(eb); IOStreamPair saslStreams = saslClient.socketSend(sock, unbufOut, @@ -338,7 +344,7 @@ private void dispatch() { sendRequest(out, eb, accessToken); receiveResponse(in); - nnc.getBytesMoved().addAndGet(block.getNumBytes()); + nnc.getBytesMoved().addAndGet(reportedBlock.getNumBytes()); target.getDDatanode().setHasSuccess(); LOG.info("Successfully moved " + this); } catch (IOException e) { @@ -367,14 +373,14 @@ private void dispatch() { } } - /** Send a block replace request to the output stream */ + /** Send a reportedBlock replace request to the output stream */ private void sendRequest(DataOutputStream out, ExtendedBlock eb, Token accessToken) throws IOException { new Sender(out).replaceBlock(eb, target.storageType, accessToken, source.getDatanodeInfo().getDatanodeUuid(), proxySource.datanode); } - /** Receive a block copy response from the input stream */ + /** Receive a reportedBlock copy response from the input stream */ private void receiveResponse(DataInputStream in) throws IOException { BlockOpResponseProto response = BlockOpResponseProto.parseFrom(vintPrefixed(in)); @@ -382,13 +388,13 @@ private void receiveResponse(DataInputStream in) throws IOException { // read intermediate responses response = BlockOpResponseProto.parseFrom(vintPrefixed(in)); } - String logInfo = "block move is failed"; + String logInfo = "reportedBlock move is failed"; DataTransferProtoUtil.checkBlockOpStatus(response, logInfo); } /** reset the object */ private void reset() { - block = null; + reportedBlock = null; source = null; proxySource = null; target = null; @@ -400,6 +406,47 @@ public static class DBlock extends MovedBlocks.Locations { public DBlock(Block block) { super(block); } + + public long getNumBytes(StorageGroup storage) { + return super.getNumBytes(); + } + } + + public static class DBlockStriped extends DBlock { + + final byte[] indices; + final short dataBlockNum; + final int cellSize; + + public DBlockStriped(Block block, byte[] indices, short dataBlockNum, + int cellSize) { + super(block); + this.indices = indices; + this.dataBlockNum = dataBlockNum; + this.cellSize = cellSize; + } + + public DBlock getInternalBlock(StorageGroup storage) { + int idxInLocs = locations.indexOf(storage); + if (idxInLocs == -1) { + return null; + } + byte idxInGroup = indices[idxInLocs]; + long blkId = getBlock().getBlockId() + idxInGroup; + long numBytes = getInternalBlockLength(getNumBytes(), cellSize, + dataBlockNum, idxInGroup); + Block blk = new Block(getBlock()); + blk.setBlockId(blkId); + blk.setNumBytes(numBytes); + DBlock dblk = new DBlock(blk); + dblk.addLocation(storage); + return dblk; + } + + @Override + public long getNumBytes(StorageGroup storage) { + return getInternalBlock(storage).getNumBytes(); + } } /** The class represents a desired move. */ @@ -475,7 +522,7 @@ synchronized void resetScheduledSize() { private PendingMove addPendingMove(DBlock block, final PendingMove pm) { if (getDDatanode().addPendingBlock(pm)) { if (pm.markMovedIfGoodBlock(block, getStorageType())) { - incScheduledSize(pm.block.getNumBytes()); + incScheduledSize(pm.reportedBlock.getNumBytes()); return pm; } else { getDDatanode().removePendingBlock(pm); @@ -650,30 +697,44 @@ Iterator getBlockIterator() { */ private long getBlockList() throws IOException { final long size = Math.min(getBlocksSize, blocksToReceive); - final BlocksWithLocations newBlocks = nnc.getBlocks(getDatanodeInfo(), size); + final BlocksWithLocations newBlksLocs = + nnc.getBlocks(getDatanodeInfo(), size); if (LOG.isTraceEnabled()) { LOG.trace("getBlocks(" + getDatanodeInfo() + ", " + StringUtils.TraditionalBinaryPrefix.long2String(size, "B", 2) - + ") returns " + newBlocks.getBlocks().length + " blocks."); + + ") returns " + newBlksLocs.getBlocks().length + " blocks."); } long bytesReceived = 0; - for (BlockWithLocations blk : newBlocks.getBlocks()) { + for (BlockWithLocations blkLocs : newBlksLocs.getBlocks()) { // Skip small blocks. - if (blk.getBlock().getNumBytes() < getBlocksMinBlockSize) { + if (blkLocs.getBlock().getNumBytes() < getBlocksMinBlockSize) { continue; } - bytesReceived += blk.getBlock().getNumBytes(); + DBlock block; + if (blkLocs instanceof StripedBlockWithLocations) { + StripedBlockWithLocations sblkLocs = + (StripedBlockWithLocations) blkLocs; + // approximate size + bytesReceived += sblkLocs.getBlock().getNumBytes() / + sblkLocs.getDataBlockNum(); + block = new DBlockStriped(sblkLocs.getBlock(), sblkLocs.getIndices(), + sblkLocs.getDataBlockNum(), sblkLocs.getCellSize()); + } else { + bytesReceived += blkLocs.getBlock().getNumBytes(); + block = new DBlock(blkLocs.getBlock()); + } + synchronized (globalBlocks) { - final DBlock block = globalBlocks.get(blk.getBlock()); + block = globalBlocks.putIfAbsent(blkLocs.getBlock(), block); synchronized (block) { block.clearLocations(); // update locations - final String[] datanodeUuids = blk.getDatanodeUuids(); - final StorageType[] storageTypes = blk.getStorageTypes(); + final String[] datanodeUuids = blkLocs.getDatanodeUuids(); + final StorageType[] storageTypes = blkLocs.getStorageTypes(); for (int i = 0; i < datanodeUuids.length; i++) { final StorageGroup g = storageGroupMap.get( datanodeUuids[i], storageTypes[i]); @@ -712,6 +773,8 @@ private boolean isGoodBlockCandidate(DBlock block) { * target throttling has been considered. They are chosen only when they * have the capacity to support this block move. The block should be * dispatched immediately after this method is returned. + * If the block is a block group. Only the internal block on this source + * will be dispatched. * * @return a move that's good for the source to dispatch immediately. */ @@ -723,7 +786,7 @@ private PendingMove chooseNextMove() { if (target.addPendingBlock(pendingBlock)) { // target is not busy, so do a tentative block allocation if (pendingBlock.chooseBlockAndProxy()) { - long blockSize = pendingBlock.block.getNumBytes(); + long blockSize = pendingBlock.reportedBlock.getNumBytes(this); incScheduledSize(-blockSize); task.size -= blockSize; if (task.size == 0) { @@ -802,7 +865,7 @@ private void dispatchBlocks() { blocksToReceive -= received; continue; } catch (IOException e) { - LOG.warn("Exception while getting block list", e); + LOG.warn("Exception while getting reportedBlock list", e); return; } } else { @@ -962,7 +1025,7 @@ public DDatanode newDatanode(DatanodeInfo datanode) { public void executePendingMove(final PendingMove p) { - // move the block + // move the reportedBlock final DDatanode targetDn = p.target.getDDatanode(); ExecutorService moveExecutor = targetDn.getMoveExecutor(); if (moveExecutor == null) { @@ -975,7 +1038,6 @@ public void executePendingMove(final PendingMove p) { LOG.warn("No mover threads available: skip moving " + p); return; } - moveExecutor.execute(new Runnable() { @Override public void run() { @@ -1020,14 +1082,14 @@ public void run() { } } - // wait for all block moving to be done + // wait for all reportedBlock moving to be done waitForMoveCompletion(targets); return getBytesMoved() - bytesLastMoved; } /** - * Wait for all block move confirmations. + * Wait for all reportedBlock move confirmations. * @return true if there is failed move execution */ public static boolean waitForMoveCompletion( @@ -1066,10 +1128,10 @@ public static boolean checkForSuccess( } /** - * Decide if the block is a good candidate to be moved from source to target. - * A block is a good candidate if + * Decide if the block/blockGroup is a good candidate to be moved from source + * to target. A block is a good candidate if * 1. the block is not in the process of being moved/has not been moved; - * 2. the block does not have a replica on the target; + * 2. the block does not have a replica/internalBlock on the target; * 3. doing the move does not reduce the number of racks that the block has */ private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target, @@ -1086,7 +1148,7 @@ private boolean isGoodBlockCandidate(StorageGroup source, StorageGroup target, } final DatanodeInfo targetDatanode = target.getDatanodeInfo(); if (source.getDatanodeInfo().equals(targetDatanode)) { - // the block is moved inside same DN + // the reportedBlock is moved inside same DN return true; } 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 95d99838a2fea..2f214be03c29b 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 @@ -39,12 +39,12 @@ public interface BlockCollection { public ContentSummary computeContentSummary(BlockStoragePolicySuite bsps); /** - * @return the number of blocks + * @return the number of blocks or block groups */ public int numBlocks(); /** - * Get the blocks. + * Get the blocks (striped or contiguous). */ public BlockInfo[] getBlocks(); @@ -54,6 +54,12 @@ public interface BlockCollection { */ public long getPreferredBlockSize(); + /** + * Get block replication for the collection. + * @return block replication value. Return 0 if the file is erasure coded. + */ + public short getPreferredBlockReplication(); + /** * @return the storage policy ID. */ @@ -65,7 +71,7 @@ public interface BlockCollection { public String getName(); /** - * Set the block at the given index. + * Set the block (contiguous or striped) at the given index. */ public void setBlock(int index, BlockInfo blk); @@ -73,13 +79,19 @@ public interface BlockCollection { * Convert the last block of the collection to an under-construction block * and set the locations. */ - public void convertLastBlockToUC(BlockInfo lastBlock, DatanodeStorageInfo[] targets) throws IOException; + public void convertLastBlockToUC(BlockInfo lastBlock, + DatanodeStorageInfo[] targets) throws IOException; /** * @return whether the block collection is under construction. */ public boolean isUnderConstruction(); + /** + * @return whether the block collection is in striping format + */ + boolean isStriped(); + /** * @return the id for the block collection */ 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 feebd87fa73ce..685cfcb10d1f3 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 @@ -54,10 +54,12 @@ public class BlockIdManager { * The global block ID space for this file system. */ private final SequentialBlockIdGenerator blockIdGenerator; + private final SequentialBlockGroupIdGenerator blockGroupIdGenerator; public BlockIdManager(BlockManager blockManager) { this.generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP; this.blockIdGenerator = new SequentialBlockIdGenerator(blockManager); + this.blockGroupIdGenerator = new SequentialBlockGroupIdGenerator(blockManager); } /** @@ -102,20 +104,37 @@ SequentialBlockIdGenerator getBlockIdGenerator() { } /** - * 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 */ @@ -187,10 +206,14 @@ public boolean isLegacyBlock(Block block) { /** * Increments, logs and then returns the block ID */ - public long nextBlockId() { + public long nextContiguousBlockId() { return blockIdGenerator.nextValue(); } + public long nextStripedBlockId() { + return blockGroupIdGenerator.nextValue(); + } + public boolean isGenStampInFuture(Block block) { if (isLegacyBlock(block)) { return block.getGenerationStamp() > getGenerationStampV1(); @@ -206,4 +229,27 @@ public void clear() { .LAST_RESERVED_BLOCK_ID); generationStampV1Limit = HdfsConstants.GRANDFATHER_GENERATION_STAMP; } -} \ No newline at end of file + + public static boolean isStripedBlockID(long id) { + 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 & (~HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); + } + + public static int getBlockIndex(Block reportedBlock) { + return (int) (reportedBlock.getBlockId() & + HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); + } + + SequentialBlockGroupIdGenerator getBlockGroupIdGenerator() { + return blockGroupIdGenerator; + } +} 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 810784dbe2260..92a113581042d 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 @@ -31,9 +31,10 @@ import static org.apache.hadoop.hdfs.server.namenode.INodeId.INVALID_INODE_ID; /** - * BlockInfo class maintains for a given block - * the {@link BlockCollection} it is part of and datanodes where the replicas of - * the block are stored. + * For a given block (or an erasure coding block group), BlockInfo class + * maintains 1) the {@link BlockCollection} it is part of, and 2) datanodes + * where the replicas of the block, or blocks belonging to the erasure coding + * block group, are stored. */ @InterfaceAudience.Private public abstract class BlockInfo extends Block @@ -72,28 +73,20 @@ public abstract class BlockInfo extends Block /** * Construct an entry for blocksmap - * @param replication the block's replication factor + * @param size the block's replication factor, or the total number of blocks + * in the block group */ - public BlockInfo(short replication) { - this.triplets = new Object[3*replication]; + public BlockInfo(short size) { + this.triplets = new Object[3 * size]; this.bcId = INVALID_INODE_ID; - this.replication = replication; + this.replication = isStriped() ? 0 : size; } - public BlockInfo(Block blk, short replication) { + public BlockInfo(Block blk, short size) { super(blk); - this.triplets = new Object[3*replication]; + this.triplets = new Object[3*size]; this.bcId = INVALID_INODE_ID; - this.replication = replication; - } - - /** - * Copy construction. - * @param from BlockInfo to copy from. - */ - protected BlockInfo(BlockInfo from) { - this(from, from.getReplication()); - this.bcId = from.bcId; + this.replication = isStriped() ? 0 : size; } public short getReplication() { @@ -133,7 +126,7 @@ BlockInfo getPrevious(int index) { BlockInfo info = (BlockInfo)triplets[index*3+1]; assert info == null || info.getClass().getName().startsWith(BlockInfo.class.getName()) : - "BlockInfo is expected at " + index*3; + "BlockInfo is expected at " + index*3; return info; } @@ -164,7 +157,7 @@ void setStorageInfo(int index, DatanodeStorageInfo storage) { BlockInfo setPrevious(int index, BlockInfo to) { assert this.triplets != null : "BlockInfo is not initialized"; assert index >= 0 && index*3+1 < triplets.length : "Index is out of bound"; - BlockInfo info = (BlockInfo)triplets[index*3+1]; + BlockInfo info = (BlockInfo) triplets[index*3+1]; triplets[index*3+1] = to; return info; } @@ -175,12 +168,12 @@ BlockInfo setPrevious(int index, BlockInfo to) { * * @param index - the datanode index * @param to - block to be set to next on the list of blocks - * * @return current next block on the list of blocks + * @return current next block on the list of blocks */ BlockInfo setNext(int index, BlockInfo to) { assert this.triplets != null : "BlockInfo is not initialized"; assert index >= 0 && index*3+2 < triplets.length : "Index is out of bound"; - BlockInfo info = (BlockInfo)triplets[index*3+2]; + BlockInfo info = (BlockInfo) triplets[index*3+2]; triplets[index*3+2] = to; return info; } @@ -192,20 +185,31 @@ public int getCapacity() { } /** - * Count the number of data-nodes the block belongs to. + * Count the number of data-nodes the block currently belongs to (i.e., NN + * has received block reports from the DN). */ public abstract int numNodes(); /** - * Add a {@link DatanodeStorageInfo} location for a block. + * Add a {@link DatanodeStorageInfo} location for a block + * @param storage The storage to add + * @param reportedBlock The block reported from the datanode. This is only + * used by erasure coded blocks, this block's id contains + * information indicating the index of the block in the + * corresponding block group. */ - abstract boolean addStorage(DatanodeStorageInfo storage); + abstract boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock); /** * Remove {@link DatanodeStorageInfo} location for a block */ abstract boolean removeStorage(DatanodeStorageInfo storage); + public abstract boolean isStriped(); + + /** @return true if there is no datanode storage associated with the block */ + abstract boolean hasNoStorage(); + /** * Find specified DatanodeStorageInfo. * @return DatanodeStorageInfo or null if not found. @@ -214,10 +218,9 @@ DatanodeStorageInfo findStorageInfo(DatanodeDescriptor dn) { int len = getCapacity(); for(int idx = 0; idx < len; idx++) { DatanodeStorageInfo cur = getStorageInfo(idx); - if(cur == null) - break; - if(cur.getDatanodeDescriptor() == dn) + if(cur != null && cur.getDatanodeDescriptor() == dn) { return cur; + } } return null; } @@ -233,9 +236,6 @@ int findStorageInfo(DatanodeStorageInfo storageInfo) { if (cur == storageInfo) { return idx; } - if (cur == null) { - break; - } } return -1; } @@ -246,16 +246,16 @@ int findStorageInfo(DatanodeStorageInfo storageInfo) { * If the head is null then form a new list. * @return current block as the new head of the list. */ - BlockInfo listInsert(BlockInfo head, - DatanodeStorageInfo storage) { + BlockInfo listInsert(BlockInfo head, DatanodeStorageInfo storage) { int dnIndex = this.findStorageInfo(storage); assert dnIndex >= 0 : "Data node is not found: current"; assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : - "Block is already in the list and cannot be inserted."; + "Block is already in the list and cannot be inserted."; this.setPrevious(dnIndex, null); this.setNext(dnIndex, head); - if(head != null) + if (head != null) { head.setPrevious(head.findStorageInfo(storage), this); + } return this; } @@ -267,24 +267,28 @@ assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : * @return the new head of the list or null if the list becomes * empy after deletion. */ - BlockInfo listRemove(BlockInfo head, - DatanodeStorageInfo storage) { - if(head == null) + BlockInfo listRemove(BlockInfo head, DatanodeStorageInfo storage) { + if (head == null) { return null; + } int dnIndex = this.findStorageInfo(storage); - if(dnIndex < 0) // this block is not on the data-node list + if (dnIndex < 0) { // this block is not on the data-node list return head; + } BlockInfo next = this.getNext(dnIndex); BlockInfo prev = this.getPrevious(dnIndex); this.setNext(dnIndex, null); this.setPrevious(dnIndex, null); - if(prev != null) + if (prev != null) { prev.setNext(prev.findStorageInfo(storage), next); - if(next != null) + } + if (next != null) { next.setPrevious(next.findStorageInfo(storage), prev); - if(this == head) // removing the head + } + if (this == head) { // removing the head head = next; + } return head; } @@ -294,8 +298,8 @@ BlockInfo listRemove(BlockInfo head, * * @return the new head of the list. */ - public BlockInfo moveBlockToHead(BlockInfo head, - DatanodeStorageInfo storage, int curIndex, int headIndex) { + public BlockInfo moveBlockToHead(BlockInfo head, DatanodeStorageInfo storage, + int curIndex, int headIndex) { if (head == this) { return this; } @@ -357,11 +361,12 @@ public boolean isComplete() { public void convertToBlockUnderConstruction(BlockUCState s, DatanodeStorageInfo[] targets) { if (isComplete()) { - uc = new BlockUnderConstructionFeature(this, s, targets); + uc = new BlockUnderConstructionFeature(this, s, targets, + this.isStriped()); } else { // the block is already under construction uc.setBlockUCState(s); - uc.setExpectedLocations(this, targets); + uc.setExpectedLocations(this, targets, this.isStriped()); } } 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 94fb222068f8e..746e29895ed46 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 @@ -25,7 +25,6 @@ */ @InterfaceAudience.Private public class BlockInfoContiguous extends BlockInfo { - public static final BlockInfoContiguous[] EMPTY_ARRAY = {}; public BlockInfoContiguous(short size) { super(size); @@ -35,14 +34,6 @@ public BlockInfoContiguous(Block blk, short size) { super(blk, size); } - /** - * Copy construction. - * @param from BlockInfoContiguous to copy from. - */ - protected BlockInfoContiguous(BlockInfoContiguous from) { - super(from); - } - /** * Ensure that there is enough space to include num more triplets. * @return first free triplet index. @@ -62,7 +53,7 @@ private int ensureCapacity(int num) { } @Override - boolean addStorage(DatanodeStorageInfo storage) { + boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) { // find the last null node int lastNode = ensureCapacity(1); setStorageInfo(lastNode, storage); @@ -104,4 +95,14 @@ public int numNodes() { } return 0; } + + @Override + public final boolean isStriped() { + return false; + } + + @Override + final boolean hasNoStorage() { + return getStorageInfo(0) == null; + } } 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 new file mode 100644 index 0000000000000..d1067b3e037ce --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockInfoStriped.java @@ -0,0 +1,236 @@ +/** + * 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.util.StripedBlockUtil; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; + +/** + * Subclass of {@link BlockInfo}, presenting a block group in erasure coding. + * + * We still use triplets to store DatanodeStorageInfo for each block in the + * block group, as well as the previous/next block in the corresponding + * DatanodeStorageInfo. For a (m+k) block group, the first (m+k) triplet units + * are sorted and strictly mapped to the corresponding block. + * + * Normally each block belonging to group is stored in only one DataNode. + * However, it is possible that some block is over-replicated. Thus the triplet + * array's size can be larger than (m+k). Thus currently we use an extra byte + * array to record the block index for each triplet. + */ +public class BlockInfoStriped extends BlockInfo { + private final ErasureCodingPolicy ecPolicy; + /** + * Always the same size with triplets. Record the block index for each triplet + * TODO: actually this is only necessary for over-replicated block. Thus can + * be further optimized to save memory usage. + */ + private byte[] indices; + + public BlockInfoStriped(Block blk, ErasureCodingPolicy ecPolicy) { + super(blk, (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits())); + indices = new byte[ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()]; + initIndices(); + this.ecPolicy = ecPolicy; + } + + public short getTotalBlockNum() { + return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()); + } + + public short getDataBlockNum() { + return (short) ecPolicy.getNumDataUnits(); + } + + public short getParityBlockNum() { + return (short) ecPolicy.getNumParityUnits(); + } + + public int getCellSize() { + return ecPolicy.getCellSize(); + } + + /** + * If the block is committed/completed and its length is less than a full + * stripe, it returns the the number of actual data blocks. + * Otherwise it returns the number of data units specified by erasure coding policy. + */ + public short getRealDataBlockNum() { + if (isComplete() || getBlockUCState() == BlockUCState.COMMITTED) { + return (short) Math.min(getDataBlockNum(), + (getNumBytes() - 1) / ecPolicy.getCellSize() + 1); + } else { + return getDataBlockNum(); + } + } + + public short getRealTotalBlockNum() { + return (short) (getRealDataBlockNum() + getParityBlockNum()); + } + + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; + } + + private void initIndices() { + for (int i = 0; i < indices.length; i++) { + indices[i] = -1; + } + } + + private int findSlot() { + int i = getTotalBlockNum(); + for (; i < getCapacity(); i++) { + if (getStorageInfo(i) == null) { + return i; + } + } + // need to expand the triplet size + ensureCapacity(i + 1, true); + return i; + } + + @Override + boolean addStorage(DatanodeStorageInfo storage, Block reportedBlock) { + int blockIndex = BlockIdManager.getBlockIndex(reportedBlock); + int index = blockIndex; + DatanodeStorageInfo old = getStorageInfo(index); + if (old != null && !old.equals(storage)) { // over replicated + // check if the storage has been stored + int i = findStorageInfo(storage); + if (i == -1) { + index = findSlot(); + } else { + return true; + } + } + addStorage(storage, index, blockIndex); + return true; + } + + private void addStorage(DatanodeStorageInfo storage, int index, + int blockIndex) { + setStorageInfo(index, storage); + setNext(index, null); + setPrevious(index, null); + indices[index] = (byte) blockIndex; + } + + private int findStorageInfoFromEnd(DatanodeStorageInfo storage) { + final int len = getCapacity(); + for(int idx = len - 1; idx >= 0; idx--) { + DatanodeStorageInfo cur = getStorageInfo(idx); + if (storage.equals(cur)) { + return idx; + } + } + return -1; + } + + int getStorageBlockIndex(DatanodeStorageInfo storage) { + int i = this.findStorageInfo(storage); + 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); + if (dnIndex < 0) { // the node is not found + return false; + } + assert getPrevious(dnIndex) == null && getNext(dnIndex) == null : + "Block is still in the list and must be removed first."; + // set the triplet to null + setStorageInfo(dnIndex, null); + setNext(dnIndex, null); + setPrevious(dnIndex, null); + indices[dnIndex] = -1; + return true; + } + + private void ensureCapacity(int totalSize, boolean keepOld) { + if (getCapacity() < totalSize) { + Object[] old = triplets; + byte[] oldIndices = indices; + triplets = new Object[totalSize * 3]; + indices = new byte[totalSize]; + initIndices(); + + if (keepOld) { + System.arraycopy(old, 0, triplets, 0, old.length); + System.arraycopy(oldIndices, 0, indices, 0, oldIndices.length); + } + } + } + + 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 StripedBlockUtil.spaceConsumedByStripedBlock(getNumBytes(), + ecPolicy.getNumDataUnits(), ecPolicy.getNumParityUnits(), + ecPolicy.getCellSize()); + } + + @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"; + int num = 0; + for (int idx = getCapacity()-1; idx >= 0; idx--) { + if (getStorageInfo(idx) != null) { + num++; + } + } + return num; + } + + @Override + final boolean hasNoStorage() { + final int len = getCapacity(); + for(int idx = 0; idx < len; idx++) { + if (getStorageInfo(idx) != null) { + return false; + } + } + return true; + } +} 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 b0a11fe8e61f0..18bfc41cfbbf0 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 @@ -22,6 +22,7 @@ import java.io.IOException; import java.io.PrintWriter; import java.util.ArrayList; +import java.util.BitSet; import java.util.Collection; import java.util.Collections; import java.util.EnumSet; @@ -42,6 +43,7 @@ import org.apache.hadoop.HadoopIllegalArgumentException; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileEncryptionInfo; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSUtilClient; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; @@ -54,9 +56,9 @@ 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.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; @@ -78,6 +80,7 @@ import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations; import org.apache.hadoop.hdfs.server.protocol.DatanodeCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; @@ -87,9 +90,14 @@ import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.util.LightWeightHashSet; import org.apache.hadoop.hdfs.util.LightWeightLinkedSet; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; + +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.getInternalBlockLength; + import org.apache.hadoop.metrics2.util.MBeans; import org.apache.hadoop.net.Node; import org.apache.hadoop.security.UserGroupInformation; +import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.Daemon; import org.apache.hadoop.util.LightWeightGSet; import org.apache.hadoop.util.Time; @@ -190,7 +198,11 @@ public int getPendingDataNodeMessageCount() { /** 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; /** @@ -207,7 +219,7 @@ public int getPendingDataNodeMessageCount() { * Maps a StorageID to the set of blocks that are "extra" for this * DataNode. We'll eventually remove these extras. */ - public final Map> excessReplicateMap = + public final Map> excessReplicateMap = new HashMap<>(); /** @@ -275,12 +287,15 @@ public int getPendingDataNodeMessageCount() { private double replicationQueuesInitProgress = 0.0; /** for block replicas placement */ - private BlockPlacementPolicy blockplacement; + private BlockPlacementPolicies placementPolicies; private final BlockStoragePolicySuite storagePolicySuite; /** Check whether name system is running before terminating */ private boolean checkNSRunning = true; + /** Check whether there are any non-EC blocks using StripedID */ + private boolean hasNonEcBlockUsingStripedID = false; + public BlockManager(final Namesystem namesystem, final Configuration conf) throws IOException { this.namesystem = namesystem; @@ -296,7 +311,7 @@ public BlockManager(final Namesystem namesystem, final Configuration conf) // Compute the map capacity by allocating 2% of total memory blocksMap = new BlocksMap( LightWeightGSet.computeCapacity(2.0, "BlocksMap")); - blockplacement = BlockPlacementPolicy.getInstance( + placementPolicies = new BlockPlacementPolicies( conf, datanodeManager.getFSClusterStats(), datanodeManager.getNetworkTopology(), datanodeManager.getHost2DatanodeMap()); @@ -498,15 +513,7 @@ public DatanodeManager getDatanodeManager() { @VisibleForTesting public BlockPlacementPolicy getBlockPlacementPolicy() { - return blockplacement; - } - - /** Set BlockPlacementPolicy */ - public void setBlockPlacementPolicy(BlockPlacementPolicy newpolicy) { - if (newpolicy == null) { - throw new HadoopIllegalArgumentException("newpolicy == null"); - } - this.blockplacement = newpolicy; + return placementPolicies.getPolicy(false); } /** Dump meta data to out. */ @@ -556,9 +563,9 @@ private void dumpBlockMeta(Block block, PrintWriter out) { NumberReplicas numReplicas = new NumberReplicas(); // source node returned is not used - chooseSourceDatanode(block, containingNodes, + chooseSourceDatanodes(getStoredBlock(block), containingNodes, containingLiveReplicasNodes, numReplicas, - UnderReplicatedBlocks.LEVEL); + new LinkedList(), UnderReplicatedBlocks.LEVEL); // containingLiveReplicasNodes can include READ_ONLY_SHARED replicas which are // not included in the numReplicas.liveReplicas() count @@ -605,11 +612,28 @@ public int getMaxReplicationStreams() { return maxReplicationStreams; } - /** - * @return true if the block has minimum replicas - */ - public boolean checkMinReplication(BlockInfo block) { - return (countNodes(block).liveReplicas() >= minReplication); + public int getDefaultStorageNum(BlockInfo block) { + if (block.isStriped()) { + return ((BlockInfoStriped) block).getRealTotalBlockNum(); + } else { + return defaultReplication; + } + } + + public short getMinStorageNum(BlockInfo block) { + if (block.isStriped()) { + return ((BlockInfoStriped) block).getRealDataBlockNum(); + } else { + return minReplication; + } + } + + public boolean hasMinStorage(BlockInfo block) { + return countNodes(block).liveReplicas() >= getMinStorageNum(block); + } + + public boolean hasMinStorage(BlockInfo block, int liveNum) { + return liveNum >= getMinStorageNum(block); } /** @@ -626,8 +650,8 @@ private static boolean commitBlock(final BlockInfo block, 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(); + "commitBlock length is less than the stored one " + + commitBlock.getNumBytes() + " vs. " + block.getNumBytes(); block.commitBlock(commitBlock); return true; } @@ -653,7 +677,7 @@ public boolean commitOrCompleteLastBlock(BlockCollection bc, return false; // already completed (e.g. by syncBlock) final boolean b = commitBlock(lastBlock, commitBlock); - if (countNodes(lastBlock).liveReplicas() >= minReplication) { + if (hasMinStorage(lastBlock)) { completeBlock(lastBlock, false); } return b; @@ -671,7 +695,7 @@ private void completeBlock(BlockInfo curBlock, boolean force) } int numNodes = curBlock.numNodes(); - if (!force && numNodes < minReplication) { + if (!force && !hasMinStorage(curBlock, numNodes)) { throw new IOException("Cannot complete block: " + "block does not satisfy minimal replication requirement."); } @@ -688,8 +712,10 @@ private void completeBlock(BlockInfo curBlock, boolean force) // 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).getRealDataBlockNum() : minReplication; namesystem.incrementSafeBlockCount( - Math.min(numNodes, minReplication)); + Math.min(numNodes, minStorage), curBlock); } /** @@ -740,14 +766,17 @@ public LocatedBlock convertLastBlockToUnderConstruction( // remove this block from the list of pending blocks to be deleted. for (DatanodeStorageInfo storage : targets) { - invalidateBlocks.remove(storage.getDatanodeDescriptor(), lastBlock); + final Block b = getBlockOnStorage(lastBlock, storage); + if (b != null) { + invalidateBlocks.remove(storage.getDatanodeDescriptor(), b); + } } // Adjust safe-mode totals, since under-construction blocks don't // count in safe-mode. namesystem.adjustSafeModeBlockTotals( // decrement safe if we had enough - targets.length >= minReplication ? -1 : 0, + hasMinStorage(lastBlock, targets.length) ? -1 : 0, // always decrement total blocks -1); @@ -761,23 +790,24 @@ public LocatedBlock convertLastBlockToUnderConstruction( /** * Get all valid locations of the block */ - private List getValidLocations(Block block) { + private List getValidLocations(BlockInfo block) { final List locations = new ArrayList(blocksMap.numNodes(block)); for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { // filter invalidate replicas - if(!invalidateBlocks.contains(storage.getDatanodeDescriptor(), block)) { + Block b = getBlockOnStorage(block, storage); + if(b != null && + !invalidateBlocks.contains(storage.getDatanodeDescriptor(), b)) { locations.add(storage); } } return locations; } - - private List createLocatedBlockList( - final BlockInfo[] blocks, + + private List 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++) { @@ -790,10 +820,10 @@ private List createLocatedBlockList( } if (nrBlocks > 0 && curBlk == nrBlocks) // offset >= end of file - return Collections.emptyList(); + return Collections.emptyList(); long endOff = offset + length; - List results = new ArrayList(blocks.length); + List results = new ArrayList<>(blocks.length); do { results.add(createLocatedBlock(blocks[curBlk], curPos, mode)); curPos += blocks[curBlk].getNumBytes(); @@ -806,7 +836,7 @@ private List createLocatedBlockList( 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++) { @@ -819,7 +849,7 @@ private LocatedBlock createLocatedBlock(final BlockInfo[] blocks, return createLocatedBlock(blocks[curBlk], curPos, mode); } - + private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos, final AccessMode mode) throws IOException { final LocatedBlock lb = createLocatedBlock(blk, pos); @@ -830,13 +860,22 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos, } /** @return a LocatedBlock for the given block */ - private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos - ) throws IOException { + private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos) + throws IOException { if (!blk.isComplete()) { - final DatanodeStorageInfo[] storages = blk.getUnderConstructionFeature() - .getExpectedStorageLocations(); - final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); - return newLocatedBlock(eb, storages, pos, false); + final BlockUnderConstructionFeature uc = blk.getUnderConstructionFeature(); + if (blk.isStriped()) { + final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); + final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), + blk); + return newLocatedStripedBlock(eb, storages, uc.getBlockIndices(), pos, + false); + } else { + final DatanodeStorageInfo[] storages = uc.getExpectedStorageLocations(); + final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), + blk); + return newLocatedBlock(eb, storages, pos, false); + } } // get block locations @@ -853,13 +892,21 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos 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 : @@ -869,7 +916,9 @@ private LocatedBlock createLocatedBlock(final BlockInfo blk, final long pos " numCorrupt: " + numCorruptNodes + " numCorruptRepls: " + numCorruptReplicas; final ExtendedBlock eb = new ExtendedBlock(namesystem.getBlockPoolId(), blk); - return newLocatedBlock(eb, machines, pos, isCorrupt); + return blockIndices == null ? + newLocatedBlock(eb, machines, pos, isCorrupt) : + newLocatedStripedBlock(eb, machines, blockIndices, pos, isCorrupt); } /** Create a LocatedBlocks. */ @@ -877,14 +926,15 @@ public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks, final long fileSizeExcludeBlocksUnderConstruction, final boolean isFileUnderConstruction, final long offset, final long length, final boolean needBlockToken, - final boolean inSnapshot, FileEncryptionInfo feInfo) + final boolean inSnapshot, FileEncryptionInfo feInfo, + ErasureCodingPolicy ecPolicy) throws IOException { assert namesystem.hasReadLock(); if (blocks == null) { return null; } else if (blocks.length == 0) { return new LocatedBlocks(0, isFileUnderConstruction, - Collections.emptyList(), null, false, feInfo); + Collections. emptyList(), null, false, feInfo, ecPolicy); } else { if (LOG.isDebugEnabled()) { LOG.debug("blocks = " + java.util.Arrays.asList(blocks)); @@ -907,9 +957,9 @@ public LocatedBlocks createLocatedBlocks(final BlockInfo[] blocks, fileSizeExcludeBlocksUnderConstruction, mode); isComplete = true; } - return new LocatedBlocks( - fileSizeExcludeBlocksUnderConstruction, isFileUnderConstruction, - locatedblocks, lastlb, isComplete, feInfo); + return new LocatedBlocks(fileSizeExcludeBlocksUnderConstruction, + isFileUnderConstruction, locatedblocks, lastlb, isComplete, feInfo, + ecPolicy); } } @@ -924,9 +974,24 @@ public void setBlockToken(final LocatedBlock b, final AccessMode mode) throws IOException { if (isBlockTokenEnabled()) { // Use cached UGI if serving RPC calls. - b.setBlockToken(blockTokenSecretManager.generateToken( - NameNode.getRemoteUser().getShortUserName(), - b.getBlock(), EnumSet.of(mode))); + if (b.isStriped()) { + Preconditions.checkState(b instanceof LocatedStripedBlock); + LocatedStripedBlock sb = (LocatedStripedBlock) b; + int[] indices = sb.getBlockIndices(); + Token[] blockTokens = new Token[indices.length]; + ExtendedBlock internalBlock = new ExtendedBlock(b.getBlock()); + for (int i = 0; i < indices.length; i++) { + internalBlock.setBlockId(b.getBlock().getBlockId() + indices[i]); + blockTokens[i] = blockTokenSecretManager.generateToken( + NameNode.getRemoteUser().getShortUserName(), + internalBlock, EnumSet.of(mode)); + } + sb.setBlockTokens(blockTokens); + } else { + b.setBlockToken(blockTokenSecretManager.generateToken( + NameNode.getRemoteUser().getShortUserName(), + b.getBlock(), EnumSet.of(mode))); + } } } @@ -1068,7 +1133,7 @@ private BlocksWithLocations getBlocksWithLocations(final DatanodeID datanode, /** Remove the blocks associated to the given datanode. */ void removeBlocksAssociatedTo(final DatanodeDescriptor node) { - final Iterator it = node.getBlockIterator(); + final Iterator it = node.getBlockIterator(); while(it.hasNext()) { removeStoredBlock(it.next(), node); } @@ -1082,12 +1147,15 @@ void removeBlocksAssociatedTo(final DatanodeDescriptor node) { /** Remove the blocks associated to the given DatanodeStorageInfo. */ void removeBlocksAssociatedTo(final DatanodeStorageInfo storageInfo) { assert namesystem.hasWriteLock(); - final Iterator it = storageInfo.getBlockIterator(); + final Iterator 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); + final Block b = getBlockOnStorage(block, storageInfo); + if (b != null) { + invalidateBlocks.remove(node, b); + } } namesystem.checkSafeMode(); } @@ -1107,22 +1175,32 @@ void addToInvalidates(final Block block, final DatanodeInfo datanode) { * 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 (!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 = getBlockOnStorage(storedBlock, storage); + if (b != null) { + invalidateBlocks.add(b, node, false); + datanodes.append(node).append(" "); + } } if (datanodes.length() != 0) { - blockLog.debug("BLOCK* addToInvalidates: {} {}", b, + blockLog.debug("BLOCK* addToInvalidates: {} {}", storedBlock, datanodes.toString()); } } + private Block getBlockOnStorage(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 @@ -1146,7 +1224,8 @@ void removeFromInvalidates(final DatanodeInfo datanode) { public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk, final DatanodeInfo dn, String storageID, String reason) throws IOException { assert namesystem.hasWriteLock(); - final BlockInfo storedBlock = getStoredBlock(blk.getLocalBlock()); + final Block reportedBlock = blk.getLocalBlock(); + final BlockInfo storedBlock = getStoredBlock(reportedBlock); if (storedBlock == null) { // Check if the replica is in the blockMap, if not // ignore the request for now. This could happen when BlockScanner @@ -1163,44 +1242,52 @@ public void findAndMarkBlockAsCorrupt(final ExtendedBlock blk, + ") does not exist"); } - markBlockAsCorrupt(new BlockToMarkCorrupt(storedBlock, + markBlockAsCorrupt(new BlockToMarkCorrupt(reportedBlock, storedBlock, blk.getGenerationStamp(), reason, Reason.CORRUPTION_REPORTED), storageID == null ? null : node.getStorageInfo(storageID), node); } /** - * - * @param b + * Mark a replica (of a contiguous block) or an internal block (of a striped + * block group) as corrupt. + * @param b Indicating the reported bad block and the corresponding BlockInfo + * stored in blocksMap. * @param storageInfo storage that contains the block, if known. null otherwise. - * @throws IOException */ private void markBlockAsCorrupt(BlockToMarkCorrupt b, DatanodeStorageInfo storageInfo, DatanodeDescriptor node) throws IOException { - - if (b.getCorrupted().isDeleted()) { + if (b.getStored().isDeleted()) { blockLog.debug("BLOCK markBlockAsCorrupt: {} cannot be marked as" + " corrupt as it does not belong to any file", b); addToInvalidates(b.getCorrupted(), node); return; - } - short expectedReplicas = b.getCorrupted().getReplication(); + } + short expectedReplicas = + getExpectedReplicaNum(b.getStored()); // Add replica to the data-node if it is not already there if (storageInfo != null) { - storageInfo.addBlock(b.getStored()); + storageInfo.addBlock(b.getStored(), b.getCorrupted()); } - // Add this replica to corruptReplicas Map - corruptReplicas.addToCorruptReplicasMap(b.getCorrupted(), node, - b.getReason(), b.getReasonCode()); + // Add this replica to corruptReplicas Map. For striped blocks, we always + // use the id of whole striped block group when adding to corruptReplicas + Block corrupted = new Block(b.getCorrupted()); + if (b.getStored().isStriped()) { + corrupted.setBlockId(b.getStored().getBlockId()); + } + corruptReplicas.addToCorruptReplicasMap(corrupted, node, b.getReason(), + b.getReasonCode()); NumberReplicas numberOfReplicas = countNodes(b.getStored()); boolean hasEnoughLiveReplicas = numberOfReplicas.liveReplicas() >= expectedReplicas; - boolean minReplicationSatisfied = - numberOfReplicas.liveReplicas() >= minReplication; + + boolean minReplicationSatisfied = hasMinStorage(b.getStored(), + numberOfReplicas.liveReplicas()); + boolean hasMoreCorruptReplicas = minReplicationSatisfied && (numberOfReplicas.liveReplicas() + numberOfReplicas.corruptReplicas()) > expectedReplicas; @@ -1215,7 +1302,7 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, if (hasEnoughLiveReplicas || hasMoreCorruptReplicas || corruptedDuringWrite) { // the block is over-replicated so invalidate the replicas immediately - invalidateBlock(b, node); + invalidateBlock(b, node, numberOfReplicas); } else if (isPopulatingReplQueues()) { // add the block to neededReplication updateNeededReplications(b.getStored(), -1, 0); @@ -1227,8 +1314,8 @@ private void markBlockAsCorrupt(BlockToMarkCorrupt b, * @return true if the block was successfully invalidated and no longer * present in the BlocksMap */ - private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn - ) throws IOException { + private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn, + NumberReplicas nr) throws IOException { blockLog.debug("BLOCK* invalidateBlock: {} on {}", b, dn); DatanodeDescriptor node = getDatanodeManager().getDatanode(dn); if (node == null) { @@ -1237,7 +1324,6 @@ private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn } // Check how many copies we have of the block - NumberReplicas nr = countNodes(b.getStored()); if (nr.replicasOnStaleNodes() > 0) { blockLog.debug("BLOCK* invalidateBlocks: postponing " + "invalidation of {} on {} because {} replica(s) are located on " + @@ -1245,17 +1331,14 @@ private boolean invalidateBlock(BlockToMarkCorrupt b, DatanodeInfo dn nr.replicasOnStaleNodes()); postponeBlock(b.getCorrupted()); return false; - } else if (nr.liveReplicas() >= 1) { - // If we have at least one copy on a live node, then we can delete it. + } else { + // we already checked the number of replicas in the caller of this + // function and know there are enough live replicas, so we can delete it. addToInvalidates(b.getCorrupted(), dn); removeStoredBlock(b.getStored(), node); blockLog.debug("BLOCK* invalidateBlocks: {} on {} listed for deletion.", b, dn); return true; - } else { - blockLog.debug("BLOCK* invalidateBlocks: {} on {} is the only copy and" + - " was not deleted", b, dn); - return false; } } @@ -1308,15 +1391,15 @@ int computeInvalidateWork(int nodesToProcess) { } /** - * 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> blocksToReplicate = null; namesystem.writeLock(); try { @@ -1326,27 +1409,35 @@ int computeReplicationWork(int blocksToProcess) { } 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> blocksToReplicate) { + int computeRecoveryWorkForBlocks(List> blocksToRecover) { + int requiredReplication, numEffectiveReplicas; + List containingNodes; + BlockCollection bc; + int additionalReplRequired; + int scheduledWork = 0; - final List work = new LinkedList<>(); + List 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)) { - ReplicationWork rw = scheduleReplication(block, priority); + for (int priority = 0; priority < blocksToRecover.size(); priority++) { + for (BlockInfo block : blocksToRecover.get(priority)) { + BlockRecoveryWork rw = scheduleRecovery(block, priority); if (rw != null) { - work.add(rw); + recovWork.add(rw); } } } @@ -1355,8 +1446,9 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { namesystem.writeUnlock(); } + // Step 2: choose target nodes for each recovery task final Set excludedNodes = new HashSet<>(); - 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(); @@ -1367,12 +1459,15 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { // choose replication targets: NOT HOLDING THE GLOBAL LOCK // It is costly to extract the filename for which chooseTargets is called, // so for now we pass in the block collection itself. - rw.chooseTargets(blockplacement, storagePolicySuite, excludedNodes); + final BlockPlacementPolicy placementPolicy = + placementPolicies.getPolicy(rw.getBlock().isStriped()); + rw.chooseTargets(placementPolicy, storagePolicySuite, excludedNodes); } + // Step 3: add tasks to the DN namesystem.writeLock(); try { - for(ReplicationWork rw : work){ + for(BlockRecoveryWork rw : recovWork){ final DatanodeStorageInfo[] targets = rw.getTargets(); if(targets == null || targets.length == 0){ rw.resetTargets(); @@ -1380,7 +1475,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { } synchronized (neededReplications) { - if (validateReplicationWork(rw)) { + if (validateRecoveryWork(rw)) { scheduledWork++; } } @@ -1391,7 +1486,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { if (blockLog.isInfoEnabled()) { // log which blocks have been scheduled for replication - for(ReplicationWork rw : work){ + for(BlockRecoveryWork rw : recovWork){ DatanodeStorageInfo[] targets = rw.getTargets(); if (targets != null && targets.length != 0) { StringBuilder targetList = new StringBuilder("datanode(s)"); @@ -1399,7 +1494,7 @@ int computeReplicationWorkForBlocks(List> blocksToReplicate) { targetList.append(' '); targetList.append(target.getDatanodeDescriptor()); } - blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNode(), + blockLog.debug("BLOCK* ask {} to replicate {} to {}", rw.getSrcNodes(), rw.getBlock(), targetList); } } @@ -1416,10 +1511,10 @@ boolean hasEnoughEffectiveReplicas(BlockInfo block, NumberReplicas numReplicas, int pendingReplicaNum, int required) { int numEffectiveReplicas = numReplicas.liveReplicas() + pendingReplicaNum; return (numEffectiveReplicas >= required) && - (pendingReplicaNum > 0 || blockHasEnoughRacks(block)); + (pendingReplicaNum > 0 || blockHasEnoughRacks(block, required)); } - private ReplicationWork scheduleReplication(BlockInfo block, int priority) { + private BlockRecoveryWork scheduleRecovery(BlockInfo block, int priority) { // block should belong to a file BlockCollection bc = getBlockCollection(block); // abandoned block or block reopened for append @@ -1436,10 +1531,14 @@ private ReplicationWork scheduleReplication(BlockInfo block, int priority) { List containingNodes = new ArrayList<>(); List liveReplicaNodes = new ArrayList<>(); NumberReplicas numReplicas = new NumberReplicas(); - DatanodeDescriptor srcNode = chooseSourceDatanode(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"); + List liveBlockIndices = new ArrayList<>(); + final DatanodeDescriptor[] srcNodes = chooseSourceDatanodes(block, + containingNodes, liveReplicaNodes, numReplicas, + liveBlockIndices, priority); + if(srcNodes == null || srcNodes.length == 0) { + // block can not be recovered from any node + LOG.debug("Block " + block + " cannot be recovered " + + "from any node"); return null; } @@ -1463,11 +1562,23 @@ private ReplicationWork scheduleReplication(BlockInfo block, int priority) { } else { additionalReplRequired = 1; // Needed on a new rack } - return new ReplicationWork(block, bc, srcNode, containingNodes, - liveReplicaNodes, additionalReplRequired, priority); + + if (block.isStriped()) { + short[] indices = new short[liveBlockIndices.size()]; + for (int i = 0 ; i < liveBlockIndices.size(); i++) { + indices[i] = liveBlockIndices.get(i); + } + return new ErasureCodingWork(block, bc, srcNodes, + containingNodes, liveReplicaNodes, additionalReplRequired, + priority, indices); + } else { + return new ReplicationWork(block, bc, srcNodes, + containingNodes, liveReplicaNodes, additionalReplRequired, + priority); + } } - private boolean validateReplicationWork(ReplicationWork rw) { + private boolean validateRecoveryWork(BlockRecoveryWork rw) { BlockInfo block = rw.getBlock(); int priority = rw.getPriority(); // Recheck since global lock was released @@ -1496,16 +1607,41 @@ private boolean validateReplicationWork(ReplicationWork rw) { DatanodeStorageInfo[] targets = rw.getTargets(); if ( (numReplicas.liveReplicas() >= requiredReplication) && - (!blockHasEnoughRacks(block)) ) { - if (rw.getSrcNode().getNetworkLocation().equals( + (!blockHasEnoughRacks(block, requiredReplication)) ) { + if (rw.getSrcNodes()[0].getNetworkLocation().equals( targets[0].getDatanodeDescriptor().getNetworkLocation())) { //No use continuing, unless a new rack in this case return false; } } - // Add block to the to be replicated list - rw.getSrcNode().addBlockToBeReplicated(block, targets); + // Add block to the to be recovered list + if (block.isStriped()) { + assert rw instanceof ErasureCodingWork; + assert rw.getTargets().length > 0; + String src = getBlockCollection(block).getName(); + ErasureCodingPolicy ecPolicy = null; + try { + ecPolicy = namesystem.getErasureCodingPolicyForPath(src); + } catch (IOException e) { + blockLog + .warn("Failed to get EC policy for the file {} ", src); + } + if (ecPolicy == null) { + blockLog.warn("No erasure coding policy found for the file {}. " + + "So cannot proceed for recovery", src); + // TODO: we may have to revisit later for what we can do better to + // handle this case. + return false; + } + rw.getTargets()[0].getDatanodeDescriptor().addBlockToBeErasureCoded( + new ExtendedBlock(namesystem.getBlockPoolId(), block), + rw.getSrcNodes(), rw.getTargets(), + ((ErasureCodingWork) rw).getLiveBlockIndicies(), ecPolicy); + } else { + rw.getSrcNodes()[0].addBlockToBeReplicated(block, targets); + } + DatanodeStorageInfo.incrementBlocksScheduled(targets); // Move the block-replication into a "pending" state. @@ -1527,7 +1663,7 @@ private boolean validateReplicationWork(ReplicationWork rw) { /** Choose target for WebHDFS redirection. */ public DatanodeStorageInfo[] chooseTarget4WebHDFS(String src, DatanodeDescriptor clientnode, Set excludes, long blocksize) { - return blockplacement.chooseTarget(src, 1, clientnode, + return placementPolicies.getPolicy(false).chooseTarget(src, 1, clientnode, Collections.emptyList(), false, excludes, blocksize, storagePolicySuite.getDefaultPolicy()); } @@ -1539,9 +1675,10 @@ public DatanodeStorageInfo[] chooseTarget4AdditionalDatanode(String src, List chosen, Set excludes, long blocksize, - byte storagePolicyID) { - + byte storagePolicyID, + boolean isStriped) { final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID); + final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped); return blockplacement.chooseTarget(src, numAdditionalNodes, clientnode, chosen, true, excludes, blocksize, storagePolicy); } @@ -1559,10 +1696,12 @@ public DatanodeStorageInfo[] chooseTarget4NewBlock(final String src, final Set excludedNodes, final long blocksize, final List favoredNodes, - final byte storagePolicyID) throws IOException { + final byte storagePolicyID, + final boolean isStriped) throws IOException { List favoredDatanodeDescriptors = getDatanodeDescriptors(favoredNodes); final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(storagePolicyID); + final BlockPlacementPolicy blockplacement = placementPolicies.getPolicy(isStriped); final DatanodeStorageInfo[] targets = blockplacement.chooseTarget(src, numOfReplicas, client, excludedNodes, blocksize, favoredDatanodeDescriptors, storagePolicy); @@ -1597,55 +1736,59 @@ List getDatanodeDescriptors(List nodes) { } /** - * 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 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 DatanodeDescriptor of the chosen node from which to replicate - * the given block - */ - @VisibleForTesting - DatanodeDescriptor chooseSourceDatanode(Block block, - List containingNodes, - List 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 containingNodes, + List nodesContainingLiveReplicas, + NumberReplicas numReplicas, + List liveBlockIndices, int priority) { containingNodes.clear(); nodesContainingLiveReplicas.clear(); - DatanodeDescriptor srcNode = null; + List srcNodes = new ArrayList<>(); int live = 0; int decommissioned = 0; int decommissioning = 0; int corrupt = 0; int excess = 0; - + liveBlockIndices.clear(); + final boolean isStriped = block.isStriped(); + Collection nodesCorrupt = corruptReplicas.getNodes(block); - for(DatanodeStorageInfo storage : blocksMap.getStorages(block)) { + for (DatanodeStorageInfo storage : blocksMap.getStorages(block)) { final DatanodeDescriptor node = storage.getDatanodeDescriptor(); - LightWeightLinkedSet excessBlocks = + LightWeightLinkedSet 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()) { @@ -1680,21 +1823,25 @@ else if (node.isDecommissionInProgress()) { if(node.isDecommissioned()) continue; - // We got this far, current node is a reasonable choice - if (srcNode == null) { - srcNode = node; + 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(ThreadLocalRandom.current().nextBoolean()) - srcNode = node; + if (!isStriped && ThreadLocalRandom.current().nextBoolean()) { + srcNodes.set(0, node); + } } if(numReplicas != null) numReplicas.initialize(live, decommissioned, decommissioning, corrupt, excess, 0); - return srcNode; + return srcNodes.toArray(new DatanodeDescriptor[srcNodes.size()]); } /** @@ -1758,7 +1905,7 @@ public long requestBlockReportLeaseId(DatanodeRegistration nodeReg) { * reported by the datanode in the block report. */ static class StatefulBlockInfo { - final BlockInfo storedBlock; + final BlockInfo storedBlock; // should be UC block final Block reportedBlock; final ReplicaState reportedState; @@ -1771,6 +1918,16 @@ static class StatefulBlockInfo { } } + private static class BlockInfoToAdd { + final BlockInfo stored; + final Block reported; + + BlockInfoToAdd(BlockInfo stored, Block reported) { + this.stored = stored; + this.reported = reported; + } + } + /** * The given storage is reporting all its blocks. * Update the (storage-->block list) and (block-->storage list) maps. @@ -1874,8 +2031,8 @@ public boolean processReport(final DatanodeID nodeID, metrics.addBlockReport((int) (endTime - startTime)); } blockLog.info("BLOCK* processReport: from storage {} node {}, " + - "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage - .getStorageID(), nodeID, newReport.getNumberOfBlocks(), + "blocks: {}, hasStaleStorage: {}, processing time: {} msecs", storage + .getStorageID(), nodeID, newReport.getNumberOfBlocks(), node.hasStaleStorages(), (endTime - startTime)); return !node.hasStaleStorages(); } @@ -1883,8 +2040,8 @@ public boolean processReport(final DatanodeID nodeID, private void removeZombieReplicas(BlockReportContext context, DatanodeStorageInfo zombie) { LOG.warn("processReport 0x{}: removing zombie storage {}, which no " + - "longer exists on the DataNode.", - Long.toHexString(context.getReportId()), zombie.getStorageID()); + "longer exists on the DataNode.", + Long.toHexString(context.getReportId()), zombie.getStorageID()); assert(namesystem.hasWriteLock()); Iterator iter = zombie.getBlockIterator(); int prevBlocks = zombie.numBlocks(); @@ -1897,13 +2054,16 @@ private void removeZombieReplicas(BlockReportContext context, // more than one storage on a datanode (and because it's a difficult // assumption to really enforce) removeStoredBlock(block, zombie.getDatanodeDescriptor()); - invalidateBlocks.remove(zombie.getDatanodeDescriptor(), block); + Block b = getBlockOnStorage(block, zombie); + if (b != null) { + invalidateBlocks.remove(zombie.getDatanodeDescriptor(), b); + } } assert(zombie.numBlocks() == 0); LOG.warn("processReport 0x{}: removed {} replicas from storage {}, " + "which no longer exists on the DataNode.", - Long.toHexString(context.getReportId()), prevBlocks, - zombie.getStorageID()); + Long.toHexString(context.getReportId()), prevBlocks, + zombie.getStorageID()); } /** @@ -1947,7 +2107,7 @@ void rescanPostponedMisreplicatedBlocks() { break; } - BlockInfo bi = blocksMap.getStoredBlock(b); + BlockInfo bi = getStoredBlock(b); if (bi == null) { if (LOG.isDebugEnabled()) { LOG.debug("BLOCK* rescanPostponedMisreplicatedBlocks: " + @@ -1987,11 +2147,11 @@ private Collection processReport( // Modify the (block-->datanode) map, according to the difference // between the old and new block report. // - Collection toAdd = new LinkedList(); - Collection toRemove = new TreeSet(); - Collection toInvalidate = new LinkedList(); - Collection toCorrupt = new LinkedList(); - Collection toUC = new LinkedList(); + Collection toAdd = new LinkedList<>(); + Collection toRemove = new TreeSet<>(); + Collection toInvalidate = new LinkedList<>(); + Collection toCorrupt = new LinkedList<>(); + Collection toUC = new LinkedList<>(); reportDiff(storageInfo, report, toAdd, toRemove, toInvalidate, toCorrupt, toUC); @@ -2000,12 +2160,13 @@ private Collection processReport( for (StatefulBlockInfo b : toUC) { addStoredBlockUnderConstruction(b, storageInfo); } - for (Block b : toRemove) { + for (BlockInfo b : toRemove) { removeStoredBlock(b, node); } int numBlocksLogged = 0; - for (BlockInfo b : toAdd) { - addStoredBlock(b, storageInfo, null, numBlocksLogged < maxNumBlocksToLog); + for (BlockInfoToAdd b : toAdd) { + addStoredBlock(b.stored, b.reported, storageInfo, null, + numBlocksLogged < maxNumBlocksToLog); numBlocksLogged++; } if (numBlocksLogged > maxNumBlocksToLog) { @@ -2026,17 +2187,18 @@ private Collection processReport( * Mark block replicas as corrupt except those on the storages in * newStorages list. */ - public void markBlockReplicasAsCorrupt(BlockInfo 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 { @@ -2094,8 +2256,8 @@ private void processFirstBlockReport( QUEUE_REASON_FUTURE_GENSTAMP); continue; } - - BlockInfo storedBlock = blocksMap.getStoredBlock(iblk); + + BlockInfo storedBlock = getStoredBlock(iblk); // If block does not belong to any file, we are done. if (storedBlock == null) continue; @@ -2126,29 +2288,31 @@ private void processFirstBlockReport( if (namesystem.isInSnapshot(storedBlock)) { int numOfReplicas = storedBlock.getUnderConstructionFeature() .getNumExpectedLocations(); - namesystem.incrementSafeBlockCount(numOfReplicas); + namesystem.incrementSafeBlockCount(numOfReplicas, storedBlock); } //and fall through to next clause } //add replica if appropriate if (reportedState == ReplicaState.FINALIZED) { - addStoredBlockImmediate(storedBlock, storageInfo); + addStoredBlockImmediate(storedBlock, iblk, storageInfo); } } } private void reportDiff(DatanodeStorageInfo storageInfo, - BlockListAsLongs newReport, - Collection toAdd, // add to DatanodeDescriptor - Collection toRemove, // remove from DatanodeDescriptor + BlockListAsLongs newReport, + Collection toAdd, // add to DatanodeDescriptor + Collection toRemove, // remove from DatanodeDescriptor Collection toInvalidate, // should be removed from DN Collection toCorrupt, // add to corrupt replicas list Collection toUC) { // add to under-construction list // place a delimiter in the list which separates blocks // that have been reported from those that have not - BlockInfo delimiter = new BlockInfoContiguous(new Block(), (short) 1); - AddBlockResult result = storageInfo.addBlock(delimiter); + Block delimiterBlock = new Block(); + BlockInfo delimiter = new BlockInfoContiguous(delimiterBlock, + (short) 1); + AddBlockResult result = storageInfo.addBlock(delimiter, delimiterBlock); assert result == AddBlockResult.ADDED : "Delimiting block cannot be present in the node"; int headIndex = 0; //currently the delimiter is in the head of the list @@ -2174,8 +2338,9 @@ private void reportDiff(DatanodeStorageInfo storageInfo, // all of them are next to the delimiter Iterator it = storageInfo.new BlockIterator(delimiter.getNext(0)); - while(it.hasNext()) + while (it.hasNext()) { toRemove.add(it.next()); + } storageInfo.removeBlock(delimiter); } @@ -2212,8 +2377,8 @@ private void reportDiff(DatanodeStorageInfo storageInfo, */ private BlockInfo processReportedBlock( final DatanodeStorageInfo storageInfo, - final Block block, final ReplicaState reportedState, - final Collection toAdd, + final Block block, final ReplicaState reportedState, + final Collection toAdd, final Collection toInvalidate, final Collection toCorrupt, final Collection toUC) { @@ -2234,7 +2399,7 @@ private BlockInfo processReportedBlock( } // find block by blockId - BlockInfo storedBlock = blocksMap.getStoredBlock(block); + BlockInfo storedBlock = getStoredBlock(block); if(storedBlock == null) { // If blocksMap does not contain reported block id, // the replica should be removed from the data-node. @@ -2282,7 +2447,7 @@ private BlockInfo processReportedBlock( if (reportedState == ReplicaState.FINALIZED && (storedBlock.findStorageInfo(storageInfo) == -1 || corruptReplicas.isReplicaCorrupt(storedBlock, dn))) { - toAdd.add(storedBlock); + toAdd.add(new BlockInfoToAdd(storedBlock, block)); } return storedBlock; } @@ -2328,7 +2493,7 @@ private void processQueuedMessages(Iterable rbis) if (rbi.getReportedState() == null) { // This is a DELETE_BLOCK request DatanodeStorageInfo storageInfo = rbi.getStorageInfo(); - removeStoredBlock(rbi.getBlock(), + removeStoredBlock(getStoredBlock(rbi.getBlock()), storageInfo.getDatanodeDescriptor()); } else { processAndHandleReportedBlock(rbi.getStorageInfo(), @@ -2376,12 +2541,26 @@ private BlockToMarkCorrupt checkReplicaCorrupt( case COMMITTED: if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(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(storedBlock, + } + boolean wrongSize; + if (storedBlock.isStriped()) { + assert BlockIdManager.isStripedBlockID(reported.getBlockId()); + assert storedBlock.getBlockId() == + BlockIdManager.convertToStripedID(reported.getBlockId()); + BlockInfoStriped stripedBlock = (BlockInfoStriped) storedBlock; + int reportedBlkIdx = BlockIdManager.getBlockIndex(reported); + wrongSize = reported.getNumBytes() != getInternalBlockLength( + stripedBlock.getNumBytes(), stripedBlock.getCellSize(), + stripedBlock.getDataBlockNum(), reportedBlkIdx); + } else { + wrongSize = storedBlock.getNumBytes() != reported.getNumBytes(); + } + if (wrongSize) { + return new BlockToMarkCorrupt(new Block(reported), storedBlock, "block is " + ucState + " and reported length " + reported.getNumBytes() + " does not match " + "length in block map " + storedBlock.getNumBytes(), @@ -2392,8 +2571,8 @@ private BlockToMarkCorrupt checkReplicaCorrupt( case UNDER_CONSTRUCTION: if (storedBlock.getGenerationStamp() > reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(storedBlock, reportedGS, "block is " - + ucState + " and reported state " + reportedState + 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 " + storedBlock.getGenerationStamp(), Reason.GENSTAMP_MISMATCH); @@ -2408,7 +2587,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( return null; // not corrupt } else if (storedBlock.getGenerationStamp() != reported.getGenerationStamp()) { final long reportedGS = reported.getGenerationStamp(); - return new BlockToMarkCorrupt(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); @@ -2423,7 +2602,7 @@ private BlockToMarkCorrupt checkReplicaCorrupt( "complete with the same genstamp"); return null; } else { - return new BlockToMarkCorrupt(storedBlock, + return new BlockToMarkCorrupt(new Block(reported), storedBlock, "reported replica has invalid state " + reportedState, Reason.INVALID_STATE); } @@ -2436,7 +2615,8 @@ private BlockToMarkCorrupt checkReplicaCorrupt( " on " + dn + " size " + storedBlock.getNumBytes(); // log here at WARN level since this is really a broken HDFS invariant LOG.warn(msg); - return new BlockToMarkCorrupt(storedBlock, msg, Reason.INVALID_STATE); + return new BlockToMarkCorrupt(new Block(reported), storedBlock, msg, + Reason.INVALID_STATE); } } @@ -2469,7 +2649,7 @@ void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, if (ucBlock.reportedState == ReplicaState.FINALIZED && (block.findStorageInfo(storageInfo) < 0)) { - addStoredBlock(block, storageInfo, null, true); + addStoredBlock(block, ucBlock.reportedBlock, storageInfo, null, true); } } @@ -2484,39 +2664,40 @@ void addStoredBlockUnderConstruction(StatefulBlockInfo ucBlock, * * @throws IOException */ - private void addStoredBlockImmediate(BlockInfo storedBlock, + private void addStoredBlockImmediate(BlockInfo storedBlock, Block reported, DatanodeStorageInfo storageInfo) throws IOException { assert (storedBlock != null && namesystem.hasWriteLock()); - if (!namesystem.isInStartupSafeMode() + if (!namesystem.isInStartupSafeMode() || isPopulatingReplQueues()) { - addStoredBlock(storedBlock, storageInfo, null, false); + addStoredBlock(storedBlock, reported, storageInfo, null, false); return; } // just add it - AddBlockResult result = storageInfo.addBlock(storedBlock); + AddBlockResult result = storageInfo.addBlock(storedBlock, reported); // Now check for completion of blocks and safe block count int numCurrentReplica = countLiveNodes(storedBlock); if (storedBlock.getBlockUCState() == BlockUCState.COMMITTED - && numCurrentReplica >= minReplication) { + && hasMinStorage(storedBlock, numCurrentReplica)) { completeBlock(storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block // 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, DatanodeStorageInfo storageInfo, DatanodeDescriptor delNodeHint, boolean logEveryBlock) @@ -2526,7 +2707,7 @@ private Block addStoredBlock(final BlockInfo block, DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); if (!block.isComplete()) { //refresh our copy in case the block got completed in another thread - storedBlock = blocksMap.getStoredBlock(block); + storedBlock = getStoredBlock(block); } else { storedBlock = block; } @@ -2543,7 +2724,7 @@ private Block addStoredBlock(final BlockInfo block, assert bc != null : "Block must belong to a file"; // add block to the datanode - AddBlockResult result = storageInfo.addBlock(storedBlock); + AddBlockResult result = storageInfo.addBlock(storedBlock, reportedBlock); int curReplicaDelta; if (result == AddBlockResult.ADDED) { @@ -2574,7 +2755,7 @@ private Block addStoredBlock(final BlockInfo block, + pendingReplications.getNumReplicas(storedBlock); if(storedBlock.getBlockUCState() == BlockUCState.COMMITTED && - numLiveReplicas >= minReplication) { + hasMinStorage(storedBlock, numLiveReplicas)) { completeBlock(storedBlock, false); } else if (storedBlock.isComplete() && result == AddBlockResult.ADDED) { // check whether safe replication is reached for the block @@ -2582,7 +2763,7 @@ private Block addStoredBlock(final BlockInfo block, // 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 @@ -2612,11 +2793,12 @@ private Block addStoredBlock(final BlockInfo block, 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)) - invalidateCorruptReplicas(storedBlock); + if ((corruptReplicasCount > 0) && (numLiveReplicas >= fileReplication)) { + invalidateCorruptReplicas(storedBlock, reportedBlock, num); + } return storedBlock; } @@ -2648,18 +2830,20 @@ private void logAddStoredBlock(BlockInfo storedBlock, * * @param blk Block whose corrupt replicas need to be invalidated */ - private void invalidateCorruptReplicas(BlockInfo blk) { + private void invalidateCorruptReplicas(BlockInfo blk, Block reported, + NumberReplicas numberReplicas) { Collection nodes = corruptReplicas.getNodes(blk); boolean removedFromBlocksMap = true; if (nodes == null) return; // make a copy of the array of nodes in order to avoid // ConcurrentModificationException, when the block is removed from the node - DatanodeDescriptor[] nodesCopy = nodes.toArray(new DatanodeDescriptor[0]); + DatanodeDescriptor[] nodesCopy = + nodes.toArray(new DatanodeDescriptor[nodes.size()]); for (DatanodeDescriptor node : nodesCopy) { try { - if (!invalidateBlock(new BlockToMarkCorrupt(blk, null, - Reason.ANY), node)) { + if (!invalidateBlock(new BlockToMarkCorrupt(reported, blk, null, + Reason.ANY), node, numberReplicas)) { removedFromBlocksMap = false; } } catch (IOException e) { @@ -2809,6 +2993,15 @@ public double getReplicationQueuesInitProgress() { return replicationQueuesInitProgress; } + /** + * Get the value of whether there are any non-EC blocks using StripedID. + * + * @return Returns the value of whether there are any non-EC blocks using StripedID. + */ + public boolean hasNonEcBlockUsingStripedID(){ + return hasNonEcBlockUsingStripedID; + } + /** * Process a single possibly misreplicated block. This adds it to the * appropriate queues if necessary, and returns a result code indicating @@ -2876,14 +3069,14 @@ public void setReplication( * 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 nonExcess = new ArrayList(); + Collection nonExcess = new ArrayList<>(); Collection corruptNodes = corruptReplicas .getNodes(block); for(DatanodeStorageInfo storage : blocksMap.getStorages(block, State.NORMAL)) { @@ -2897,8 +3090,8 @@ private void processOverReplicatedBlock(final Block block, postponeBlock(block); return; } - LightWeightLinkedSet excessBlocks = excessReplicateMap.get(cur - .getDatanodeUuid()); + LightWeightLinkedSet excessBlocks = excessReplicateMap.get( + cur.getDatanodeUuid()); if (excessBlocks == null || !excessBlocks.contains(block)) { if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { // exclude corrupt replicas @@ -2908,10 +3101,29 @@ private void processOverReplicatedBlock(final Block block, } } } - chooseExcessReplicates(nonExcess, block, replication, - addedNode, delNodeHint, blockplacement); + chooseExcessReplicates(nonExcess, block, replication, addedNode, + delNodeHint); } + private void chooseExcessReplicates( + final Collection nonExcess, + BlockInfo storedBlock, short replication, + DatanodeDescriptor addedNode, + DatanodeDescriptor delNodeHint) { + assert namesystem.hasWriteLock(); + // first form a rack to datanodes map and + BlockCollection bc = getBlockCollection(storedBlock); + if (storedBlock.isStriped()) { + chooseExcessReplicasStriped(bc, nonExcess, storedBlock, delNodeHint); + } else { + final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( + bc.getStoragePolicyID()); + final List excessTypes = storagePolicy.chooseExcess( + replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); + chooseExcessReplicasContiguous(bc, nonExcess, storedBlock, + replication, addedNode, delNodeHint, excessTypes); + } + } /** * We want "replication" replicates for the block, but we now have too many. @@ -2927,24 +3139,16 @@ private void processOverReplicatedBlock(final Block block, * If no such a node is available, * then pick a node with least free space */ - private void chooseExcessReplicates(final Collection nonExcess, - Block b, short replication, - DatanodeDescriptor addedNode, - DatanodeDescriptor delNodeHint, - BlockPlacementPolicy replicator) { - assert namesystem.hasWriteLock(); - // first form a rack to datanodes map and - BlockInfo bi = getStoredBlock(b); - BlockCollection bc = getBlockCollection(bi); - final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy(bc.getStoragePolicyID()); - final List excessTypes = storagePolicy.chooseExcess( - replication, DatanodeStorageInfo.toStorageTypes(nonExcess)); - - - final Map> rackMap - = new HashMap>(); - final List moreThanOne = new ArrayList(); - final List exactlyOne = new ArrayList(); + private void chooseExcessReplicasContiguous(BlockCollection bc, + final Collection nonExcess, + BlockInfo storedBlock, short replication, + DatanodeDescriptor addedNode, + DatanodeDescriptor delNodeHint, + List excessTypes) { + BlockPlacementPolicy replicator = placementPolicies.getPolicy(false); + final Map> rackMap = new HashMap<>(); + final List moreThanOne = new ArrayList<>(); + final List exactlyOne = new ArrayList<>(); // split nodes into two sets // moreThanOne contains nodes on rack with more than one replica @@ -2965,33 +3169,111 @@ private void chooseExcessReplicates(final Collection nonExc 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; - // adjust rackmap, moreThanOne, and exactlyOne replicator.adjustSetsWithChosenReplica(rackMap, moreThanOne, exactlyOne, cur); - nonExcess.remove(cur); - addToExcessReplicate(cur.getDatanodeDescriptor(), b); + processChosenExcessReplica(nonExcess, cur, storedBlock); + } + } - // - // The 'excessblocks' tracks blocks until we get confirmation - // that the datanode has deleted them; the only way we remove them - // is when we get a "removeBlock" message. - // - // 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. - // - addToInvalidates(b, cur.getDatanodeDescriptor()); - blockLog.debug("BLOCK* chooseExcessReplicates: " - +"({}, {}) is added to invalidated blocks set", cur, b); + /** + * We want block group has every internal block, but we have redundant + * internal blocks (which have the same index). + * In this method, we delete the redundant internal blocks until only one + * left for each index. + * + * The block placement policy will make sure that the left internal blocks are + * spread across racks and also try hard to pick one with least free space. + */ + private void chooseExcessReplicasStriped(BlockCollection bc, + final Collection nonExcess, + BlockInfo storedBlock, + DatanodeDescriptor delNodeHint) { + assert storedBlock instanceof BlockInfoStriped; + BlockInfoStriped sblk = (BlockInfoStriped) storedBlock; + short groupSize = sblk.getTotalBlockNum(); + BlockPlacementPolicy placementPolicy = placementPolicies.getPolicy(true); + List empty = new ArrayList<>(0); + + // find all duplicated indices + BitSet found = new BitSet(groupSize); //indices found + BitSet duplicated = new BitSet(groupSize); //indices found more than once + HashMap storage2index = new HashMap<>(); + for (DatanodeStorageInfo storage : nonExcess) { + int index = sblk.getStorageBlockIndex(storage); + assert index >= 0; + if (found.get(index)) { + duplicated.set(index); + } + found.set(index); + storage2index.put(storage, index); + } + // the number of target left replicas equals to the of number of the found + // indices. + int numOfTarget = found.cardinality(); + + final BlockStoragePolicy storagePolicy = storagePolicySuite.getPolicy( + bc.getStoragePolicyID()); + final List excessTypes = storagePolicy.chooseExcess( + (short)numOfTarget, DatanodeStorageInfo.toStorageTypes(nonExcess)); + + // use delHint only if delHint is duplicated + final DatanodeStorageInfo delStorageHint = + DatanodeStorageInfo.getDatanodeStorageInfo(nonExcess, delNodeHint); + if (delStorageHint != null) { + Integer index = storage2index.get(delStorageHint); + if (index != null && duplicated.get(index)) { + processChosenExcessReplica(nonExcess, delStorageHint, storedBlock); + } + } + + // for each duplicated index, delete some replicas until only one left + for (int targetIndex = duplicated.nextSetBit(0); targetIndex >= 0; + targetIndex = duplicated.nextSetBit(targetIndex + 1)) { + List candidates = new ArrayList<>(); + for (DatanodeStorageInfo storage : nonExcess) { + int index = storage2index.get(storage); + if (index == targetIndex) { + candidates.add(storage); + } + } + Block internalBlock = new Block(storedBlock); + internalBlock.setBlockId(storedBlock.getBlockId() + targetIndex); + while (candidates.size() > 1) { + DatanodeStorageInfo target = placementPolicy.chooseReplicaToDelete(bc, + internalBlock, (short)1, candidates, empty, excessTypes); + processChosenExcessReplica(nonExcess, target, storedBlock); + candidates.remove(target); + } + duplicated.clear(targetIndex); } } + private void processChosenExcessReplica( + final Collection nonExcess, + final DatanodeStorageInfo chosen, BlockInfo storedBlock) { + nonExcess.remove(chosen); + addToExcessReplicate(chosen.getDatanodeDescriptor(), storedBlock); + // + // The 'excessblocks' tracks blocks until we get confirmation + // that the datanode has deleted them; the only way we remove them + // is when we get a "removeBlock" message. + // + // 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 datanodes. + // + final Block blockToInvalidate = getBlockOnStorage(storedBlock, chosen); + addToInvalidates(blockToInvalidate, chosen.getDatanodeDescriptor()); + blockLog.debug("BLOCK* chooseExcessReplicates: " + + "({}, {}) is added to invalidated blocks set", chosen, storedBlock); + } + /** Check if we can use delHint */ static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint, DatanodeStorageInfo added, List moreThan1Racks, @@ -3013,17 +3295,18 @@ static boolean useDelHint(boolean isFirst, DatanodeStorageInfo delHint, } } - private void addToExcessReplicate(DatanodeInfo dn, Block block) { + private void addToExcessReplicate(DatanodeInfo dn, BlockInfo storedBlock) { assert namesystem.hasWriteLock(); - LightWeightLinkedSet excessBlocks = excessReplicateMap.get(dn.getDatanodeUuid()); + LightWeightLinkedSet excessBlocks = excessReplicateMap.get( + dn.getDatanodeUuid()); if (excessBlocks == null) { - excessBlocks = new LightWeightLinkedSet(); + 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); } } @@ -3035,26 +3318,25 @@ private void removeStoredBlock(DatanodeStorageInfo storageInfo, Block block, QUEUE_REASON_FUTURE_GENSTAMP); return; } - removeStoredBlock(block, node); + removeStoredBlock(getStoredBlock(block), node); } /** * 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; } CachedBlock cblock = namesystem.getCacheManager().getCachedBlocks() - .get(new CachedBlock(block.getBlockId(), (short) 0, false)); + .get(new CachedBlock(storedBlock.getBlockId(), (short) 0, false)); if (cblock != null) { boolean removed = false; removed |= node.getPendingCached().remove(cblock); @@ -3062,7 +3344,7 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { removed |= node.getPendingUncached().remove(cblock); if (removed) { blockLog.debug("BLOCK* removeStoredBlock: {} removed from caching " - + "related lists on node {}", block, node); + + "related lists on node {}", storedBlock, node); } } @@ -3082,13 +3364,13 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { // We've removed a block from a node, so it's definitely no longer // in "excess" there. // - LightWeightLinkedSet excessBlocks = excessReplicateMap.get(node - .getDatanodeUuid()); + LightWeightLinkedSet 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()); } @@ -3096,15 +3378,16 @@ public void removeStoredBlock(Block block, DatanodeDescriptor node) { } // Remove the replica from corruptReplicas - corruptReplicas.removeFromCorruptReplicasMap(block, node); + corruptReplicas.removeFromCorruptReplicasMap(storedBlock, node); } } /** * Get all valid locations of the block & add the block to results - * return the length of the added block; 0 if the block is not added + * @return the length of the added block; 0 if the block is not added. If the + * added block is a block group, return its approximate internal block size */ - private long addBlock(Block block, List results) { + private long addBlock(BlockInfo block, List results) { final List locations = getValidLocations(block); if(locations.size() == 0) { return 0; @@ -3118,9 +3401,23 @@ private long addBlock(Block block, List results) { storageIDs[i] = s.getStorageID(); storageTypes[i] = s.getStorageType(); } - results.add(new BlockWithLocations(block, datanodeUuids, storageIDs, - storageTypes)); - return block.getNumBytes(); + BlockWithLocations blkWithLocs = new BlockWithLocations(block, + datanodeUuids, storageIDs, storageTypes); + if(block.isStriped()) { + BlockInfoStriped blockStriped = (BlockInfoStriped) block; + byte[] indices = new byte[locations.size()]; + for (int i = 0; i < locations.size(); i++) { + indices[i] = + (byte) blockStriped.getStorageBlockIndex(locations.get(i)); + } + results.add(new StripedBlockWithLocations(blkWithLocs, indices, + blockStriped.getDataBlockNum(), blockStriped.getCellSize())); + // approximate size + return block.getNumBytes() / blockStriped.getDataBlockNum(); + }else{ + results.add(blkWithLocs); + return block.getNumBytes(); + } } } @@ -3151,7 +3448,7 @@ void addBlock(DatanodeStorageInfo storageInfo, Block block, String delHint) // BlockInfo storedBlock = getStoredBlock(block); if (storedBlock != null) { - pendingReplications.decrement(getStoredBlock(block), node); + pendingReplications.decrement(storedBlock, node); } processAndHandleReportedBlock(storageInfo, block, ReplicaState.FINALIZED, delHintNode); @@ -3162,14 +3459,14 @@ private void processAndHandleReportedBlock( ReplicaState reportedState, DatanodeDescriptor delHintNode) throws IOException { // blockReceived reports a finalized block - Collection toAdd = new LinkedList(); + Collection toAdd = new LinkedList<>(); Collection toInvalidate = new LinkedList(); Collection toCorrupt = new LinkedList(); Collection toUC = new LinkedList(); final DatanodeDescriptor node = storageInfo.getDatanodeDescriptor(); - processReportedBlock(storageInfo, block, reportedState, - toAdd, toInvalidate, toCorrupt, toUC); + processReportedBlock(storageInfo, block, reportedState, toAdd, toInvalidate, + toCorrupt, toUC); // the block is only in one of the to-do lists // if it is in none then data-node already has it assert toUC.size() + toAdd.size() + toInvalidate.size() + toCorrupt.size() <= 1 @@ -3179,8 +3476,9 @@ private void processAndHandleReportedBlock( addStoredBlockUnderConstruction(b, storageInfo); } long numBlocksLogged = 0; - for (BlockInfo b : toAdd) { - addStoredBlock(b, storageInfo, delHintNode, numBlocksLogged < maxNumBlocksToLog); + for (BlockInfoToAdd b : toAdd) { + addStoredBlock(b.stored, b.reported, storageInfo, delHintNode, + numBlocksLogged < maxNumBlocksToLog); numBlocksLogged++; } if (numBlocksLogged > maxNumBlocksToLog) { @@ -3263,6 +3561,8 @@ public void processIncrementalBlockReport(final DatanodeID nodeID, /** * 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(BlockInfo b) { int decommissioned = 0; @@ -3281,8 +3581,8 @@ public NumberReplicas countNodes(BlockInfo b) { } else if (node.isDecommissioned()) { decommissioned++; } else { - LightWeightLinkedSet blocksExcess = excessReplicateMap.get(node - .getDatanodeUuid()); + LightWeightLinkedSet blocksExcess = excessReplicateMap.get( + node.getDatanodeUuid()); if (blocksExcess != null && blocksExcess.contains(b)) { excess++; } else { @@ -3335,12 +3635,13 @@ void processOverReplicatedBlocksOnReCommission( int numOverReplicated = 0; while(it.hasNext()) { final BlockInfo block = it.next(); - short expectedReplication = block.getReplication(); + 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++; } } @@ -3366,7 +3667,7 @@ boolean isNodeHealthyForDecommission(DatanodeDescriptor node) { if (pendingReplicationBlocksCount == 0 && underReplicatedBlocksCount == 0) { LOG.info("Node {} is dead and there are no under-replicated" + - " blocks or blocks pending replication. Safe to decommission.", + " blocks or blocks pending replication. Safe to decommission.", node); return true; } @@ -3393,6 +3694,11 @@ public DatanodeStorageInfo[] getStorages(BlockInfo block) { return storages; } + /** @return an iterator of the datanodes. */ + public Iterable getStorages(final Block block) { + return blocksMap.getStorages(block); + } + public int getTotalBlocks() { return blocksMap.size(); } @@ -3414,7 +3720,19 @@ public void removeBlock(BlockInfo block) { } public BlockInfo getStoredBlock(Block block) { - return blocksMap.getStoredBlock(block); + if (!BlockIdManager.isStripedBlockID(block.getBlockId())) { + return blocksMap.getStoredBlock(block); + } + if (!hasNonEcBlockUsingStripedID) { + return blocksMap.getStoredBlock( + new Block(BlockIdManager.convertToStripedID(block.getBlockId()))); + } + BlockInfo info = blocksMap.getStoredBlock(block); + if (info != null) { + return info; + } + return blocksMap.getStoredBlock( + new Block(BlockIdManager.convertToStripedID(block.getBlockId()))); } /** updates a block in under replication queue */ @@ -3450,7 +3768,7 @@ private void updateNeededReplications(final BlockInfo block, */ public void checkReplication(BlockCollection bc) { for (BlockInfo block : bc.getBlocks()) { - final short expected = block.getReplication(); + short expected = getExpectedReplicaNum(block); final NumberReplicas n = countNodes(block); if (isNeededReplication(block, n.liveReplicas())) { neededReplications.add(block, n.liveReplicas(), @@ -3470,10 +3788,11 @@ public boolean checkBlocksProperlyReplicated( for (BlockInfo b: blocks) { if (!b.isComplete()) { final int numNodes = b.numNodes(); - LOG.info("BLOCK* " + b + " is not COMPLETE (ucState = " - + b.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; } } @@ -3528,21 +3847,65 @@ private int invalidateWorkForOneNode(DatanodeInfo dn) { return toInvalidate.size(); } - boolean blockHasEnoughRacks(BlockInfo b) { + @VisibleForTesting + public boolean containsInvalidateBlock(final DatanodeInfo dn, + final Block block) { + return invalidateBlocks.contains(dn, block); + } + + boolean blockHasEnoughRacks(BlockInfo storedBlock, int expectedStorageNum) { if (!this.shouldCheckForEnoughRacks) { return true; } - boolean enoughRacks = false;; - Collection corruptNodes = - corruptReplicas.getNodes(b); - int numExpectedReplicas = getReplication(b); + Collection corruptNodes = + corruptReplicas.getNodes(storedBlock); + + if (storedBlock.isStriped()) { + return blockHasEnoughRacksStriped(storedBlock, corruptNodes); + } else { + return blockHashEnoughRacksContiguous(storedBlock, expectedStorageNum, + corruptNodes); + } + } + + /** + * Verify whether given striped block is distributed through enough racks. + * As dicussed in HDFS-7613, ec file requires racks at least as many as + * the number of data block number. + */ + boolean blockHasEnoughRacksStriped(BlockInfo storedBlock, + Collection corruptNodes) { + if (!datanodeManager.hasClusterEverBeenMultiRack()) { + return true; + } + boolean enoughRacks = false; + Set rackNameSet = new HashSet<>(); + int dataBlockNum = ((BlockInfoStriped)storedBlock).getRealDataBlockNum(); + for (DatanodeStorageInfo storage : blocksMap.getStorages(storedBlock)) { + final DatanodeDescriptor cur = storage.getDatanodeDescriptor(); + if (!cur.isDecommissionInProgress() && !cur.isDecommissioned()) { + if ((corruptNodes == null) || !corruptNodes.contains(cur)) { + String rackNameNew = cur.getNetworkLocation(); + rackNameSet.add(rackNameNew); + if (rackNameSet.size() >= dataBlockNum) { + enoughRacks = true; + break; + } + } + } + } + return enoughRacks; + } + + boolean blockHashEnoughRacksContiguous(BlockInfo storedBlock, + int expectedStorageNum, Collection corruptNodes) { + boolean enoughRacks = false; 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; @@ -3565,14 +3928,16 @@ boolean blockHasEnoughRacks(BlockInfo b) { * or if it does not have enough racks. */ boolean isNeededReplication(BlockInfo storedBlock, int current) { - int expected = storedBlock.getReplication(); - return current < expected || !blockHasEnoughRacks(storedBlock); + int expected = getExpectedReplicaNum(storedBlock); + return current < expected || !blockHasEnoughRacks(storedBlock, expected); } public short getExpectedReplicaNum(BlockInfo block) { - return block.getReplication(); + return block.isStriped() ? + ((BlockInfoStriped) block).getRealTotalBlockNum() : + block.getReplication(); } - + public long getMissingBlocksCount() { // not locking return this.neededReplications.getCorruptBlockSize(); @@ -3588,13 +3953,22 @@ public BlockInfo addBlockCollection(BlockInfo block, return blocksMap.addBlockCollection(block, bc); } - public BlockCollection getBlockCollection(BlockInfo b) { - return namesystem.getBlockCollection(b.getBlockCollectionId()); + /** + * Do some check when adding a block to blocksmap. + * For HDFS-7994 to check whether then block is a NonEcBlockUsingStripedID. + * + */ + public BlockInfo addBlockCollectionWithCheck( + BlockInfo block, BlockCollection bc) { + if (!hasNonEcBlockUsingStripedID && !block.isStriped() && + BlockIdManager.isStripedBlockID(block.getBlockId())) { + hasNonEcBlockUsingStripedID = true; + } + return addBlockCollection(block, bc); } - /** @return an iterator of the datanodes. */ - public Iterable getStorages(final Block block) { - return blocksMap.getStorages(block); + public BlockCollection getBlockCollection(BlockInfo b) { + return namesystem.getBlockCollection(b.getBlockCollectionId()); } public int numCorruptReplicas(Block block) { @@ -3614,7 +3988,7 @@ public void removeBlockFromMap(Block block) { private void removeFromExcessReplicateMap(Block block) { for (DatanodeStorageInfo info : blocksMap.getStorages(block)) { String uuid = info.getDatanodeDescriptor().getDatanodeUuid(); - LightWeightLinkedSet excessReplicas = excessReplicateMap.get(uuid); + LightWeightLinkedSet excessReplicas = excessReplicateMap.get(uuid); if (excessReplicas != null) { if (excessReplicas.remove(block)) { excessBlocksCount.decrementAndGet(); @@ -3658,7 +4032,7 @@ public int numOfUnderReplicatedBlocks() { } /** - * Periodically calls computeReplicationWork(). + * Periodically calls computeBlockRecoveryWork(). */ private class ReplicationMonitor implements Runnable { @@ -3716,7 +4090,7 @@ int computeDatanodeWork() { final int nodesToProcess = (int) Math.ceil(numlive * this.blocksInvalidateWorkPct); - int workFound = this.computeReplicationWork(blocksToProcess); + int workFound = this.computeBlockRecoveryWork(blocksToProcess); // Update counters namesystem.writeLock(); @@ -3756,6 +4130,31 @@ public static LocatedBlock newLocatedBlock( null); } + public static LocatedStripedBlock newLocatedStripedBlock( + ExtendedBlock b, DatanodeStorageInfo[] storages, + int[] indices, long startOffset, boolean corrupt) { + // startOffset is unknown + return new LocatedStripedBlock( + b, DatanodeStorageInfo.toDatanodeInfos(storages), + DatanodeStorageInfo.toStorageIDs(storages), + DatanodeStorageInfo.toStorageTypes(storages), + indices, startOffset, corrupt, + null); + } + + public static LocatedBlock newLocatedBlock(ExtendedBlock eb, BlockInfo info, + DatanodeStorageInfo[] locs, long offset) throws IOException { + final LocatedBlock lb; + if (info.isStriped()) { + lb = newLocatedStripedBlock(eb, locs, + info.getUnderConstructionFeature().getBlockIndices(), + offset, false); + } else { + lb = newLocatedBlock(eb, locs, offset, false); + } + return lb; + } + /** * A simple result enum for the result of * {@link BlockManager#processMisReplicatedBlock(BlockInfo)}. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.java new file mode 100644 index 0000000000000..622b2581fdfae --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicies.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.hdfs.server.blockmanagement; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.net.NetworkTopology; +import org.apache.hadoop.util.ReflectionUtils; + +public class BlockPlacementPolicies{ + + private final BlockPlacementPolicy replicationPolicy; + private final BlockPlacementPolicy ecPolicy; + + public BlockPlacementPolicies(Configuration conf, FSClusterStats stats, + NetworkTopology clusterMap, + Host2NodesMap host2datanodeMap){ + final Class replicatorClass = conf + .getClass(DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, + DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT, + BlockPlacementPolicy.class); + replicationPolicy = ReflectionUtils.newInstance(replicatorClass, conf); + replicationPolicy.initialize(conf, stats, clusterMap, host2datanodeMap); + final Class blockPlacementECClass = + conf.getClass(DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_KEY, + DFSConfigKeys.DFS_BLOCK_PLACEMENT_EC_CLASSNAME_DEFAULT, + BlockPlacementPolicy.class); + ecPolicy = ReflectionUtils.newInstance(blockPlacementECClass, conf); + ecPolicy.initialize(conf, stats, clusterMap, host2datanodeMap); + } + + public BlockPlacementPolicy getPolicy(boolean isStriped){ + if (isStriped) { + return ecPolicy; + } else { + return replicationPolicy; + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java index 9696179185ed4..86aaf79cedcdc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockPlacementPolicy.java @@ -145,31 +145,7 @@ abstract public DatanodeStorageInfo chooseReplicaToDelete( abstract protected void initialize(Configuration conf, FSClusterStats stats, NetworkTopology clusterMap, Host2NodesMap host2datanodeMap); - - /** - * Get an instance of the configured Block Placement Policy based on the - * the configuration property - * {@link DFSConfigKeys#DFS_BLOCK_REPLICATOR_CLASSNAME_KEY}. - * - * @param conf the configuration to be used - * @param stats an object that is used to retrieve the load on the cluster - * @param clusterMap the network topology of the cluster - * @return an instance of BlockPlacementPolicy - */ - public static BlockPlacementPolicy getInstance(Configuration conf, - FSClusterStats stats, - NetworkTopology clusterMap, - Host2NodesMap host2datanodeMap) { - final Class replicatorClass = conf.getClass( - DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_KEY, - DFSConfigKeys.DFS_BLOCK_REPLICATOR_CLASSNAME_DEFAULT, - BlockPlacementPolicy.class); - final BlockPlacementPolicy replicator = ReflectionUtils.newInstance( - replicatorClass, conf); - replicator.initialize(conf, stats, clusterMap, host2datanodeMap); - return replicator; - } - + /** * Adjust rackmap, moreThanOne, and exactlyOne after removing replica on cur. * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java new file mode 100644 index 0000000000000..ed546dfc5f87f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockRecoveryWork.java @@ -0,0 +1,111 @@ +/** + * 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.net.Node; + +import java.util.Collections; +import java.util.List; +import java.util.Set; + +/** + * This class is used internally by + * {@link BlockManager#computeRecoveryWorkForBlocks} to represent a task to + * recover a block through replication or erasure coding. Recovery is done by + * transferring data from srcNodes to targets + */ +abstract class BlockRecoveryWork { + private final BlockInfo block; + + private 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 + */ + private final DatanodeDescriptor[] srcNodes; + /** Nodes containing the block; avoid them in choosing new targets */ + private final List containingNodes; + /** Required by {@link BlockPlacementPolicy#chooseTarget} */ + private final List liveReplicaStorages; + private final int additionalReplRequired; + + private DatanodeStorageInfo[] targets; + private final int priority; + + public BlockRecoveryWork(BlockInfo block, + BlockCollection bc, + DatanodeDescriptor[] srcNodes, + List containingNodes, + List liveReplicaStorages, + int additionalReplRequired, + int priority) { + this.block = block; + this.bc = bc; + this.srcNodes = srcNodes; + this.containingNodes = containingNodes; + this.liveReplicaStorages = liveReplicaStorages; + this.additionalReplRequired = additionalReplRequired; + this.priority = priority; + this.targets = null; + } + + DatanodeStorageInfo[] getTargets() { + return targets; + } + + void resetTargets() { + this.targets = null; + } + + void setTargets(DatanodeStorageInfo[] targets) { + this.targets = targets; + } + + List getContainingNodes() { + return Collections.unmodifiableList(containingNodes); + } + + public int getPriority() { + return priority; + } + + public BlockInfo getBlock() { + return block; + } + + public DatanodeDescriptor[] getSrcNodes() { + return srcNodes; + } + + BlockCollection getBc() { + return bc; + } + + List getLiveReplicaStorages() { + return liveReplicaStorages; + } + + public int getAdditionalReplRequired() { + return additionalReplRequired; + } + + abstract void chooseTargets(BlockPlacementPolicy blockplacement, + BlockStoragePolicySuite storagePolicySuite, + Set excludedNodes); +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java index 3842e562ac3bb..a871390ae2274 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockToMarkCorrupt.java @@ -20,6 +20,7 @@ import static org.apache.hadoop.hdfs.server.blockmanagement.CorruptReplicasMap.Reason; import com.google.common.base.Preconditions; +import org.apache.hadoop.hdfs.protocol.Block; /** * BlockToMarkCorrupt is used to build the "toCorrupt" list, which is a @@ -27,7 +28,7 @@ */ class BlockToMarkCorrupt { /** The corrupted block in a datanode. */ - private final BlockInfo corrupted; + private final Block corrupted; /** The corresponding block stored in the BlockManager. */ private final BlockInfo stored; /** The reason to mark corrupt. */ @@ -35,7 +36,7 @@ class BlockToMarkCorrupt { /** The reason code to be stored */ private final CorruptReplicasMap.Reason reasonCode; - BlockToMarkCorrupt(BlockInfo corrupted, BlockInfo stored, String reason, + BlockToMarkCorrupt(Block corrupted, BlockInfo stored, String reason, CorruptReplicasMap.Reason reasonCode) { Preconditions.checkNotNull(corrupted, "corrupted is null"); Preconditions.checkNotNull(stored, "stored is null"); @@ -46,24 +47,18 @@ class BlockToMarkCorrupt { this.reasonCode = reasonCode; } - BlockToMarkCorrupt(BlockInfo stored, String reason, + BlockToMarkCorrupt(Block corrupted, BlockInfo stored, long gs, String reason, CorruptReplicasMap.Reason reasonCode) { - this(stored, stored, reason, reasonCode); - } - - BlockToMarkCorrupt(BlockInfo stored, long gs, String reason, - CorruptReplicasMap.Reason reasonCode) { - this(new BlockInfoContiguous((BlockInfoContiguous)stored), stored, - reason, reasonCode); + this(corrupted, stored, reason, reasonCode); //the corrupted block in datanode has a different generation stamp - corrupted.setGenerationStamp(gs); + this.corrupted.setGenerationStamp(gs); } public boolean isCorruptedDuringWrite() { return stored.getGenerationStamp() > corrupted.getGenerationStamp(); } - public BlockInfo getCorrupted() { + public Block getCorrupted() { return corrupted; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java index 88cf06dbcbf1e..1d4cff3340ea6 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/BlockUnderConstructionFeature.java @@ -58,20 +58,38 @@ public class BlockUnderConstructionFeature { private Block truncateBlock; public BlockUnderConstructionFeature(Block blk, - BlockUCState state, DatanodeStorageInfo[] targets) { + BlockUCState state, DatanodeStorageInfo[] targets, boolean isStriped) { assert getBlockUCState() != COMPLETE : "BlockUnderConstructionFeature cannot be in COMPLETE state"; this.blockUCState = state; - setExpectedLocations(blk, targets); + setExpectedLocations(blk, targets, isStriped); } /** Set expected locations */ - public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets) { - int numLocations = targets == null ? 0 : targets.length; + public void setExpectedLocations(Block block, DatanodeStorageInfo[] targets, + boolean isStriped) { + if (targets == null) { + return; + } + int numLocations = 0; + for (DatanodeStorageInfo target : targets) { + if (target != null) { + numLocations++; + } + } + this.replicas = new ReplicaUnderConstruction[numLocations]; - for(int i = 0; i < numLocations; i++) { - replicas[i] = new ReplicaUnderConstruction(block, targets[i], - ReplicaState.RBW); + int offset = 0; + for(int i = 0; i < targets.length; i++) { + if (targets[i] != null) { + // when creating a new striped block we simply sequentially assign block + // index to each storage + Block replicaBlock = isStriped ? + new Block(block.getBlockId() + i, 0, block.getGenerationStamp()) : + block; + replicas[offset++] = new ReplicaUnderConstruction(replicaBlock, + targets[i], ReplicaState.RBW); + } } } @@ -88,7 +106,19 @@ public DatanodeStorageInfo[] getExpectedStorageLocations() { return storages; } - /** Get the number of expected locations */ + /** + * @return the index array indicating the block index in each storage. Used + * only by striped blocks. + */ + 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; + } + public int getNumExpectedLocations() { return replicas == null ? 0 : replicas.length; } 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 9189c3261adae..f41b093fcdcbc 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 @@ -44,8 +44,15 @@ private static class StorageIterator implements Iterator { @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 @@ -119,13 +126,17 @@ void removeBlock(Block block) { return; blockInfo.setBlockCollectionId(INodeId.INVALID_INODE_ID); - for(int idx = blockInfo.numNodes()-1; idx >= 0; idx--) { + final int size = blockInfo.isStriped() ? + blockInfo.getCapacity() : blockInfo.numNodes(); + 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 + } } } - - /** 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); } @@ -186,8 +197,8 @@ boolean removeNode(Block b, DatanodeDescriptor node) { // remove block from the data-node list and the node from the block info boolean removed = node.removeBlock(info); - if (info.getDatanode(0) == null // no datanodes left - && info.isDeleted()) { // does not belong to a file + if (info.hasNoStorage() // no datanodes left + && info.isDeleted()) { // does not belong to a file blocks.remove(b); // remove block from the map } return removed; 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 0b398c51cad5f..b258f0600e2ef 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 @@ -39,8 +39,10 @@ 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.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.CachedBlock; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockReportContext; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; @@ -48,6 +50,7 @@ import org.apache.hadoop.hdfs.server.protocol.VolumeFailureSummary; import org.apache.hadoop.hdfs.util.EnumCounters; import org.apache.hadoop.hdfs.util.LightWeightHashSet; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.util.IntrusiveCollection; import org.apache.hadoop.util.Time; import org.slf4j.Logger; @@ -223,6 +226,9 @@ public CachedBlocksList getPendingUncached() { /** A queue of blocks to be replicated by this datanode */ private final BlockQueue replicateBlocks = new BlockQueue<>(); + /** A queue of blocks to be erasure coded by this datanode */ + private final BlockQueue erasurecodeBlocks = + new BlockQueue<>(); /** A queue of blocks to be recovered by this datanode */ private final BlockQueue recoverBlocks = new BlockQueue<>(); /** A set of blocks to be invalidated by this datanode */ @@ -282,7 +288,8 @@ public DatanodeStorageInfo getStorageInfo(String storageID) { } } - DatanodeStorageInfo[] getStorageInfos() { + @VisibleForTesting + public DatanodeStorageInfo[] getStorageInfos() { synchronized (storageMap) { final Collection storages = storageMap.values(); return storages.toArray(new DatanodeStorageInfo[storages.size()]); @@ -379,6 +386,7 @@ public void clearBlockQueues() { this.invalidateBlocks.clear(); this.recoverBlocks.clear(); this.replicateBlocks.clear(); + this.erasurecodeBlocks.clear(); } // pendingCached, cached, and pendingUncached are protected by the // FSN lock. @@ -580,6 +588,7 @@ private void update() { Iterator getBlockIterator() { return new BlockIterator(getStorageInfos()); } + Iterator getBlockIterator(final String storageID) { return new BlockIterator(getStorageInfo(storageID)); } @@ -600,6 +609,20 @@ void addBlockToBeReplicated(Block block, DatanodeStorageInfo[] targets) { replicateBlocks.offer(new BlockTargetPair(block, targets)); } + /** + * Store block erasure coding work. + */ + void addBlockToBeErasureCoded(ExtendedBlock block, + DatanodeDescriptor[] sources, DatanodeStorageInfo[] targets, + short[] liveBlockIndices, ErasureCodingPolicy ecPolicy) { + assert (block != null && sources != null && sources.length > 0); + BlockECRecoveryInfo task = new BlockECRecoveryInfo(block, sources, targets, + liveBlockIndices, ecPolicy); + erasurecodeBlocks.offer(task); + BlockManager.LOG.debug("Adding block recovery task " + task + "to " + + getName() + ", current queue size is " + erasurecodeBlocks.size()); + } + /** * Store block recovery work. */ @@ -631,6 +654,14 @@ int getNumberOfBlocksToBeReplicated() { return PendingReplicationWithoutTargets + replicateBlocks.size(); } + /** + * The number of work items that are pending to be replicated + */ + @VisibleForTesting + public int getNumberOfBlocksToBeErasureCoded() { + return erasurecodeBlocks.size(); + } + /** * The number of block invalidation items that are pending to * be sent to the datanode @@ -645,6 +676,10 @@ public List getReplicationCommand(int maxTransfers) { return replicateBlocks.poll(maxTransfers); } + public List getErasureCodeCommand(int maxTransfers) { + return erasurecodeBlocks.poll(maxTransfers); + } + public BlockInfo[] getLeaseRecoveryCommand(int maxTransfers) { List blocks = recoverBlocks.poll(maxTransfers); if(blocks == null) @@ -663,6 +698,13 @@ public Block[] getInvalidateBlocks(int maxblocks) { } } + @VisibleForTesting + public boolean containsInvalidateBlock(Block block) { + synchronized (invalidateBlocks) { + return invalidateBlocks.contains(block); + } + } + /** * Find whether the datanode contains good storage of given type to * place block of size blockSize. @@ -868,6 +910,10 @@ public String dumpDatanode() { 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;"); 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 a484fccf9c0a7..28ab7161c4896 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 @@ -39,6 +39,7 @@ import org.apache.hadoop.hdfs.server.namenode.NameNode; import org.apache.hadoop.hdfs.server.namenode.Namesystem; import org.apache.hadoop.hdfs.server.protocol.*; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.ipc.Server; import org.apache.hadoop.net.*; @@ -512,6 +513,10 @@ public DatanodeStorageInfo[] getDatanodeStorageInfos( } final DatanodeStorageInfo[] storages = new DatanodeStorageInfo[datanodeID.length]; for(int i = 0; i < datanodeID.length; i++) { + if (datanodeID[i].equals(DatanodeID.EMPTY_DATANODE_ID)) { + storages[i] = null; + continue; + } final DatanodeDescriptor dd = getDatanode(datanodeID[i]); storages[i] = dd.getStorageInfo(storageIDs[i]); } @@ -1365,9 +1370,9 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, // Skip stale nodes during recovery - not heart beated for some time (30s by default). final List recoveryLocations = new ArrayList<>(storages.length); - for (int i = 0; i < storages.length; i++) { - if (!storages[i].getDatanodeDescriptor().isStale(staleInterval)) { - recoveryLocations.add(storages[i]); + for (DatanodeStorageInfo storage : storages) { + if (!storage.getDatanodeDescriptor().isStale(staleInterval)) { + recoveryLocations.add(storage); } } // If we are performing a truncate recovery than set recovery fields @@ -1414,6 +1419,13 @@ public DatanodeCommand[] handleHeartbeat(DatanodeRegistration nodeReg, cmds.add(new BlockCommand(DatanodeProtocol.DNA_TRANSFER, blockPoolId, pendingList)); } + // checking pending erasure coding tasks + List pendingECList = + nodeinfo.getErasureCodeCommand(maxTransfers); + if (pendingECList != null) { + cmds.add(new BlockECRecoveryCommand(DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, + pendingECList)); + } //check block invalidation Block[] blks = nodeinfo.getInvalidateBlocks(blockInvalidateLimit); if (blks != null) { 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 216d6d2bf14a9..bb9a706d32da6 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 @@ -21,14 +21,15 @@ import java.util.Iterator; import java.util.List; -import com.google.common.annotations.VisibleForTesting; - 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.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage.State; import org.apache.hadoop.hdfs.server.protocol.StorageReport; +import com.google.common.annotations.VisibleForTesting; + /** * A Datanode has one or more storages. A storage in the Datanode is represented * by this class. @@ -209,6 +210,7 @@ boolean areBlocksOnFailedStorage() { return getState() == State.FAILED && numBlocks != 0; } + @VisibleForTesting public String getStorageID() { return storageID; } @@ -233,7 +235,7 @@ long getBlockPoolUsed() { return blockPoolUsed; } - public AddBlockResult addBlock(BlockInfo b) { + public AddBlockResult addBlock(BlockInfo b, Block reportedBlock) { // First check whether the block belongs to a different storage // on the same DN. AddBlockResult result = AddBlockResult.ADDED; @@ -252,10 +254,18 @@ public AddBlockResult addBlock(BlockInfo b) { } // add to the head of the data-node list - b.addStorage(this); + b.addStorage(this, reportedBlock); + insertToList(b); + return result; + } + + AddBlockResult addBlock(BlockInfo b) { + return addBlock(b, b); + } + + public void insertToList(BlockInfo b) { blockList = b.listInsert(blockList, this); numBlocks++; - return result; } public boolean removeBlock(BlockInfo b) { @@ -274,7 +284,6 @@ int numBlocks() { Iterator getBlockIterator() { return new BlockIterator(blockList); - } /** 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 6d199d7b4dab8..fb86ff3e9f939 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 @@ -234,14 +234,14 @@ private void setDecommissioned(DatanodeDescriptor dn) { } /** - * 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(BlockInfo block, - BlockCollection bc, + private boolean isSufficient(BlockInfo block, BlockCollection bc, NumberReplicas numberReplicas) { - final int numExpected = block.getReplication(); + final int numExpected = blockManager.getExpectedReplicaNum(block); final int numLive = numberReplicas.liveReplicas(); if (!blockManager.isNeededReplication(block, numLive)) { // Block doesn't need replication. Skip. @@ -255,18 +255,19 @@ private boolean isSufficientlyReplicated(BlockInfo block, 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; } } @@ -274,12 +275,12 @@ private boolean isSufficientlyReplicated(BlockInfo block, return false; } - private static void logBlockReplicationInfo(BlockInfo block, + private void logBlockReplicationInfo(BlockInfo block, BlockCollection bc, DatanodeDescriptor srcNode, NumberReplicas num, Iterable storages) { int curReplicas = num.liveReplicas(); - int curExpectedReplicas = block.getReplication(); + int curExpectedReplicas = blockManager.getExpectedReplicaNum(block); StringBuilder nodeList = new StringBuilder(); for (DatanodeStorageInfo storage : storages) { final DatanodeDescriptor node = storage.getDatanodeDescriptor(); @@ -408,14 +409,14 @@ private void check() { // 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) { @@ -427,7 +428,7 @@ private void check() { // 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, @@ -468,25 +469,23 @@ private void check() { } /** - * 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, + private void pruneReliableBlocks(final DatanodeDescriptor datanode, AbstractList 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. *

- * 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 handleInsufficientlyReplicated( + private AbstractList handleInsufficientlyStored( final DatanodeDescriptor datanode) { AbstractList insufficient = new ChunkedArrayList<>(); processBlocksForDecomInternal(datanode, datanode.getBlockIterator(), @@ -497,24 +496,22 @@ private AbstractList handleInsufficientlyReplicated( /** * 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 it, - final List insufficientlyReplicated, - boolean pruneSufficientlyReplicated) { + final List insufficientList, + boolean pruneReliableBlocks) { boolean firstReplicationLog = true; int underReplicatedBlocks = 0; int decommissionOnlyReplicas = 0; @@ -539,7 +536,6 @@ private void processBlocksForDecomInternal( BlockCollection bc = namesystem.getBlockCollection(bcId); 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 @@ -549,24 +545,24 @@ private void processBlocksForDecomInternal( blockManager.isPopulatingReplQueues()) { // Process these blocks only when active NN is out of safe mode. blockManager.neededReplications.add(block, - curReplicas, + liveReplicas, num.decommissionedAndDecommissioning(), - block.getReplication()); + blockManager.getExpectedReplicaNum(block)); } } // Even if the block is under-replicated, - // it doesn't block decommission if it's sufficiently replicated - if (isSufficientlyReplicated(block, bc, num)) { - if (pruneSufficientlyReplicated) { + // it doesn't block decommission if it's sufficiently replicated + if (isSufficient(block, bc, num)) { + if (pruneReliableBlocks) { it.remove(); } continue; } // We've found an insufficiently replicated block. - if (insufficientlyReplicated != null) { - insufficientlyReplicated.add(block); + if (insufficientList != null) { + insufficientList.add(block); } // Log if this is our first time through if (firstReplicationLog) { @@ -579,7 +575,7 @@ private void processBlocksForDecomInternal( if (bc.isUnderConstruction()) { underReplicatedInOpenFiles++; } - if ((curReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) { + if ((liveReplicas == 0) && (num.decommissionedAndDecommissioning() > 0)) { decommissionOnlyReplicas++; } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java new file mode 100644 index 0000000000000..761d6d063cc16 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ErasureCodingWork.java @@ -0,0 +1,60 @@ +/** + * 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.net.Node; + +import java.util.List; +import java.util.Set; + +class ErasureCodingWork extends BlockRecoveryWork { + private final short[] liveBlockIndicies; + + public ErasureCodingWork(BlockInfo block, + BlockCollection bc, + DatanodeDescriptor[] srcNodes, + List containingNodes, + List liveReplicaStorages, + int additionalReplRequired, + int priority, short[] liveBlockIndicies) { + super(block, bc, srcNodes, containingNodes, + liveReplicaStorages, additionalReplRequired, priority); + this.liveBlockIndicies = liveBlockIndicies; + BlockManager.LOG.debug("Creating an ErasureCodingWork to recover " + block); + } + + short[] getLiveBlockIndicies() { + return liveBlockIndicies; + } + + @Override + void chooseTargets(BlockPlacementPolicy blockplacement, + BlockStoragePolicySuite storagePolicySuite, + Set excludedNodes) { + try { + // TODO: new placement policy for EC considering multiple writers + DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget( + getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0], + getLiveReplicaStorages(), false, excludedNodes, + getBlock().getNumBytes(), + storagePolicySuite.getPolicy(getBc().getStoragePolicyID())); + setTargets(chosenTargets); + } finally { + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java index f8a6dad1d1d2b..8266f453737a2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/ReplicationWork.java @@ -23,65 +23,31 @@ import java.util.List; import java.util.Set; -class ReplicationWork { - private final BlockInfo block; - private final BlockCollection bc; - private final DatanodeDescriptor srcNode; - private final int additionalReplRequired; - private final int priority; - private final List containingNodes; - private final List liveReplicaStorages; - private DatanodeStorageInfo[] targets; - +class ReplicationWork extends BlockRecoveryWork { public ReplicationWork(BlockInfo block, BlockCollection bc, - DatanodeDescriptor srcNode, List containingNodes, + DatanodeDescriptor[] srcNodes, List containingNodes, List liveReplicaStorages, int additionalReplRequired, int priority) { - this.block = block; - this.bc = bc; - this.srcNode = srcNode; - this.srcNode.incrementPendingReplicationWithoutTargets(); - this.containingNodes = containingNodes; - this.liveReplicaStorages = liveReplicaStorages; - this.additionalReplRequired = additionalReplRequired; - this.priority = priority; - this.targets = null; + super(block, bc, srcNodes, containingNodes, + liveReplicaStorages, additionalReplRequired, priority); + BlockManager.LOG.debug("Creating a ReplicationWork to recover " + block); } + @Override void chooseTargets(BlockPlacementPolicy blockplacement, BlockStoragePolicySuite storagePolicySuite, Set excludedNodes) { + assert getSrcNodes().length > 0 + : "At least 1 source node should have been selected"; try { - targets = blockplacement.chooseTarget(bc.getName(), - additionalReplRequired, srcNode, liveReplicaStorages, false, - excludedNodes, block.getNumBytes(), - storagePolicySuite.getPolicy(bc.getStoragePolicyID())); + DatanodeStorageInfo[] chosenTargets = blockplacement.chooseTarget( + getBc().getName(), getAdditionalReplRequired(), getSrcNodes()[0], + getLiveReplicaStorages(), false, excludedNodes, + getBlock().getNumBytes(), + storagePolicySuite.getPolicy(getBc().getStoragePolicyID())); + setTargets(chosenTargets); } finally { - srcNode.decrementPendingReplicationWithoutTargets(); + getSrcNodes()[0].decrementPendingReplicationWithoutTargets(); } } - - DatanodeStorageInfo[] getTargets() { - return targets; - } - - void resetTargets() { - this.targets = null; - } - - List getContainingNodes() { - return Collections.unmodifiableList(containingNodes); - } - - public int getPriority() { - return priority; - } - - public BlockInfo getBlock() { - return block; - } - - public DatanodeDescriptor getSrcNode() { - return srcNode; - } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java new file mode 100644 index 0000000000000..7a522730e50f1 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockGroupIdGenerator.java @@ -0,0 +1,86 @@ +/** + * 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.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.util.SequentialNumber; + +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_BLOCKS_IN_GROUP; + +/** + * Generate the next valid block group ID by incrementing the maximum block + * group ID allocated so far, with the first 2^10 block group IDs reserved. + * HDFS-EC introduces a hierarchical protocol to name blocks and groups: + * Contiguous: {reserved block IDs | flag | block ID} + * Striped: {reserved block IDs | flag | block group ID | index in group} + * + * Following n bits of reserved block IDs, The (n+1)th bit in an ID + * distinguishes contiguous (0) and striped (1) blocks. For a striped block, + * bits (n+2) to (64-m) represent the ID of its block group, while the last m + * bits represent its index of the group. The value m is determined by the + * maximum number of blocks in a group (MAX_BLOCKS_IN_GROUP). + * + * Note that the {@link #nextValue()} methods requires external lock to + * guarantee IDs have no conflicts. + */ +@InterfaceAudience.Private +public class SequentialBlockGroupIdGenerator extends SequentialNumber { + + private final BlockManager blockManager; + + SequentialBlockGroupIdGenerator(BlockManager blockManagerRef) { + super(Long.MIN_VALUE); + this.blockManager = blockManagerRef; + } + + @Override // NumberGenerator + public long nextValue() { + skipTo((getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + MAX_BLOCKS_IN_GROUP); + // Make sure there's no conflict with existing random block IDs + final Block b = new Block(getCurrentValue()); + while (hasValidBlockInRange(b)) { + skipTo(getCurrentValue() + MAX_BLOCKS_IN_GROUP); + b.setBlockId(getCurrentValue()); + } + if (b.getBlockId() >= 0) { + throw new IllegalStateException("All negative block group IDs are used, " + + "growing into positive IDs, " + + "which might conflict with non-erasure coded blocks."); + } + return getCurrentValue(); + } + + /** + * @param b A block object whose id is set to the starting point for check + * @return true if any ID in the range + * {id, id+HdfsConstants.MAX_BLOCKS_IN_GROUP} is pointed-to by a stored + * block. + */ + private boolean hasValidBlockInRange(Block b) { + final long id = b.getBlockId(); + for (int i = 0; i < MAX_BLOCKS_IN_GROUP; i++) { + b.setBlockId(id + i); + if (blockManager.getStoredBlock(b) != null) { + return true; + } + } + return false; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java index f053b7b541888..631b43538bd41 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/blockmanagement/SequentialBlockIdGenerator.java @@ -19,7 +19,6 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.namenode.INodeId; import org.apache.hadoop.util.SequentialNumber; @@ -55,6 +54,11 @@ public long nextValue() { while(isValidBlock(b)) { b.setBlockId(super.nextValue()); } + if (b.getBlockId() < 0) { + throw new IllegalStateException("All positive block IDs are used, " + + "wrapping to negative IDs, " + + "which might conflict with erasure coded block groups."); + } return b.getBlockId(); } 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 ebc15b871c958..7e8f4797873b2 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 @@ -34,7 +34,7 @@ * *

* The policy for choosing which priority to give added blocks - * is implemented in {@link #getPriority(int, int, int)}. + * is implemented in {@link #getPriority(BlockInfo, int, int, int)}. *

*

The queue order is as follows:

*
    @@ -145,14 +145,28 @@ synchronized boolean contains(BlockInfo 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(int curReplicas, + private int getPriority(BlockInfo block, + int curReplicas, int decommissionedReplicas, int expectedReplicas) { assert curReplicas >= 0 : "Negative replicas!"; if (curReplicas >= expectedReplicas) { // Block has enough copies, but not enough racks return QUEUE_REPLICAS_BADLY_DISTRIBUTED; - } else if (curReplicas == 0) { + } + if (block.isStriped()) { + BlockInfoStriped sblk = (BlockInfoStriped) block; + return getPriorityStriped(curReplicas, decommissionedReplicas, + sblk.getRealDataBlockNum(), sblk.getParityBlockNum()); + } else { + return getPriorityContiguous(curReplicas, decommissionedReplicas, + expectedReplicas); + } + } + + private int getPriorityContiguous(int curReplicas, int decommissionedReplicas, + int expectedReplicas) { + if (curReplicas == 0) { // If there are zero non-decommissioned replicas but there are // some decommissioned replicas, then assign them highest priority if (decommissionedReplicas > 0) { @@ -161,7 +175,7 @@ private int getPriority(int curReplicas, //all we have are corrupt blocks return QUEUE_WITH_CORRUPT_BLOCKS; } else if (curReplicas == 1) { - //only on replica -risk of loss + // only one replica, highest risk of loss // highest priority return QUEUE_HIGHEST_PRIORITY; } else if ((curReplicas * 3) < expectedReplicas) { @@ -174,6 +188,27 @@ private int getPriority(int curReplicas, } } + private int getPriorityStriped(int curReplicas, int decommissionedReplicas, + short dataBlkNum, short parityBlkNum) { + if (curReplicas < dataBlkNum) { + // There are some replicas on decommissioned nodes so it's not corrupted + if (curReplicas + decommissionedReplicas >= dataBlkNum) { + return QUEUE_HIGHEST_PRIORITY; + } + return QUEUE_WITH_CORRUPT_BLOCKS; + } else if (curReplicas == dataBlkNum) { + // highest risk of loss, highest priority + return QUEUE_HIGHEST_PRIORITY; + } else if ((curReplicas - dataBlkNum) * 3 < parityBlkNum + 1) { + // can only afford one replica loss + // this is considered very under-replicated + return QUEUE_VERY_UNDER_REPLICATED; + } else { + // add to the normal queue for under replicated blocks + return QUEUE_UNDER_REPLICATED; + } + } + /** add a block to a under replication queue according to its priority * @param block a under replication block * @param curReplicas current number of replicas of the block @@ -186,7 +221,7 @@ synchronized boolean add(BlockInfo block, int decomissionedReplicas, int expectedReplicas) { assert curReplicas >= 0 : "Negative replicas!"; - int priLevel = getPriority(curReplicas, decomissionedReplicas, + int priLevel = getPriority(block, curReplicas, decomissionedReplicas, expectedReplicas); if(priorityQueues.get(priLevel).add(block)) { if (priLevel == QUEUE_WITH_CORRUPT_BLOCKS && @@ -209,7 +244,7 @@ synchronized boolean remove(BlockInfo block, int oldReplicas, int decommissionedReplicas, int oldExpectedReplicas) { - int priLevel = getPriority(oldReplicas, + int priLevel = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas); boolean removedBlock = remove(block, priLevel); @@ -283,9 +318,9 @@ synchronized void update(BlockInfo block, int curReplicas, int curReplicasDelta, int expectedReplicasDelta) { int oldReplicas = curReplicas-curReplicasDelta; int oldExpectedReplicas = curExpectedReplicas-expectedReplicasDelta; - int curPri = getPriority(curReplicas, decommissionedReplicas, + int curPri = getPriority(block, curReplicas, decommissionedReplicas, curExpectedReplicas); - int oldPri = getPriority(oldReplicas, decommissionedReplicas, + int oldPri = getPriority(block, oldReplicas, decommissionedReplicas, oldExpectedReplicas); if(NameNode.stateChangeLog.isDebugEnabled()) { NameNode.stateChangeLog.debug("UnderReplicationBlocks.update " + 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 c4c3d8abeed21..6208a7dceceb1 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 @@ -368,4 +368,9 @@ enum BlockUCState { "raw.hdfs.crypto.file.encryption.info"; String SECURITY_XATTR_UNREADABLE_BY_SUPERUSER = "security.hdfs.unreadable.by.superuser"; + String XATTR_ERASURECODING_POLICY = + "raw.hdfs.erasurecoding.policy"; + + long BLOCK_GROUP_INDEX_MASK = 15; + byte MAX_BLOCKS_IN_GROUP = 16; } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java index 7aab4f71ae149..96e74e5dff320 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/BPOfferService.java @@ -32,11 +32,13 @@ import org.apache.hadoop.hdfs.protocol.RollingUpgradeStatus; import org.apache.hadoop.hdfs.protocolPB.DatanodeProtocolClientSideTranslatorPB; import org.apache.hadoop.hdfs.server.protocol.*; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; import java.io.IOException; import java.net.InetSocketAddress; import java.util.ArrayList; +import java.util.Collection; import java.util.List; import java.util.Set; import java.util.concurrent.CopyOnWriteArrayList; @@ -721,6 +723,11 @@ assert getBlockPoolId().equals(bp) : dxcs.balanceThrottler.setBandwidth(bandwidth); } break; + case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY: + LOG.info("DatanodeCommand action: DNA_ERASURE_CODING_RECOVERY"); + Collection ecTasks = ((BlockECRecoveryCommand) cmd).getECTasks(); + dn.getErasureCodingWorker().processErasureCodingTasks(ecTasks); + break; default: LOG.warn("Unknown DatanodeCommand action: " + cmd.getAction()); } @@ -750,6 +757,7 @@ private boolean processCommandFromStandby(DatanodeCommand cmd, case DatanodeProtocol.DNA_BALANCERBANDWIDTHUPDATE: case DatanodeProtocol.DNA_CACHE: case DatanodeProtocol.DNA_UNCACHE: + case DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY: LOG.warn("Got a command from standby NN - ignoring command:" + cmd.getAction()); break; default: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java index bd4943d026fa7..b3cb48b595f92 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DNConf.java @@ -253,6 +253,33 @@ public long getMaxLockedMemory() { return maxLockedMemory; } + /** + * Returns true if connect to datanode via hostname + * + * @return boolean true if connect to datanode via hostname + */ + public boolean getConnectToDnViaHostname() { + return connectToDnViaHostname; + } + + /** + * Returns socket timeout + * + * @return int socket timeout + */ + public int getSocketTimeout() { + return socketTimeout; + } + + /** + * Returns socket write timeout + * + * @return int socket write timeout + */ + public int getSocketWriteTimeout() { + return socketWriteTimeout; + } + /** * Returns the SaslPropertiesResolver configured for use with * DataTransferProtocol, or null if not configured. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java index 264608936b625..43ddf74620cfd 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/DataNode.java @@ -94,6 +94,7 @@ import javax.management.ObjectName; + import org.apache.commons.logging.Log; import org.apache.commons.logging.LogFactory; import org.apache.hadoop.classification.InterfaceAudience; @@ -158,6 +159,7 @@ import org.apache.hadoop.hdfs.server.common.Storage; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.datanode.SecureDataNodeStarter.SecureResources; +import org.apache.hadoop.hdfs.server.datanode.erasurecode.ErasureCodingWorker; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsDatasetSpi; import org.apache.hadoop.hdfs.server.datanode.fsdataset.FsVolumeSpi; import org.apache.hadoop.hdfs.server.datanode.metrics.DataNodeMetrics; @@ -368,7 +370,7 @@ public static InetSocketAddress createSocketAddr(String target) { private String supergroup; private boolean isPermissionEnabled; private String dnUserName = null; - + private ErasureCodingWorker ecWorker; final Tracer tracer; private final TracerConfigurationManager tracerConfigurationManager; private static final int NUM_CORES = Runtime.getRuntime() @@ -1209,6 +1211,8 @@ void startDataNode(Configuration conf, saslClient = new SaslDataTransferClient(dnConf.conf, dnConf.saslPropsResolver, dnConf.trustedChannelResolver); saslServer = new SaslDataTransferServer(dnConf, blockPoolTokenSecretManager); + // Initialize ErasureCoding worker + ecWorker = new ErasureCodingWorker(conf, this); startMetricsLogger(conf); } @@ -1274,6 +1278,10 @@ public static String generateUuid() { return UUID.randomUUID().toString(); } + public SaslDataTransferClient getSaslClient() { + return saslClient; + } + /** * Verify that the DatanodeUuid has been initialized. If this is a new * datanode then we generate a new Datanode Uuid and persist it to disk. @@ -1533,7 +1541,7 @@ public DatanodeRegistration getDNRegistrationForBP(String bpid) /** * Creates either NIO or regular depending on socketWriteTimeout. */ - protected Socket newSocket() throws IOException { + public Socket newSocket() throws IOException { return (dnConf.socketWriteTimeout > 0) ? SocketChannel.open().socket() : new Socket(); } @@ -1918,6 +1926,21 @@ void incrDatanodeNetworkErrors(String host) { int getXmitsInProgress() { return xmitsInProgress.get(); } + + /** + * Increments the xmitsInProgress count. xmitsInProgress count represents the + * number of data replication/reconstruction tasks running currently. + */ + public void incrementXmitsInProgress() { + xmitsInProgress.getAndIncrement(); + } + + /** + * Decrements the xmitsInProgress count + */ + public void decrementXmitsInProgress() { + xmitsInProgress.getAndDecrement(); + } private void reportBadBlock(final BPOfferService bpos, final ExtendedBlock block, final String msg) { @@ -2137,7 +2160,7 @@ private class DataTransfer implements Runnable { */ @Override public void run() { - xmitsInProgress.getAndIncrement(); + incrementXmitsInProgress(); Socket sock = null; DataOutputStream out = null; DataInputStream in = null; @@ -2157,11 +2180,8 @@ public void run() { // // Header info // - Token accessToken = BlockTokenSecretManager.DUMMY_TOKEN; - if (isBlockTokenEnabled) { - accessToken = blockPoolTokenSecretManager.generateToken(b, - EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)); - } + Token accessToken = getBlockAccessToken(b, + EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)); long writeTimeout = dnConf.socketWriteTimeout + HdfsConstants.WRITE_TIMEOUT_EXTENSION * (targets.length-1); @@ -2219,7 +2239,7 @@ public void run() { // check if there are any disk problem checkDiskErrorAsync(); } finally { - xmitsInProgress.getAndDecrement(); + decrementXmitsInProgress(); IOUtils.closeStream(blockSender); IOUtils.closeStream(out); IOUtils.closeStream(in); @@ -2228,6 +2248,19 @@ public void run() { } } + /*** + * Use BlockTokenSecretManager to generate block token for current user. + */ + public Token getBlockAccessToken(ExtendedBlock b, + EnumSet mode) throws IOException { + Token accessToken = + BlockTokenSecretManager.DUMMY_TOKEN; + if (isBlockTokenEnabled) { + accessToken = blockPoolTokenSecretManager.generateToken(b, mode); + } + return accessToken; + } + /** * Returns a new DataEncryptionKeyFactory that generates a key from the * BlockPoolTokenSecretManager, using the block pool ID of the given block. @@ -2235,7 +2268,7 @@ public void run() { * @param block for which the factory needs to create a key * @return DataEncryptionKeyFactory for block's block pool ID */ - DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock( + public DataEncryptionKeyFactory getDataEncryptionKeyFactoryForBlock( final ExtendedBlock block) { return new DataEncryptionKeyFactory() { @Override @@ -3289,6 +3322,10 @@ public void removeSpanReceiver(long id) throws IOException { checkSuperuserPrivilege(); tracerConfigurationManager.removeSpanReceiver(id); } + + public ErasureCodingWorker getErasureCodingWorker(){ + return ecWorker; + } /** * Get timeout value of each OOB type from configuration diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java index 5c8dd85b2cabe..78734595ae869 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/StorageLocation.java @@ -54,7 +54,7 @@ private StorageLocation(StorageType storageType, URI uri) { // drop any (illegal) authority in the URI for backwards compatibility this.file = new File(uri.getPath()); } else { - throw new IllegalArgumentException("Unsupported URI schema in " + uri); + throw new IllegalArgumentException("Unsupported URI ecPolicy in " + uri); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java new file mode 100644 index 0000000000000..7c64b37d44d91 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/datanode/erasurecode/ErasureCodingWorker.java @@ -0,0 +1,1016 @@ +/** + * 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.datanode.erasurecode; + +import java.io.BufferedOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.io.InputStream; +import java.io.OutputStream; +import java.net.InetSocketAddress; +import java.net.Socket; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.BitSet; +import java.util.Collection; +import java.util.EnumSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.Callable; +import java.util.concurrent.CompletionService; +import java.util.concurrent.ExecutorCompletionService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.SynchronousQueue; +import java.util.concurrent.ThreadPoolExecutor; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.logging.Log; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.BlockReader; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.DFSPacket; +import org.apache.hadoop.hdfs.DFSUtil; +import org.apache.hadoop.hdfs.DFSUtilClient; +import org.apache.hadoop.hdfs.RemoteBlockReader2; +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.datatransfer.BlockConstructionStage; +import org.apache.hadoop.hdfs.protocol.datatransfer.IOStreamPair; +import org.apache.hadoop.hdfs.protocol.datatransfer.PacketHeader; +import org.apache.hadoop.hdfs.protocol.datatransfer.Sender; +import org.apache.hadoop.hdfs.protocol.datatransfer.sasl.DataEncryptionKeyFactory; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.server.datanode.CachingStrategy; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.hdfs.util.StripedBlockUtil.StripingChunkReadResult; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.apache.hadoop.net.NetUtils; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.util.Daemon; +import org.apache.hadoop.util.DataChecksum; + +import com.google.common.base.Preconditions; + +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.convertIndex4Decode; + +/** + * ErasureCodingWorker handles the erasure coding recovery work commands. These + * commands would be issued from Namenode as part of Datanode's heart beat + * response. BPOfferService delegates the work to this class for handling EC + * commands. + */ +public final class ErasureCodingWorker { + private static final Log LOG = DataNode.LOG; + + private final DataNode datanode; + private final Configuration conf; + + private ThreadPoolExecutor STRIPED_BLK_RECOVERY_THREAD_POOL; + private ThreadPoolExecutor STRIPED_READ_THREAD_POOL; + private final int STRIPED_READ_TIMEOUT_MILLIS; + private final int STRIPED_READ_BUFFER_SIZE; + + public ErasureCodingWorker(Configuration conf, DataNode datanode) { + this.datanode = datanode; + this.conf = conf; + + STRIPED_READ_TIMEOUT_MILLIS = conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_TIMEOUT_MILLIS_DEFAULT); + initializeStripedReadThreadPool(conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_THREADS_DEFAULT)); + STRIPED_READ_BUFFER_SIZE = conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_DEFAULT); + + initializeStripedBlkRecoveryThreadPool(conf.getInt( + DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_KEY, + DFSConfigKeys.DFS_DATANODE_STRIPED_BLK_RECOVERY_THREADS_DEFAULT)); + } + + private RawErasureDecoder newDecoder(int numDataUnits, int numParityUnits) { + return CodecUtil.createRSRawDecoder(conf, numDataUnits, numParityUnits); + } + + private void initializeStripedReadThreadPool(int num) { + if (LOG.isDebugEnabled()) { + LOG.debug("Using striped reads; pool threads=" + num); + } + STRIPED_READ_THREAD_POOL = new ThreadPoolExecutor(1, num, 60, + TimeUnit.SECONDS, new SynchronousQueue(), + 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); + } + + private void initializeStripedBlkRecoveryThreadPool(int num) { + if (LOG.isDebugEnabled()) { + LOG.debug("Using striped block recovery; pool threads=" + num); + } + STRIPED_BLK_RECOVERY_THREAD_POOL = new ThreadPoolExecutor(2, num, 60, + TimeUnit.SECONDS, new LinkedBlockingQueue(), + new Daemon.DaemonFactory() { + private final AtomicInteger threadIdx = new AtomicInteger(0); + + @Override + public Thread newThread(Runnable r) { + Thread t = super.newThread(r); + t.setName("stripedBlockRecovery-" + threadIdx.getAndIncrement()); + return t; + } + }); + STRIPED_BLK_RECOVERY_THREAD_POOL.allowCoreThreadTimeOut(true); + } + + /** + * Handles the Erasure Coding recovery work commands. + * + * @param ecTasks + * BlockECRecoveryInfo + */ + public void processErasureCodingTasks(Collection ecTasks) { + for (BlockECRecoveryInfo recoveryInfo : ecTasks) { + try { + STRIPED_BLK_RECOVERY_THREAD_POOL + .submit(new ReconstructAndTransferBlock(recoveryInfo)); + } catch (Throwable e) { + LOG.warn("Failed to recover striped block " + + recoveryInfo.getExtendedBlock().getLocalBlock(), e); + } + } + } + + /** + * ReconstructAndTransferBlock recover one or more missed striped block in the + * striped block group, the minimum number of live striped blocks should be + * no less than data block number. + * + * | <- Striped Block Group -> | + * blk_0 blk_1 blk_2(*) blk_3 ... <- A striped block group + * | | | | + * v v v v + * +------+ +------+ +------+ +------+ + * |cell_0| |cell_1| |cell_2| |cell_3| ... + * +------+ +------+ +------+ +------+ + * |cell_4| |cell_5| |cell_6| |cell_7| ... + * +------+ +------+ +------+ +------+ + * |cell_8| |cell_9| |cell10| |cell11| ... + * +------+ +------+ +------+ +------+ + * ... ... ... ... + * + * + * We use following steps to recover striped block group, in each round, we + * recover bufferSize data until finish, the + * bufferSize is configurable and may be less or larger than + * cell size: + * step1: read bufferSize data from minimum number of sources + * required by recovery. + * step2: decode data for targets. + * step3: transfer data to targets. + * + * In step1, try to read bufferSize data from minimum number + * of sources , if there is corrupt or stale sources, read from new source + * will be scheduled. The best sources are remembered for next round and + * may be updated in each round. + * + * In step2, typically if source blocks we read are all data blocks, we + * need to call encode, and if there is one parity block, we need to call + * decode. Notice we only read once and recover all missed striped block + * if they are more than one. + * + * In step3, send the recovered data to targets by constructing packet + * and send them directly. Same as continuous block replication, we + * don't check the packet ack. Since the datanode doing the recovery work + * are one of the source datanodes, so the recovered data are sent + * remotely. + * + * There are some points we can do further improvements in next phase: + * 1. we can read the block file directly on the local datanode, + * currently we use remote block reader. (Notice short-circuit is not + * a good choice, see inline comments). + * 2. We need to check the packet ack for EC recovery? Since EC recovery + * is more expensive than continuous block replication, it needs to + * read from several other datanodes, should we make sure the + * recovered result received by targets? + */ + private class ReconstructAndTransferBlock implements Runnable { + private final int dataBlkNum; + private final int parityBlkNum; + private final int cellSize; + + private RawErasureDecoder decoder; + + // Striped read buffer size + private int bufferSize; + + private final ExtendedBlock blockGroup; + private final int minRequiredSources; + // position in striped internal block + private long positionInBlock; + + // sources + private final short[] liveIndices; + private final DatanodeInfo[] sources; + + private final List stripedReaders; + + // The buffers and indices for striped blocks whose length is 0 + private ByteBuffer[] zeroStripeBuffers; + private short[] zeroStripeIndices; + + // targets + private final DatanodeInfo[] targets; + private final StorageType[] targetStorageTypes; + + private final short[] targetIndices; + private final ByteBuffer[] targetBuffers; + + private final Socket[] targetSockets; + private final DataOutputStream[] targetOutputStreams; + private final DataInputStream[] targetInputStreams; + + private final long[] blockOffset4Targets; + private final long[] seqNo4Targets; + + private final static int WRITE_PACKET_SIZE = 64 * 1024; + private DataChecksum checksum; + private int maxChunksPerPacket; + private byte[] packetBuf; + private byte[] checksumBuf; + private int bytesPerChecksum; + private int checksumSize; + + private final CachingStrategy cachingStrategy; + + private final Map, Integer> futures = new HashMap<>(); + private final CompletionService readService = + new ExecutorCompletionService<>(STRIPED_READ_THREAD_POOL); + + ReconstructAndTransferBlock(BlockECRecoveryInfo recoveryInfo) { + ErasureCodingPolicy ecPolicy = recoveryInfo.getErasureCodingPolicy(); + dataBlkNum = ecPolicy.getNumDataUnits(); + parityBlkNum = ecPolicy.getNumParityUnits(); + cellSize = ecPolicy.getCellSize(); + + blockGroup = recoveryInfo.getExtendedBlock(); + final int cellsNum = (int)((blockGroup.getNumBytes() - 1) / cellSize + 1); + minRequiredSources = Math.min(cellsNum, dataBlkNum); + + liveIndices = recoveryInfo.getLiveBlockIndices(); + sources = recoveryInfo.getSourceDnInfos(); + stripedReaders = new ArrayList<>(sources.length); + + Preconditions.checkArgument(liveIndices.length >= minRequiredSources, + "No enough live striped blocks."); + Preconditions.checkArgument(liveIndices.length == sources.length, + "liveBlockIndices and source dns should match"); + + if (minRequiredSources < dataBlkNum) { + zeroStripeBuffers = + new ByteBuffer[dataBlkNum - minRequiredSources]; + zeroStripeIndices = new short[dataBlkNum - minRequiredSources]; + } + + targets = recoveryInfo.getTargetDnInfos(); + targetStorageTypes = recoveryInfo.getTargetStorageTypes(); + targetIndices = new short[targets.length]; + targetBuffers = new ByteBuffer[targets.length]; + + Preconditions.checkArgument(targetIndices.length <= parityBlkNum, + "Too much missed striped blocks."); + + targetSockets = new Socket[targets.length]; + targetOutputStreams = new DataOutputStream[targets.length]; + targetInputStreams = new DataInputStream[targets.length]; + + blockOffset4Targets = new long[targets.length]; + seqNo4Targets = new long[targets.length]; + + for (int i = 0; i < targets.length; i++) { + blockOffset4Targets[i] = 0; + seqNo4Targets[i] = 0; + } + + getTargetIndices(); + cachingStrategy = CachingStrategy.newDefaultStrategy(); + } + + private ByteBuffer allocateBuffer(int length) { + return ByteBuffer.allocate(length); + } + + private ExtendedBlock getBlock(ExtendedBlock blockGroup, int i) { + return StripedBlockUtil.constructInternalBlock(blockGroup, cellSize, + dataBlkNum, i); + } + + private long getBlockLen(ExtendedBlock blockGroup, int i) { + return StripedBlockUtil.getInternalBlockLength(blockGroup.getNumBytes(), + cellSize, dataBlkNum, i); + } + + /** + * StripedReader is used to read from one source DN, it contains a block + * reader, buffer and striped block index. + * Only allocate StripedReader once for one source, and the StripedReader + * has the same array order with sources. Typically we only need to allocate + * minimum number (minRequiredSources) of StripedReader, and allocate + * new for new source DN if some existing DN invalid or slow. + * If some source DN is corrupt, set the corresponding blockReader to + * null and will never read from it again. + * + * @param i the array index of sources + * @param offsetInBlock offset for the internal block + * @return StripedReader + */ + private StripedReader addStripedReader(int i, long offsetInBlock) { + StripedReader reader = new StripedReader(liveIndices[i]); + stripedReaders.add(reader); + + BlockReader blockReader = newBlockReader( + getBlock(blockGroup, liveIndices[i]), offsetInBlock, sources[i]); + if (blockReader != null) { + initChecksumAndBufferSizeIfNeeded(blockReader); + reader.blockReader = blockReader; + } + reader.buffer = allocateBuffer(bufferSize); + return reader; + } + + @Override + public void run() { + datanode.incrementXmitsInProgress(); + try { + // Store the array indices of source DNs we have read successfully. + // In each iteration of read, the success list may be updated if + // some source DN is corrupted or slow. And use the updated success + // list of DNs for next iteration read. + int[] success = new int[minRequiredSources]; + + int nsuccess = 0; + for (int i = 0; + i < sources.length && nsuccess < minRequiredSources; i++) { + StripedReader reader = addStripedReader(i, 0); + if (reader.blockReader != null) { + success[nsuccess++] = i; + } + } + + if (nsuccess < minRequiredSources) { + String error = "Can't find minimum sources required by " + + "recovery, block id: " + blockGroup.getBlockId(); + throw new IOException(error); + } + + if (zeroStripeBuffers != null) { + for (int i = 0; i < zeroStripeBuffers.length; i++) { + zeroStripeBuffers[i] = allocateBuffer(bufferSize); + } + } + + for (int i = 0; i < targets.length; i++) { + targetBuffers[i] = allocateBuffer(bufferSize); + } + + checksumSize = checksum.getChecksumSize(); + int chunkSize = bytesPerChecksum + checksumSize; + maxChunksPerPacket = Math.max( + (WRITE_PACKET_SIZE - PacketHeader.PKT_MAX_HEADER_LEN)/chunkSize, 1); + int maxPacketSize = chunkSize * maxChunksPerPacket + + PacketHeader.PKT_MAX_HEADER_LEN; + + packetBuf = new byte[maxPacketSize]; + checksumBuf = new byte[checksumSize * (bufferSize / bytesPerChecksum)]; + + // targetsStatus store whether some target is success, it will record + // any failed target once, if some target failed (invalid DN or transfer + // failed), will not transfer data to it any more. + boolean[] targetsStatus = new boolean[targets.length]; + if (initTargetStreams(targetsStatus) == 0) { + String error = "All targets are failed."; + throw new IOException(error); + } + + long firstStripedBlockLength = getBlockLen(blockGroup, 0); + while (positionInBlock < firstStripedBlockLength) { + int toRead = Math.min( + bufferSize, (int)(firstStripedBlockLength - positionInBlock)); + // step1: read from minimum source DNs required for reconstruction. + // The returned success list is the source DNs we do real read from + success = readMinimumStripedData4Recovery(success); + + // step2: decode to reconstruct targets + long remaining = firstStripedBlockLength - positionInBlock; + int toRecoverLen = remaining < bufferSize ? + (int)remaining : bufferSize; + recoverTargets(success, targetsStatus, toRecoverLen); + + // step3: transfer data + if (transferData2Targets(targetsStatus) == 0) { + String error = "Transfer failed for all targets."; + throw new IOException(error); + } + + clearBuffers(); + positionInBlock += toRead; + } + + endTargetBlocks(targetsStatus); + + // Currently we don't check the acks for packets, this is similar as + // block replication. + } catch (Throwable e) { + LOG.warn("Failed to recover striped block: " + blockGroup, e); + } finally { + datanode.decrementXmitsInProgress(); + // close block readers + for (StripedReader stripedReader : stripedReaders) { + closeBlockReader(stripedReader.blockReader); + } + for (int i = 0; i < targets.length; i++) { + IOUtils.closeStream(targetOutputStreams[i]); + IOUtils.closeStream(targetInputStreams[i]); + IOUtils.closeStream(targetSockets[i]); + } + } + } + + // init checksum from block reader + private void initChecksumAndBufferSizeIfNeeded(BlockReader blockReader) { + if (checksum == null) { + checksum = blockReader.getDataChecksum(); + bytesPerChecksum = checksum.getBytesPerChecksum(); + // The bufferSize is flat to divide bytesPerChecksum + int readBufferSize = STRIPED_READ_BUFFER_SIZE; + bufferSize = readBufferSize < bytesPerChecksum ? bytesPerChecksum : + readBufferSize - readBufferSize % bytesPerChecksum; + } else { + assert blockReader.getDataChecksum().equals(checksum); + } + } + + private void getTargetIndices() { + BitSet bitset = new BitSet(dataBlkNum + parityBlkNum); + for (int i = 0; i < sources.length; i++) { + bitset.set(liveIndices[i]); + } + int m = 0; + int k = 0; + for (int i = 0; i < dataBlkNum + parityBlkNum; i++) { + if (!bitset.get(i)) { + if (getBlockLen(blockGroup, i) > 0) { + if (m < targets.length) { + targetIndices[m++] = (short)i; + } + } else { + zeroStripeIndices[k++] = (short)i; + } + } + } + } + + private long getReadLength(int index) { + long blockLen = getBlockLen(blockGroup, index); + long remaining = blockLen - positionInBlock; + return remaining > bufferSize ? bufferSize : remaining; + } + + /** + * Read from minimum source DNs required for reconstruction in the iteration. + * First try the success list which we think they are the best DNs + * If source DN is corrupt or slow, try to read some other source DN, + * and will update the success list. + * + * Remember the updated success list and return it for following + * operations and next iteration read. + * + * @param success the initial success list of source DNs we think best + * @return updated success list of source DNs we do real read + * @throws IOException + */ + private int[] readMinimumStripedData4Recovery(final int[] success) + throws IOException { + int nsuccess = 0; + int[] newSuccess = new int[minRequiredSources]; + BitSet used = new BitSet(sources.length); + /* + * Read from minimum source DNs required, the success list contains + * source DNs which we think best. + */ + for (int i = 0; i < minRequiredSources; i++) { + StripedReader reader = stripedReaders.get(success[i]); + if (getReadLength(liveIndices[success[i]]) > 0) { + Callable readCallable = readFromBlock( + reader.blockReader, reader.buffer); + Future f = readService.submit(readCallable); + futures.put(f, success[i]); + } else { + // If the read length is 0, we don't need to do real read + reader.buffer.position(0); + newSuccess[nsuccess++] = success[i]; + } + used.set(success[i]); + } + + while (!futures.isEmpty()) { + try { + StripingChunkReadResult result = + StripedBlockUtil.getNextCompletedStripedRead( + readService, futures, STRIPED_READ_TIMEOUT_MILLIS); + int resultIndex = -1; + if (result.state == StripingChunkReadResult.SUCCESSFUL) { + resultIndex = result.index; + } else if (result.state == StripingChunkReadResult.FAILED) { + // If read failed for some source DN, we should not use it anymore + // and schedule read from another source DN. + StripedReader failedReader = stripedReaders.get(result.index); + closeBlockReader(failedReader.blockReader); + failedReader.blockReader = null; + resultIndex = scheduleNewRead(used); + } else if (result.state == StripingChunkReadResult.TIMEOUT) { + // If timeout, we also schedule a new read. + resultIndex = scheduleNewRead(used); + } + if (resultIndex >= 0) { + newSuccess[nsuccess++] = resultIndex; + if (nsuccess >= minRequiredSources) { + // cancel remaining reads if we read successfully from minimum + // number of source DNs required by reconstruction. + cancelReads(futures.keySet()); + futures.clear(); + break; + } + } + } catch (InterruptedException e) { + LOG.info("Read data interrupted.", e); + break; + } + } + + if (nsuccess < minRequiredSources) { + String error = "Can't read data from minimum number of sources " + + "required by reconstruction, block id: " + blockGroup.getBlockId(); + throw new IOException(error); + } + + return newSuccess; + } + + private void paddingBufferToLen(ByteBuffer buffer, int len) { + int toPadding = len - buffer.position(); + for (int i = 0; i < toPadding; i++) { + buffer.put((byte) 0); + } + } + + // Initialize decoder + private void initDecoderIfNecessary() { + if (decoder == null) { + decoder = newDecoder(dataBlkNum, parityBlkNum); + } + } + + private int[] getErasedIndices(boolean[] targetsStatus) { + int[] result = new int[targets.length]; + int m = 0; + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + result[m++] = convertIndex4Decode(targetIndices[i], + dataBlkNum, parityBlkNum); + } + } + return Arrays.copyOf(result, m); + } + + private void recoverTargets(int[] success, boolean[] targetsStatus, + int toRecoverLen) { + initDecoderIfNecessary(); + ByteBuffer[] inputs = new ByteBuffer[dataBlkNum + parityBlkNum]; + for (int i = 0; i < success.length; i++) { + StripedReader reader = stripedReaders.get(success[i]); + ByteBuffer buffer = reader.buffer; + paddingBufferToLen(buffer, toRecoverLen); + inputs[convertIndex4Decode(reader.index, dataBlkNum, parityBlkNum)] = + (ByteBuffer)buffer.flip(); + } + if (success.length < dataBlkNum) { + for (int i = 0; i < zeroStripeBuffers.length; i++) { + ByteBuffer buffer = zeroStripeBuffers[i]; + paddingBufferToLen(buffer, toRecoverLen); + int index = convertIndex4Decode(zeroStripeIndices[i], dataBlkNum, + parityBlkNum); + inputs[index] = (ByteBuffer)buffer.flip(); + } + } + int[] erasedIndices = getErasedIndices(targetsStatus); + ByteBuffer[] outputs = new ByteBuffer[erasedIndices.length]; + int m = 0; + for (int i = 0; i < targetBuffers.length; i++) { + if (targetsStatus[i]) { + outputs[m++] = targetBuffers[i]; + outputs[i].limit(toRecoverLen); + } + } + decoder.decode(inputs, erasedIndices, outputs); + + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + long blockLen = getBlockLen(blockGroup, targetIndices[i]); + long remaining = blockLen - positionInBlock; + if (remaining < 0) { + targetBuffers[i].limit(0); + } else if (remaining < toRecoverLen) { + targetBuffers[i].limit((int)remaining); + } + } + } + } + + /** + * Schedule a read from some new source DN if some DN is corrupted + * or slow, this is called from the read iteration. + * Initially we may only have minRequiredSources number of + * StripedReader. + * If the position is at the end of target block, don't need to do + * real read, and return the array index of source DN, otherwise -1. + * + * @param used the used source DNs in this iteration. + * @return the array index of source DN if don't need to do real read. + */ + private int scheduleNewRead(BitSet used) { + StripedReader reader = null; + // step1: initially we may only have minRequiredSources + // number of StripedReader, and there may be some source DNs we never + // read before, so will try to create StripedReader for one new source DN + // and try to read from it. If found, go to step 3. + int m = stripedReaders.size(); + while (reader == null && m < sources.length) { + reader = addStripedReader(m, positionInBlock); + if (getReadLength(liveIndices[m]) > 0) { + if (reader.blockReader == null) { + reader = null; + m++; + } + } else { + used.set(m); + return m; + } + } + + // step2: if there is no new source DN we can use, try to find a source + // DN we ever read from but because some reason, e.g., slow, it + // is not in the success DN list at the begin of this iteration, so + // we have not tried it in this iteration. Now we have a chance to + // revisit it again. + for (int i = 0; reader == null && i < stripedReaders.size(); i++) { + if (!used.get(i)) { + StripedReader r = stripedReaders.get(i); + if (getReadLength(liveIndices[i]) > 0) { + closeBlockReader(r.blockReader); + r.blockReader = newBlockReader( + getBlock(blockGroup, liveIndices[i]), positionInBlock, + sources[i]); + if (r.blockReader != null) { + m = i; + reader = r; + } + } else { + used.set(i); + r.buffer.position(0); + return i; + } + } + } + + // step3: schedule if find a correct source DN and need to do real read. + if (reader != null) { + Callable readCallable = readFromBlock( + reader.blockReader, reader.buffer); + Future f = readService.submit(readCallable); + futures.put(f, m); + used.set(m); + } + + return -1; + } + + // cancel all reads. + private void cancelReads(Collection> futures) { + for (Future future : futures) { + future.cancel(true); + } + } + + private Callable readFromBlock(final BlockReader reader, + final ByteBuffer buf) { + return new Callable() { + + @Override + public Void call() throws Exception { + try { + actualReadFromBlock(reader, buf); + return null; + } catch (IOException e) { + LOG.info(e.getMessage()); + throw e; + } + } + + }; + } + + /** + * Read bytes from block + */ + private void actualReadFromBlock(BlockReader reader, ByteBuffer buf) + throws IOException { + int len = buf.remaining(); + int n = 0; + while (n < len) { + int nread = reader.read(buf); + if (nread <= 0) { + break; + } + n += nread; + } + } + + // close block reader + private void closeBlockReader(BlockReader blockReader) { + try { + if (blockReader != null) { + blockReader.close(); + } + } catch (IOException e) { + // ignore + } + } + + private InetSocketAddress getSocketAddress4Transfer(DatanodeInfo dnInfo) { + return NetUtils.createSocketAddr(dnInfo.getXferAddr( + datanode.getDnConf().getConnectToDnViaHostname())); + } + + private BlockReader newBlockReader(final ExtendedBlock block, + long offsetInBlock, DatanodeInfo dnInfo) { + if (offsetInBlock >= block.getNumBytes()) { + return null; + } + try { + InetSocketAddress dnAddr = getSocketAddress4Transfer(dnInfo); + Token blockToken = datanode.getBlockAccessToken( + block, EnumSet.of(BlockTokenIdentifier.AccessMode.READ)); + /* + * This can be further improved if the replica is local, then we can + * read directly from DN and need to check the replica is FINALIZED + * state, notice we should not use short-circuit local read which + * requires config for domain-socket in UNIX or legacy config in Windows. + * + * TODO: add proper tracer + */ + return RemoteBlockReader2.newBlockReader( + "dummy", block, blockToken, offsetInBlock, + block.getNumBytes() - offsetInBlock, true, + "", newConnectedPeer(block, dnAddr, blockToken, dnInfo), dnInfo, + null, cachingStrategy, null); + } catch (IOException e) { + return null; + } + } + + private Peer newConnectedPeer(ExtendedBlock b, InetSocketAddress addr, + Token blockToken, DatanodeID datanodeId) + throws IOException { + Peer peer = null; + boolean success = false; + Socket sock = null; + final int socketTimeout = datanode.getDnConf().getSocketTimeout(); + try { + sock = NetUtils.getDefaultSocketFactory(conf).createSocket(); + NetUtils.connect(sock, addr, socketTimeout); + peer = DFSUtilClient.peerFromSocketAndKey(datanode.getSaslClient(), + sock, datanode.getDataEncryptionKeyFactoryForBlock(b), + blockToken, datanodeId); + peer.setReadTimeout(socketTimeout); + success = true; + return peer; + } finally { + if (!success) { + IOUtils.cleanup(LOG, peer); + IOUtils.closeSocket(sock); + } + } + } + + /** + * Send data to targets + */ + private int transferData2Targets(boolean[] targetsStatus) { + int nsuccess = 0; + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + boolean success = false; + try { + ByteBuffer buffer = targetBuffers[i]; + + if (buffer.remaining() == 0) { + continue; + } + + checksum.calculateChunkedSums( + buffer.array(), 0, buffer.remaining(), checksumBuf, 0); + + int ckOff = 0; + while (buffer.remaining() > 0) { + DFSPacket packet = new DFSPacket(packetBuf, maxChunksPerPacket, + blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, false); + int maxBytesToPacket = maxChunksPerPacket * bytesPerChecksum; + int toWrite = buffer.remaining() > maxBytesToPacket ? + maxBytesToPacket : buffer.remaining(); + int ckLen = ((toWrite - 1) / bytesPerChecksum + 1) * checksumSize; + packet.writeChecksum(checksumBuf, ckOff, ckLen); + ckOff += ckLen; + packet.writeData(buffer, toWrite); + + // Send packet + packet.writeTo(targetOutputStreams[i]); + + blockOffset4Targets[i] += toWrite; + nsuccess++; + success = true; + } + } catch (IOException e) { + LOG.warn(e.getMessage()); + } + targetsStatus[i] = success; + } + } + return nsuccess; + } + + /** + * clear all buffers + */ + private void clearBuffers() { + for (StripedReader stripedReader : stripedReaders) { + if (stripedReader.buffer != null) { + stripedReader.buffer.clear(); + } + } + + if (zeroStripeBuffers != null) { + for (int i = 0; i < zeroStripeBuffers.length; i++) { + zeroStripeBuffers[i].clear(); + } + } + + for (int i = 0; i < targetBuffers.length; i++) { + if (targetBuffers[i] != null) { + cleanBuffer(targetBuffers[i]); + } + } + } + + private ByteBuffer cleanBuffer(ByteBuffer buffer) { + Arrays.fill(buffer.array(), (byte) 0); + return (ByteBuffer)buffer.clear(); + } + + // send an empty packet to mark the end of the block + private void endTargetBlocks(boolean[] targetsStatus) { + for (int i = 0; i < targets.length; i++) { + if (targetsStatus[i]) { + try { + DFSPacket packet = new DFSPacket(packetBuf, 0, + blockOffset4Targets[i], seqNo4Targets[i]++, checksumSize, true); + packet.writeTo(targetOutputStreams[i]); + targetOutputStreams[i].flush(); + } catch (IOException e) { + LOG.warn(e.getMessage()); + } + } + } + } + + /** + * Initialize output/input streams for transferring data to target + * and send create block request. + */ + private int initTargetStreams(boolean[] targetsStatus) { + int nsuccess = 0; + for (int i = 0; i < targets.length; i++) { + Socket socket = null; + DataOutputStream out = null; + DataInputStream in = null; + boolean success = false; + try { + InetSocketAddress targetAddr = + getSocketAddress4Transfer(targets[i]); + socket = datanode.newSocket(); + NetUtils.connect(socket, targetAddr, + datanode.getDnConf().getSocketTimeout()); + socket.setSoTimeout(datanode.getDnConf().getSocketTimeout()); + + ExtendedBlock block = getBlock(blockGroup, targetIndices[i]); + Token blockToken = + datanode.getBlockAccessToken(block, + EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE)); + + long writeTimeout = datanode.getDnConf().getSocketWriteTimeout(); + OutputStream unbufOut = NetUtils.getOutputStream(socket, writeTimeout); + InputStream unbufIn = NetUtils.getInputStream(socket); + DataEncryptionKeyFactory keyFactory = + datanode.getDataEncryptionKeyFactoryForBlock(block); + IOStreamPair saslStreams = datanode.getSaslClient().socketSend( + socket, unbufOut, unbufIn, keyFactory, blockToken, targets[i]); + + unbufOut = saslStreams.out; + unbufIn = saslStreams.in; + + out = new DataOutputStream(new BufferedOutputStream(unbufOut, + DFSUtilClient.getSmallBufferSize(conf))); + in = new DataInputStream(unbufIn); + + DatanodeInfo source = new DatanodeInfo(datanode.getDatanodeId()); + new Sender(out).writeBlock(block, targetStorageTypes[i], + blockToken, "", new DatanodeInfo[]{targets[i]}, + new StorageType[]{targetStorageTypes[i]}, source, + BlockConstructionStage.PIPELINE_SETUP_CREATE, 0, 0, 0, 0, + checksum, cachingStrategy, false, false, null); + + targetSockets[i] = socket; + targetOutputStreams[i] = out; + targetInputStreams[i] = in; + nsuccess++; + success = true; + } catch (Throwable e) { + LOG.warn(e.getMessage()); + } finally { + if (!success) { + IOUtils.closeStream(out); + IOUtils.closeStream(in); + IOUtils.closeStream(socket); + } + } + targetsStatus[i] = success; + } + return nsuccess; + } + } + + private static class StripedReader { + private final short index; // internal block index + private BlockReader blockReader; + private ByteBuffer buffer; + + private StripedReader(short index) { + this.index = index; + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java index afacebb51f8c0..1e07cad69b6b4 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/mover/Mover.java @@ -45,6 +45,7 @@ import org.apache.hadoop.hdfs.server.protocol.DatanodeStorageReport; import org.apache.hadoop.hdfs.server.protocol.StorageReport; import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.net.NetworkTopology; import org.apache.hadoop.util.StringUtils; import org.apache.hadoop.util.Time; @@ -174,8 +175,21 @@ private ExitStatus run() { } } - DBlock newDBlock(Block block, List locations) { - final DBlock db = new DBlock(block); + DBlock newDBlock(LocatedBlock lb, List locations, + ErasureCodingPolicy ecPolicy) { + Block blk = lb.getBlock().getLocalBlock(); + DBlock db; + if (lb.isStriped()) { + LocatedStripedBlock lsb = (LocatedStripedBlock) lb; + byte[] indices = new byte[lsb.getBlockIndices().length]; + for (int i = 0; i < indices.length; i++) { + indices[i] = (byte) lsb.getBlockIndices()[i]; + } + db = new DBlockStriped(blk, indices, (short) ecPolicy.getNumDataUnits(), + ecPolicy.getCellSize()); + } else { + db = new DBlock(blk); + } for(MLocation ml : locations) { StorageGroup source = storages.getSource(ml); if (source != null) { @@ -358,9 +372,10 @@ private void processFile(String fullPath, HdfsLocatedFileStatus status, LOG.warn("Failed to get the storage policy of file " + fullPath); return; } - final List types = policy.chooseStorageTypes( + List types = policy.chooseStorageTypes( status.getReplication()); + final ErasureCodingPolicy ecPolicy = status.getErasureCodingPolicy(); final LocatedBlocks locatedBlocks = status.getBlockLocations(); final boolean lastBlkComplete = locatedBlocks.isLastBlockComplete(); List lbs = locatedBlocks.getLocatedBlocks(); @@ -370,10 +385,13 @@ private void processFile(String fullPath, HdfsLocatedFileStatus status, continue; } LocatedBlock lb = lbs.get(i); + if (lb.isStriped()) { + types = policy.chooseStorageTypes((short) lb.getLocations().length); + } final StorageTypeDiff diff = new StorageTypeDiff(types, lb.getStorageTypes()); if (!diff.removeOverlap(true)) { - if (scheduleMoves4Block(diff, lb)) { + if (scheduleMoves4Block(diff, lb, ecPolicy)) { result.updateHasRemaining(diff.existing.size() > 1 && diff.expected.size() > 1); // One block scheduled successfully, set noBlockMoved to false @@ -385,10 +403,13 @@ private void processFile(String fullPath, HdfsLocatedFileStatus status, } } - boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb) { + boolean scheduleMoves4Block(StorageTypeDiff diff, LocatedBlock lb, + ErasureCodingPolicy ecPolicy) { final List locations = MLocation.toLocations(lb); - Collections.shuffle(locations); - final DBlock db = newDBlock(lb.getBlock().getLocalBlock(), locations); + if (!(lb instanceof LocatedStripedBlock)) { + Collections.shuffle(locations); + } + final DBlock db = newDBlock(lb, locations, ecPolicy); for (final StorageType t : diff.existing) { for (final MLocation ml : locations) { @@ -781,4 +802,4 @@ public static void main(String[] args) { System.exit(-1); } } -} \ No newline at end of file +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java new file mode 100644 index 0000000000000..b77279b4f0f97 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/ErasureCodingPolicyManager.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.namenode; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.ECSchema; + +import java.util.Map; +import java.util.TreeMap; + +/** + * This manages erasure coding policies predefined and activated in the system. + * It loads customized policies and syncs with persisted ones in + * NameNode image. + * + * This class is instantiated by the FSNamesystem. + */ +@InterfaceAudience.LimitedPrivate({"HDFS"}) +public final class ErasureCodingPolicyManager { + + /** + * TODO: HDFS-8095 + */ + private static final int DEFAULT_DATA_BLOCKS = 6; + private static final int DEFAULT_PARITY_BLOCKS = 3; + private static final int DEFAULT_CELLSIZE = 64 * 1024; + private static final String DEFAULT_CODEC_NAME = "rs"; + private static final String DEFAULT_POLICY_NAME = "RS-6-3-64k"; + private static final ECSchema SYS_DEFAULT_SCHEMA = new ECSchema( + DEFAULT_CODEC_NAME, DEFAULT_DATA_BLOCKS, DEFAULT_PARITY_BLOCKS); + private static final ErasureCodingPolicy SYS_DEFAULT_POLICY = + new ErasureCodingPolicy(DEFAULT_POLICY_NAME, SYS_DEFAULT_SCHEMA, + DEFAULT_CELLSIZE); + + //We may add more later. + private static ErasureCodingPolicy[] SYS_POLICY = new ErasureCodingPolicy[] { + SYS_DEFAULT_POLICY + }; + + /** + * All active policies maintained in NN memory for fast querying, + * identified and sorted by its name. + */ + private final Map activePolicies; + + ErasureCodingPolicyManager() { + + this.activePolicies = new TreeMap<>(); + for (ErasureCodingPolicy policy : SYS_POLICY) { + activePolicies.put(policy.getName(), policy); + } + + /** + * TODO: HDFS-7859 persist into NameNode + * load persistent policies from image and editlog, which is done only once + * during NameNode startup. This can be done here or in a separate method. + */ + } + + /** + * Get system defined policies. + * @return system policies + */ + public static ErasureCodingPolicy[] getSystemPolices() { + return SYS_POLICY; + } + + /** + * Get system-wide default policy, which can be used by default + * when no policy is specified for a path. + * @return ecPolicy + */ + public static ErasureCodingPolicy getSystemDefaultPolicy() { + return SYS_DEFAULT_POLICY; + } + + /** + * Get all policies that's available to use. + * @return all policies + */ + public ErasureCodingPolicy[] getPolicies() { + ErasureCodingPolicy[] results = new ErasureCodingPolicy[activePolicies.size()]; + return activePolicies.values().toArray(results); + } + + /** + * Get the policy specified by the policy name. + */ + public ErasureCodingPolicy getPolicy(String name) { + return activePolicies.get(name); + } + + /** + * Clear and clean up + */ + public void clear() { + activePolicies.clear(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java index 53255e66c1d5b..ae84f39506101 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirAppendOp.java @@ -106,6 +106,13 @@ static LastBlockWithStatus appendFile(final FSNamesystem fsn, + clientMachine); } final INodeFile file = INodeFile.valueOf(inode, path, true); + + // not support appending file with striped blocks + if (file.isStriped()) { + throw new UnsupportedOperationException( + "Cannot append to files with striped block " + src); + } + BlockManager blockManager = fsd.getBlockManager(); final BlockStoragePolicy lpPolicy = blockManager .getStoragePolicy("LAZY_PERSIST"); 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 df0bc2079ba6c..46e172d3d69a2 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 @@ -122,7 +122,7 @@ static HdfsFileStatus setTimes( " does not exist."); } boolean changed = unprotectedSetTimes(fsd, inode, mtime, atime, true, - iip.getLatestSnapshotId()); + iip.getLatestSnapshotId()); if (changed) { fsd.getEditLog().logTimes(src, mtime, atime); } @@ -399,17 +399,18 @@ static INodeDirectory unprotectedSetQuota( static BlockInfo[] unprotectedSetReplication( FSDirectory fsd, String src, short replication) throws QuotaExceededException, UnresolvedLinkException, - SnapshotAccessControlException { + SnapshotAccessControlException, UnsupportedActionException { assert fsd.hasWriteLock(); final BlockManager bm = fsd.getBlockManager(); final INodesInPath iip = fsd.getINodesInPath4Write(src, true); final INode inode = iip.getLastINode(); - if (inode == null || !inode.isFile()) { + if (inode == null || !inode.isFile() || inode.asFile().isStriped()) { + // TODO we do not support replication on stripe layout files yet return null; } - INodeFile file = inode.asFile(); + INodeFile file = inode.asFile(); // Make sure the directory has sufficient quotas short oldBR = file.getPreferredBlockReplication(); 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 492994eb2d606..e8b2c544fbd30 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 @@ -144,6 +144,7 @@ private static INodeFile[] verifySrcFiles(FSDirectory fsd, String[] srcs, 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() > @@ -153,6 +154,11 @@ private static INodeFile[] verifySrcFiles(FSDirectory fsd, String[] srcs, + " 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); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java new file mode 100644 index 0000000000000..d39da28d988ee --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirErasureCodingOp.java @@ -0,0 +1,277 @@ +/** + * 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 java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.EnumSet; +import java.util.List; + +import com.google.common.base.Preconditions; +import com.google.common.collect.Lists; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.fs.XAttr; +import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.fs.permission.FsAction; +import org.apache.hadoop.hdfs.XAttrHelper; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.io.IOUtils; +import org.apache.hadoop.io.WritableUtils; + +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.XATTR_ERASURECODING_POLICY; + +/** + * Helper class to perform erasure coding related operations. + */ +final class FSDirErasureCodingOp { + + /** + * Private constructor for preventing FSDirErasureCodingOp object + * creation. Static-only class. + */ + private FSDirErasureCodingOp() {} + + /** + * Set an erasure coding policy on the given path. + * + * @param fsn The namespace + * @param srcArg The path of the target directory. + * @param ecPolicy The erasure coding policy to set on the target directory. + * @param logRetryCache whether to record RPC ids in editlog for retry + * cache rebuilding + * @return {@link HdfsFileStatus} + * @throws IOException + */ + static HdfsFileStatus setErasureCodingPolicy(final FSNamesystem fsn, + final String srcArg, final ErasureCodingPolicy ecPolicy, + final boolean logRetryCache) throws IOException { + assert fsn.hasWriteLock(); + + String src = srcArg; + FSPermissionChecker pc = null; + byte[][] pathComponents = null; + pathComponents = FSDirectory.getPathComponentsForReservedPath(src); + pc = fsn.getPermissionChecker(); + FSDirectory fsd = fsn.getFSDirectory(); + src = fsd.resolvePath(pc, src, pathComponents); + final INodesInPath iip; + List xAttrs; + fsd.writeLock(); + try { + iip = fsd.getINodesInPath4Write(src, false); + xAttrs = createErasureCodingPolicyXAttr(fsn, iip, ecPolicy); + } finally { + fsd.writeUnlock(); + } + fsn.getEditLog().logSetXAttrs(src, xAttrs, logRetryCache); + return fsd.getAuditFileInfo(iip); + } + + static List createErasureCodingPolicyXAttr(final FSNamesystem fsn, + final INodesInPath srcIIP, ErasureCodingPolicy ecPolicy) throws IOException { + FSDirectory fsd = fsn.getFSDirectory(); + assert fsd.hasWriteLock(); + Preconditions.checkNotNull(srcIIP, "INodes cannot be null"); + String src = srcIIP.getPath(); + if (srcIIP.getLastINode() != null && + !srcIIP.getLastINode().isDirectory()) { + throw new IOException("Attempt to set an erasure coding policy " + + "for a file " + src); + } + if (getErasureCodingPolicyForPath(fsn, srcIIP) != null) { + throw new IOException("Directory " + src + " already has an " + + "erasure coding policy."); + } + + // System default erasure coding policy will be used since no specified. + if (ecPolicy == null) { + ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + } else { + // If ecPolicy is specified check if it is one among active policies. + boolean validPolicy = false; + ErasureCodingPolicy[] activePolicies = + FSDirErasureCodingOp.getErasureCodingPolicies(fsd.getFSNamesystem()); + for (ErasureCodingPolicy activePolicy : activePolicies) { + if (activePolicy.equals(ecPolicy)) { + validPolicy = true; + break; + } + } + if (!validPolicy) { + List ecPolicyNames = new ArrayList(); + for (ErasureCodingPolicy activePolicy : activePolicies) { + ecPolicyNames.add(activePolicy.getName()); + } + throw new HadoopIllegalArgumentException("Policy [ " + + ecPolicy.getName()+ " ] does not match any of the " + + "supported policies. Please select any one of " + ecPolicyNames); + } + } + + final XAttr ecXAttr; + DataOutputStream dOut = null; + try { + ByteArrayOutputStream bOut = new ByteArrayOutputStream(); + dOut = new DataOutputStream(bOut); + WritableUtils.writeString(dOut, ecPolicy.getName()); + ecXAttr = XAttrHelper.buildXAttr(XATTR_ERASURECODING_POLICY, + bOut.toByteArray()); + } finally { + IOUtils.closeStream(dOut); + } + final List xattrs = Lists.newArrayListWithCapacity(1); + xattrs.add(ecXAttr); + FSDirXAttrOp.unprotectedSetXAttrs(fsd, src, xattrs, + EnumSet.of(XAttrSetFlag.CREATE)); + return xattrs; + } + + /** + * Get the erasure coding policy information for specified path. + * + * @param fsn namespace + * @param src path + * @return {@link ErasureCodingPolicy} + * @throws IOException + */ + static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn, + final String src) throws IOException { + assert fsn.hasReadLock(); + + final INodesInPath iip = getINodesInPath(fsn, src); + return getErasureCodingPolicyForPath(fsn, iip); + } + + /** + * Check if the file or directory has an erasure coding policy. + * + * @param fsn namespace + * @param srcArg path + * @return Whether the file or directory has an erasure coding policy. + * @throws IOException + */ + static boolean hasErasureCodingPolicy(final FSNamesystem fsn, + final String srcArg) throws IOException { + return hasErasureCodingPolicy(fsn, getINodesInPath(fsn, srcArg)); + } + + /** + * Check if the file or directory has an erasure coding policy. + * + * @param fsn namespace + * @param iip inodes in the path containing the file + * @return Whether the file or directory has an erasure coding policy. + * @throws IOException + */ + static boolean hasErasureCodingPolicy(final FSNamesystem fsn, + final INodesInPath iip) throws IOException { + return getErasureCodingPolicy(fsn, iip) != null; + } + + /** + * Get the erasure coding policy. + * + * @param fsn namespace + * @param iip inodes in the path containing the file + * @return {@link ErasureCodingPolicy} + * @throws IOException + */ + static ErasureCodingPolicy getErasureCodingPolicy(final FSNamesystem fsn, + final INodesInPath iip) throws IOException { + assert fsn.hasReadLock(); + + return getErasureCodingPolicyForPath(fsn, iip); + } + + /** + * Get available erasure coding polices. + * + * @param fsn namespace + * @return {@link ErasureCodingPolicy} array + */ + static ErasureCodingPolicy[] getErasureCodingPolicies(final FSNamesystem fsn) + throws IOException { + assert fsn.hasReadLock(); + + return fsn.getErasureCodingPolicyManager().getPolicies(); + } + + private static INodesInPath getINodesInPath(final FSNamesystem fsn, + final String srcArg) throws IOException { + String src = srcArg; + final byte[][] pathComponents = FSDirectory + .getPathComponentsForReservedPath(src); + final FSDirectory fsd = fsn.getFSDirectory(); + final FSPermissionChecker pc = fsn.getPermissionChecker(); + src = fsd.resolvePath(pc, src, pathComponents); + INodesInPath iip = fsd.getINodesInPath(src, true); + if (fsn.isPermissionEnabled()) { + fsn.getFSDirectory().checkPathAccess(pc, iip, FsAction.READ); + } + return iip; + } + + private static ErasureCodingPolicy getErasureCodingPolicyForPath(FSNamesystem fsn, + INodesInPath iip) throws IOException { + Preconditions.checkNotNull(iip, "INodes cannot be null"); + FSDirectory fsd = fsn.getFSDirectory(); + fsd.readLock(); + try { + List inodes = iip.getReadOnlyINodes(); + for (int i = inodes.size() - 1; i >= 0; i--) { + final INode inode = inodes.get(i); + if (inode == null) { + continue; + } + /** + * TODO: lookup {@link ErasureCodingPolicyManager#getSystemPolices()} + */ + if (inode.isFile()) { + return inode.asFile().getErasureCodingPolicyID() == 0 ? + null : ErasureCodingPolicyManager.getSystemDefaultPolicy(); + } + // We don't allow setting EC policies on paths with a symlink. Thus + // if a symlink is encountered, the dir shouldn't have EC policy. + // TODO: properly support symlinks + if (inode.isSymlink()) { + return null; + } + final XAttrFeature xaf = inode.getXAttrFeature(); + if (xaf != null) { + XAttr xattr = xaf.getXAttr(XATTR_ERASURECODING_POLICY); + if (xattr != null) { + ByteArrayInputStream bIn = new ByteArrayInputStream(xattr.getValue()); + DataInputStream dIn = new DataInputStream(bIn); + String ecPolicyName = WritableUtils.readString(dIn); + return fsd.getFSNamesystem().getErasureCodingPolicyManager(). + getPolicy(ecPolicyName); + } + } + } + } finally { + fsd.readUnlock(); + } + return null; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java index 98af5922ce842..67c6fc1671ff3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirStatAndListingOp.java @@ -19,6 +19,7 @@ package org.apache.hadoop.hdfs.server.namenode; import com.google.common.base.Preconditions; + import org.apache.commons.io.Charsets; import org.apache.hadoop.fs.ContentSummary; import org.apache.hadoop.fs.DirectoryListingStartAfterNotFoundException; @@ -29,6 +30,7 @@ import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.FsPermissionExtension; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; @@ -179,11 +181,13 @@ static GetBlockLocationsResult getBlockLocations( final FileEncryptionInfo feInfo = isReservedName ? null : FSDirEncryptionZoneOp.getFileEncryptionInfo(fsd, inode, - iip.getPathSnapshotId(), iip); + iip.getPathSnapshotId(), iip); + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp. + getErasureCodingPolicy(fsd.getFSNamesystem(), iip); final LocatedBlocks blocks = bm.createLocatedBlocks( inode.getBlocks(iip.getPathSnapshotId()), fileSize, isUc, offset, - length, needBlockToken, iip.isSnapshot(), feInfo); + length, needBlockToken, iip.isSnapshot(), feInfo, ecPolicy); // Set caching information for the located blocks. for (LocatedBlock lb : blocks.getLocatedBlocks()) { @@ -375,7 +379,7 @@ static HdfsFileStatus getFileInfo( if (fsd.getINode4DotSnapshot(srcs) != null) { return new HdfsFileStatus(0, true, 0, 0, 0, 0, null, null, null, null, HdfsFileStatus.EMPTY_NAME, -1L, 0, null, - HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED); + HdfsConstants.BLOCK_STORAGE_POLICY_ID_UNSPECIFIED, null); } return null; } @@ -443,6 +447,9 @@ static HdfsFileStatus createFileStatus( final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp .getFileEncryptionInfo(fsd, node, snapshot, iip); + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( + fsd.getFSNamesystem(), iip); + if (node.isFile()) { final INodeFile fileNode = node.asFile(); size = fileNode.computeFileSize(snapshot); @@ -474,7 +481,8 @@ static HdfsFileStatus createFileStatus( node.getId(), childrenNum, feInfo, - storagePolicy); + storagePolicy, + ecPolicy); } private static INodeAttributes getINodeAttributes( @@ -497,6 +505,8 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( final boolean isEncrypted; final FileEncryptionInfo feInfo = isRawPath ? null : FSDirEncryptionZoneOp .getFileEncryptionInfo(fsd, node, snapshot, iip); + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( + fsd.getFSNamesystem(), iip); if (node.isFile()) { final INodeFile fileNode = node.asFile(); size = fileNode.computeFileSize(snapshot); @@ -510,7 +520,7 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( loc = fsd.getBlockManager().createLocatedBlocks( fileNode.getBlocks(snapshot), fileSize, isUc, 0L, size, false, - inSnapshot, feInfo); + inSnapshot, feInfo, ecPolicy); if (loc == null) { loc = new LocatedBlocks(); } @@ -531,7 +541,7 @@ private static HdfsLocatedFileStatus createLocatedFileStatus( getPermissionForFileStatus(nodeAttrs, isEncrypted), nodeAttrs.getUserName(), nodeAttrs.getGroupName(), node.isSymlink() ? node.asSymlink().getSymlink() : null, path, - node.getId(), loc, childrenNum, feInfo, storagePolicy); + node.getId(), loc, childrenNum, feInfo, storagePolicy, ecPolicy); // Set caching information for the located blocks. if (loc != null) { CacheManager cacheManager = fsd.getFSNamesystem().getCacheManager(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java index 6d37530cc1030..b46a195e45375 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirTruncateOp.java @@ -84,6 +84,13 @@ static TruncateResult truncate(final FSNamesystem fsn, final String srcArg, fsd.checkPathAccess(pc, iip, FsAction.WRITE); } INodeFile file = INodeFile.valueOf(iip.getLastINode(), src); + + // not support truncating file with striped blocks + if (file.isStriped()) { + throw new UnsupportedOperationException( + "Cannot truncate file with striped block " + src); + } + final BlockStoragePolicy lpPolicy = fsd.getBlockManager() .getStoragePolicy("LAZY_PERSIST"); @@ -208,6 +215,7 @@ static Block prepareFileForTruncate(FSNamesystem fsn, INodesInPath iip, assert fsn.hasWriteLock(); INodeFile file = iip.getLastINode().asFile(); + assert !file.isStriped(); file.recordModification(iip.getLatestSnapshotId()); file.toUnderConstruction(leaseHolder, clientMachine); assert file.isUnderConstruction() : "inode should be under construction."; @@ -215,10 +223,11 @@ static Block prepareFileForTruncate(FSNamesystem fsn, INodesInPath iip, file.getFileUnderConstructionFeature().getClientName(), file.getId()); boolean shouldRecoverNow = (newBlock == null); BlockInfo oldBlock = file.getLastBlock(); + boolean shouldCopyOnTruncate = shouldCopyOnTruncate(fsn, file, oldBlock); if (newBlock == null) { - newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock() : new Block( - oldBlock.getBlockId(), oldBlock.getNumBytes(), + newBlock = (shouldCopyOnTruncate) ? fsn.createNewBlock(false) + : new Block(oldBlock.getBlockId(), oldBlock.getNumBytes(), fsn.nextGenerationStamp(fsn.getBlockIdManager().isLegacyBlock( oldBlock))); } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java index 575b1fd3885a3..4623264bc0e1a 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/FSDirWriteFileOp.java @@ -35,6 +35,7 @@ import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; @@ -44,6 +45,8 @@ import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; 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.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -88,7 +91,7 @@ static boolean unprotectedRemoveBlock( // update space consumed fsd.updateCount(iip, 0, -fileNode.getPreferredBlockSize(), - fileNode.getPreferredBlockReplication(), true); + fileNode.getPreferredBlockReplication(), true); return true; } @@ -131,6 +134,9 @@ static void abandonBlock( FSNamesystem fsn = fsd.getFSNamesystem(); final INodeFile file = fsn.checkLease(src, holder, inode, fileId); Preconditions.checkState(file.isUnderConstruction()); + if (file.isStriped()) { + return; // do not abandon block for striped file + } Block localBlock = ExtendedBlock.getLocalBlock(b); fsd.writeLock(); @@ -168,9 +174,10 @@ static ValidateAddBlockResult validateAddBlock( String src, long fileId, String clientName, ExtendedBlock previous, LocatedBlock[] onRetryBlock) throws IOException { final long blockSize; - final int replication; + final short numTargets; final byte storagePolicyID; String clientMachine; + final boolean isStriped; byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); src = fsn.dir.resolvePath(pc, src, pathComponents); @@ -196,18 +203,26 @@ static ValidateAddBlockResult validateAddBlock( blockSize = pendingFile.getPreferredBlockSize(); clientMachine = pendingFile.getFileUnderConstructionFeature() .getClientMachine(); - replication = pendingFile.getFileReplication(); + isStriped = pendingFile.isStriped(); + ErasureCodingPolicy ecPolicy = null; + if (isStriped) { + ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy(fsn, src); + numTargets = (short) (ecPolicy.getSchema().getNumDataUnits() + + ecPolicy.getSchema().getNumParityUnits()); + } else { + numTargets = pendingFile.getFileReplication(); + } storagePolicyID = pendingFile.getStoragePolicyID(); - return new ValidateAddBlockResult(blockSize, replication, storagePolicyID, - clientMachine); + return new ValidateAddBlockResult(blockSize, numTargets, storagePolicyID, + clientMachine, isStriped); } - static LocatedBlock makeLocatedBlock(FSNamesystem fsn, Block blk, + static LocatedBlock makeLocatedBlock(FSNamesystem fsn, BlockInfo blk, DatanodeStorageInfo[] locs, long offset) throws IOException { - LocatedBlock lBlk = BlockManager.newLocatedBlock(fsn.getExtendedBlock(blk), - locs, offset, false); - fsn.getFSDirectory().getBlockManager() - .setBlockToken(lBlk, BlockTokenIdentifier.AccessMode.WRITE); + LocatedBlock lBlk = BlockManager.newLocatedBlock( + fsn.getExtendedBlock(new Block(blk)), blk, locs, offset); + fsn.getBlockManager().setBlockToken(lBlk, + BlockTokenIdentifier.AccessMode.WRITE); return lBlk; } @@ -237,8 +252,8 @@ static LocatedBlock storeAllocatedBlock(FSNamesystem fsn, String src, } else { // add new chosen targets to already allocated block and return BlockInfo lastBlockInFile = pendingFile.getLastBlock(); - lastBlockInFile.getUnderConstructionFeature(). - setExpectedLocations(lastBlockInFile, targets); + lastBlockInFile.getUnderConstructionFeature().setExpectedLocations( + lastBlockInFile, targets, pendingFile.isStriped()); offset = pendingFile.computeFileSize(); return makeLocatedBlock(fsn, lastBlockInFile, targets, offset); } @@ -249,15 +264,17 @@ static LocatedBlock storeAllocatedBlock(FSNamesystem fsn, String src, ExtendedBlock.getLocalBlock(previous)); // allocate new block, record block locations in INode. - Block newBlock = fsn.createNewBlock(); + final boolean isStriped = pendingFile.isStriped(); + // allocate new block, record block locations in INode. + Block newBlock = fsn.createNewBlock(isStriped); INodesInPath inodesInPath = INodesInPath.fromINode(pendingFile); - saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets); + saveAllocatedBlock(fsn, src, inodesInPath, newBlock, targets, isStriped); persistNewBlock(fsn, src, pendingFile); offset = pendingFile.computeFileSize(); // Return located block - return makeLocatedBlock(fsn, newBlock, targets, offset); + return makeLocatedBlock(fsn, fsn.getStoredBlock(newBlock), targets, offset); } static DatanodeStorageInfo[] chooseTargetForNewBlock( @@ -278,9 +295,10 @@ static DatanodeStorageInfo[] chooseTargetForNewBlock( : Arrays.asList(favoredNodes); // choose targets for the new block to be allocated. - return bm.chooseTarget4NewBlock(src, r.replication, clientNode, + return bm.chooseTarget4NewBlock(src, r.numTargets, clientNode, excludedNodesSet, r.blockSize, - favoredNodesList, r.storagePolicyID); + favoredNodesList, r.storagePolicyID, + r.isStriped); } /** @@ -469,22 +487,22 @@ static INodeFile addFileForEditLog( long preferredBlockSize, boolean underConstruction, String clientName, String clientMachine, byte storagePolicyId) { final INodeFile newNode; + Preconditions.checkNotNull(existing); assert fsd.hasWriteLock(); - if (underConstruction) { - newNode = newINodeFile(id, permissions, modificationTime, - modificationTime, replication, - preferredBlockSize, - storagePolicyId); - newNode.toUnderConstruction(clientName, clientMachine); - } else { - newNode = newINodeFile(id, permissions, modificationTime, - atime, replication, - preferredBlockSize, - storagePolicyId); - } - - newNode.setLocalName(localName); try { + // check if the file has an EC policy + final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy( + fsd.getFSNamesystem(), existing); + if (underConstruction) { + newNode = newINodeFile(id, permissions, modificationTime, + modificationTime, replication, preferredBlockSize, storagePolicyId, + isStriped); + newNode.toUnderConstruction(clientName, clientMachine); + } else { + newNode = newINodeFile(id, permissions, modificationTime, atime, + replication, preferredBlockSize, storagePolicyId, isStriped); + } + newNode.setLocalName(localName); INodesInPath iip = fsd.addINode(existing, newNode); if (iip != null) { if (aclEntries != null) { @@ -508,23 +526,39 @@ static INodeFile addFileForEditLog( /** * Add a block to the file. Returns a reference to the added block. */ - private static BlockInfo addBlock( - FSDirectory fsd, String path, INodesInPath inodesInPath, Block block, - DatanodeStorageInfo[] targets) throws IOException { + private static BlockInfo addBlock(FSDirectory fsd, String path, + INodesInPath inodesInPath, Block block, DatanodeStorageInfo[] targets, + boolean isStriped) throws IOException { fsd.writeLock(); try { final INodeFile fileINode = inodesInPath.getLastINode().asFile(); Preconditions.checkState(fileINode.isUnderConstruction()); - // check quota limits and updated space consumed - fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), - fileINode.getFileReplication(), true); - // associate new last block for the file - BlockInfo blockInfo = new BlockInfoContiguous(block, - fileINode.getFileReplication()); - blockInfo.convertToBlockUnderConstruction( - HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); + final BlockInfo blockInfo; + if (isStriped) { + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( + fsd.getFSNamesystem(), inodesInPath); + short numDataUnits = (short) ecPolicy.getNumDataUnits(); + short numParityUnits = (short) ecPolicy.getNumParityUnits(); + short numLocations = (short) (numDataUnits + numParityUnits); + + // check quota limits and updated space consumed + fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), + numLocations, true); + blockInfo = new BlockInfoStriped(block, ecPolicy); + blockInfo.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); + } else { + // check quota limits and updated space consumed + fsd.updateCount(inodesInPath, 0, fileINode.getPreferredBlockSize(), + fileINode.getFileReplication(), true); + + short numLocations = fileINode.getFileReplication(); + blockInfo = new BlockInfoContiguous(block, numLocations); + blockInfo.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, targets); + } fsd.getBlockManager().addBlockCollection(blockInfo, fileINode); fileINode.addBlock(blockInfo); @@ -550,22 +584,24 @@ private static INodesInPath addFile( String clientName, String clientMachine) throws IOException { + Preconditions.checkNotNull(existing); long modTime = now(); - INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions, - modTime, modTime, replication, preferredBlockSize); - newNode.setLocalName(localName.getBytes(Charsets.UTF_8)); - newNode.toUnderConstruction(clientName, clientMachine); - INodesInPath newiip; fsd.writeLock(); try { + final boolean isStriped = FSDirErasureCodingOp.hasErasureCodingPolicy( + fsd.getFSNamesystem(), existing); + INodeFile newNode = newINodeFile(fsd.allocateNewInodeId(), permissions, + modTime, modTime, replication, preferredBlockSize, isStriped); + newNode.setLocalName(localName.getBytes(Charsets.UTF_8)); + newNode.toUnderConstruction(clientName, clientMachine); newiip = fsd.addINode(existing, newNode); } finally { fsd.writeUnlock(); } if (newiip == null) { NameNode.stateChangeLog.info("DIR* addFile: failed to add " + - existing.getPath() + "/" + localName); + existing.getPath() + "/" + localName); return null; } @@ -578,7 +614,7 @@ private static INodesInPath addFile( private static FileState analyzeFileState( FSNamesystem fsn, String src, long fileId, String clientName, ExtendedBlock previous, LocatedBlock[] onRetryBlock) - throws IOException { + throws IOException { assert fsn.hasReadLock(); checkBlock(fsn, previous); @@ -687,14 +723,8 @@ static boolean completeFile(FSNamesystem fsn, FSPermissionChecker pc, checkBlock(fsn, last); byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); src = fsn.dir.resolvePath(pc, src, pathComponents); - boolean success = completeFileInternal(fsn, src, holder, - ExtendedBlock.getLocalBlock(last), - fileId); - if (success) { - NameNode.stateChangeLog.info("DIR* completeFile: " + srcArg - + " is closed by " + holder); - } - return success; + return completeFileInternal(fsn, src, holder, + ExtendedBlock.getLocalBlock(last), fileId); } private static boolean completeFileInternal( @@ -759,16 +789,18 @@ private static boolean completeFileInternal( private static INodeFile newINodeFile( long id, PermissionStatus permissions, long mtime, long atime, - short replication, long preferredBlockSize, byte storagePolicyId) { + short replication, long preferredBlockSize, byte storagePolicyId, + boolean isStriped) { return new INodeFile(id, null, permissions, mtime, atime, BlockInfo.EMPTY_ARRAY, replication, preferredBlockSize, - storagePolicyId); + storagePolicyId, isStriped); } private static INodeFile newINodeFile(long id, PermissionStatus permissions, - long mtime, long atime, short replication, long preferredBlockSize) { + long mtime, long atime, short replication, long preferredBlockSize, + boolean isStriped) { return newINodeFile(id, permissions, mtime, atime, replication, - preferredBlockSize, (byte)0); + preferredBlockSize, (byte)0, isStriped); } /** @@ -796,13 +828,12 @@ private static void persistNewBlock( * @param targets target datanodes where replicas of the new block is placed * @throws QuotaExceededException If addition of block exceeds space quota */ - private static void saveAllocatedBlock( - FSNamesystem fsn, String src, INodesInPath inodesInPath, Block newBlock, - DatanodeStorageInfo[] targets) - throws IOException { + private static void saveAllocatedBlock(FSNamesystem fsn, String src, + INodesInPath inodesInPath, Block newBlock, DatanodeStorageInfo[] targets, + boolean isStriped) throws IOException { assert fsn.hasWriteLock(); - BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, - targets); + BlockInfo b = addBlock(fsn.dir, src, inodesInPath, newBlock, targets, + isStriped); NameNode.stateChangeLog.info("BLOCK* allocate " + b + " for " + src); DatanodeStorageInfo.incrementBlocksScheduled(targets); } @@ -851,17 +882,19 @@ private static class FileState { static class ValidateAddBlockResult { final long blockSize; - final int replication; + final int numTargets; final byte storagePolicyID; final String clientMachine; + final boolean isStriped; ValidateAddBlockResult( - long blockSize, int replication, byte storagePolicyID, - String clientMachine) { + long blockSize, int numTargets, byte storagePolicyID, + String clientMachine, boolean isStriped) { this.blockSize = blockSize; - this.replication = replication; + this.numTargets = numTargets; this.storagePolicyID = storagePolicyID; this.clientMachine = clientMachine; + this.isStriped = isStriped; } } 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 866305f11ad19..5b144b8ce2909 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 @@ -688,7 +688,7 @@ void updateCount(INodesInPath iip, long nsDelta, long ssDelta, short replication final INodeFile fileINode = iip.getLastINode().asFile(); EnumCounters typeSpaceDeltas = getStorageTypeDeltas(fileINode.getStoragePolicyID(), ssDelta, - replication, replication);; + replication, replication); updateCount(iip, iip.length() - 1, new QuotaCounts.Builder().nameSpace(nsDelta).storageSpace(ssDelta * replication). typeSpaces(typeSpaceDeltas).build(), 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 c2cccb56c1019..6819d8dd43615 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 @@ -36,16 +36,20 @@ import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.FileSystem; import org.apache.hadoop.fs.XAttrSetFlag; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; 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.BlockInfoStriped; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.LayoutVersion; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.RollingUpgradeStartupOption; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.hdfs.server.common.Storage; @@ -412,7 +416,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // 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); + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( + fsDir.getFSNamesystem(), iip); + updateBlocks(fsDir, addCloseOp, iip, newFile, ecPolicy); break; } case OP_CLOSE: { @@ -432,7 +438,9 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, // 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); + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( + fsDir.getFSNamesystem(), iip); + updateBlocks(fsDir, addCloseOp, iip, file, ecPolicy); // Now close the file if (!file.isUnderConstruction() && @@ -490,8 +498,10 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, INodesInPath iip = fsDir.getINodesInPath(path, true); INodeFile oldFile = INodeFile.valueOf(iip.getLastINode(), path); // Update in-memory data structures - updateBlocks(fsDir, updateOp, iip, oldFile); - + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( + fsDir.getFSNamesystem(), iip); + updateBlocks(fsDir, updateOp, iip, oldFile, ecPolicy); + if (toAddRetryCache) { fsNamesys.addCacheEntry(updateOp.rpcClientId, updateOp.rpcCallId); } @@ -504,9 +514,12 @@ private long applyEditLogOp(FSEditLogOp op, FSDirectory fsDir, 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 - addNewBlock(addBlockOp, oldFile); + ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp.getErasureCodingPolicy( + fsDir.getFSNamesystem(), iip); + addNewBlock(addBlockOp, oldFile, ecPolicy); break; } case OP_SET_REPLICATION: { @@ -786,8 +799,15 @@ fsDir, renameReservedPathsOnUpgrade(timesOp.path, logVersion), } 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: { @@ -940,14 +960,14 @@ private static String formatEditLogReplayError(EditLogInputStream in, /** * Add a new block into the given INodeFile */ - private void addNewBlock(AddBlockOp op, INodeFile file) - throws IOException { + private void addNewBlock(AddBlockOp op, INodeFile file, + ErasureCodingPolicy ecPolicy) 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 BlockInfo oldLastBlock = oldBlocks[oldBlocks.length - 1]; if (oldLastBlock.getBlockId() != pBlock.getBlockId() @@ -967,12 +987,18 @@ private void addNewBlock(AddBlockOp op, INodeFile file) Preconditions.checkState(oldBlocks == null || oldBlocks.length == 0); } // add the new block - BlockInfo newBI = new BlockInfoContiguous(newBlock, - file.getPreferredBlockReplication()); - newBI.convertToBlockUnderConstruction( - HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); - fsNamesys.getBlockManager().addBlockCollection(newBI, file); - file.addBlock(newBI); + final BlockInfo newBlockInfo; + boolean isStriped = ecPolicy != null; + if (isStriped) { + newBlockInfo = new BlockInfoStriped(newBlock, ecPolicy); + } else { + newBlockInfo = new BlockInfoContiguous(newBlock, + file.getPreferredBlockReplication()); + } + newBlockInfo.convertToBlockUnderConstruction( + BlockUCState.UNDER_CONSTRUCTION, null); + fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBlockInfo, file); + file.addBlock(newBlockInfo); fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock); } @@ -981,7 +1007,8 @@ private void addNewBlock(AddBlockOp op, INodeFile file) * @throws IOException */ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, - INodesInPath iip, INodeFile file) throws IOException { + INodesInPath iip, INodeFile file, ErasureCodingPolicy ecPolicy) + throws IOException { // Update its block list BlockInfo[] oldBlocks = file.getBlocks(); Block[] newBlocks = op.getBlocks(); @@ -1040,27 +1067,37 @@ private void updateBlocks(FSDirectory fsDir, BlockListUpdatingOp op, throw new IOException("Trying to delete non-existant block " + oldBlock); } } else if (newBlocks.length > oldBlocks.length) { + final boolean isStriped = ecPolicy != null; // We're adding blocks for (int i = oldBlocks.length; i < newBlocks.length; i++) { Block newBlock = newBlocks[i]; - BlockInfo 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 BlockInfoContiguous(newBlock, - file.getPreferredBlockReplication()); + if (isStriped) { + newBI = new BlockInfoStriped(newBlock, ecPolicy); + } else { + newBI = new BlockInfoContiguous(newBlock, + file.getPreferredBlockReplication()); + } newBI.convertToBlockUnderConstruction( - HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); + BlockUCState.UNDER_CONSTRUCTION, null); } else { // OP_CLOSE should add finalized blocks. This code path // is only executed when loading edits written by prior // versions of Hadoop. Current versions always log // OP_ADD operations as each block is allocated. - newBI = new BlockInfoContiguous(newBlock, - file.getFileReplication()); + if (isStriped) { + newBI = new BlockInfoStriped(newBlock, + ErasureCodingPolicyManager.getSystemDefaultPolicy()); + } else { + newBI = new BlockInfoContiguous(newBlock, + file.getFileReplication()); + } } - fsNamesys.getBlockManager().addBlockCollection(newBI, file); + fsNamesys.getBlockManager().addBlockCollectionWithCheck(newBI, file); file.addBlock(newBI); fsNamesys.getBlockManager().processQueuedMessagesForBlock(newBlock); } 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 92f333ac3b8c3..0b1902f2274a8 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 @@ -48,6 +48,7 @@ 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; @@ -358,7 +359,8 @@ public void load(File curFile) throws IOException { // read the max sequential block ID. long maxSequentialBlockId = in.readLong(); - namesystem.getBlockIdManager().setLastAllocatedBlockId(maxSequentialBlockId); + namesystem.getBlockIdManager().setLastAllocatedContiguousBlockId( + maxSequentialBlockId); } else { long startingGenStamp = namesystem.getBlockIdManager() @@ -690,7 +692,7 @@ public void updateBlocksMap(INodeFile file) { if (blocks != null) { final BlockManager bm = namesystem.getBlockManager(); for (int i = 0; i < blocks.length; i++) { - file.setBlock(i, bm.addBlockCollection(blocks[i], file)); + file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file)); } } } @@ -753,7 +755,7 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, // file // read blocks - BlockInfo[] blocks = new BlockInfo[numBlocks]; + BlockInfo[] blocks = new BlockInfoContiguous[numBlocks]; for (int j = 0; j < numBlocks; j++) { blocks[j] = new BlockInfoContiguous(replication); blocks[j].readFields(in); @@ -790,14 +792,15 @@ INode loadINode(final byte[] localName, boolean isSnapshotINode, counter.increment(); } - final INodeFile file = new INodeFile(inodeId, localName, permissions, - modificationTime, atime, blocks, replication, blockSize); + INodeFile 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(); @@ -896,7 +899,7 @@ public INodeFileAttributes loadINodeFileAttributes(DataInput in) final long preferredBlockSize = in.readLong(); return new INodeFileAttributes.SnapshotCopy(name, permissions, null, modificationTime, - accessTime, replication, preferredBlockSize, (byte) 0, null); + accessTime, replication, preferredBlockSize, (byte) 0, null, false); } public INodeDirectoryAttributes loadINodeDirectoryAttributes(DataInput in) @@ -960,8 +963,8 @@ LayoutVersion.Feature.ADD_INODE_ID, getLayoutVersion())) { if (oldnode.numBlocks() > 0) { BlockInfo ucBlock = cons.getLastBlock(); // we do not replace the inode, just replace the last block of oldnode - BlockInfo info = namesystem.getBlockManager().addBlockCollection( - ucBlock, oldnode); + BlockInfo info = namesystem.getBlockManager() + .addBlockCollectionWithCheck(ucBlock, oldnode); oldnode.setBlock(oldnode.numBlocks() - 1, info); } @@ -1140,7 +1143,7 @@ static String renameReservedPathsOnUpgrade(String path, + " option to automatically rename these paths during upgrade."; /** - * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single + * Same as {@link #renameReservedPathsOnUpgrade}, but for a single * byte array path component. */ private static byte[] renameReservedComponentOnUpgrade(byte[] component, @@ -1160,7 +1163,7 @@ private static byte[] renameReservedComponentOnUpgrade(byte[] component, } /** - * Same as {@link #renameReservedPathsOnUpgrade(String)}, but for a single + * Same as {@link #renameReservedPathsOnUpgrade}, but for a single * byte array path component. */ private static byte[] renameReservedRootComponentOnUpgrade(byte[] component, @@ -1268,7 +1271,7 @@ void save(File newFile, FSImageCompression compression) throws IOException { 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(context.getTxId()); out.writeLong(sourceNamesystem.dir.getLastInodeId()); 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 0ae739c5256d7..34b28e4ba0465 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 @@ -39,11 +39,13 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.HdfsConstants; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos.BlockProto; import org.apache.hadoop.hdfs.protocolPB.PBHelperClient; 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.BlockManager; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.FSImageFormatProtobuf.LoaderContext; @@ -218,7 +220,7 @@ public static void updateBlocksMap(INodeFile file, BlockManager bm) { final BlockInfo[] blocks = file.getBlocks(); if (blocks != null) { for (int i = 0; i < blocks.length; i++) { - file.setBlock(i, bm.addBlockCollection(blocks[i], file)); + file.setBlock(i, bm.addBlockCollectionWithCheck(blocks[i], file)); } } } @@ -331,27 +333,35 @@ private INodeFile loadINodeFile(INodeSection.INode n) { INodeSection.INodeFile f = n.getFile(); List bp = f.getBlocksList(); short replication = (short) f.getReplication(); + boolean isStriped = f.getIsStriped(); LoaderContext state = parent.getLoaderContext(); + ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); BlockInfo[] blocks = new BlockInfo[bp.size()]; - for (int i = 0, e = bp.size(); i < e; ++i) { - blocks[i] = - new BlockInfoContiguous(PBHelperClient.convert(bp.get(i)), replication); + for (int i = 0; i < bp.size(); ++i) { + BlockProto b = bp.get(i); + if (isStriped) { + blocks[i] = new BlockInfoStriped(PBHelperClient.convert(b), ecPolicy); + } else { + blocks[i] = new BlockInfoContiguous(PBHelperClient.convert(b), + replication); + } } + final PermissionStatus permissions = loadPermission(f.getPermission(), parent.getLoaderContext().getStringTable()); final INodeFile file = new INodeFile(n.getId(), n.getName().toByteArray(), permissions, f.getModificationTime(), f.getAccessTime(), blocks, replication, f.getPreferredBlockSize(), - (byte)f.getStoragePolicyID()); + (byte)f.getStoragePolicyID(), isStriped); if (f.hasAcl()) { int[] entries = AclEntryStatusFormat.toInt(loadAclEntries( f.getAcl(), state.getStringTable())); file.addAclFeature(new AclFeature(entries)); } - + if (f.hasXAttrs()) { file.addXAttrFeature(new XAttrFeature( loadXAttrs(f.getXAttrs(), state.getStringTable()))); @@ -363,8 +373,18 @@ private INodeFile loadINodeFile(INodeSection.INode n) { file.toUnderConstruction(uc.getClientName(), uc.getClientMachine()); if (blocks.length > 0) { BlockInfo lastBlk = file.getLastBlock(); - lastBlk.convertToBlockUnderConstruction( + // replace the last block of file + final BlockInfo ucBlk; + if (isStriped) { + BlockInfoStriped striped = (BlockInfoStriped) lastBlk; + ucBlk = new BlockInfoStriped(striped, ecPolicy); + } else { + ucBlk = new BlockInfoContiguous(lastBlk, + replication); + } + ucBlk.convertToBlockUnderConstruction( HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); + file.setBlock(file.numBlocks() - 1, ucBlk); } } return file; @@ -478,7 +498,8 @@ public static INodeSection.INodeFile.Builder buildINodeFile( .setPermission(buildPermissionStatus(file, state.getStringMap())) .setPreferredBlockSize(file.getPreferredBlockSize()) .setReplication(file.getFileReplication()) - .setStoragePolicyID(file.getLocalStoragePolicyID()); + .setStoragePolicyID(file.getLocalStoragePolicyID()) + .setIsStriped(file.isStriped()); AclFeature f = file.getAclFeature(); if (f != null) { @@ -632,8 +653,9 @@ private void save(OutputStream out, INodeDirectory n) throws IOException { private void save(OutputStream out, INodeFile n) throws IOException { INodeSection.INodeFile.Builder b = buildINodeFile(n, parent.getSaverContext()); + BlockInfo[] blocks = n.getBlocks(); - if (n.getBlocks() != null) { + if (blocks != null) { for (Block block : n.getBlocks()) { b.addBlocks(PBHelperClient.convert(block)); } @@ -667,7 +689,7 @@ private void save(OutputStream out, INodeSymlink n) throws IOException { 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())); 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 7c8a857c89640..ef0cc1dd7ba4a 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 @@ -297,7 +297,11 @@ private void loadNameSystemSection(InputStream in) throws IOException { 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()) { @@ -549,7 +553,8 @@ private void saveNameSystemSection(FileSummary.Builder summary) .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 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 96e4ecb12e00f..419a704d9031c 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 @@ -33,7 +33,6 @@ import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CachePoolInfo; import org.apache.hadoop.hdfs.protocol.LayoutVersion; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BlockUCState; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotFSImageFormat; @@ -124,22 +123,25 @@ static INodeFile readINodeUnderConstruction( short blockReplication = in.readShort(); long modificationTime = in.readLong(); long preferredBlockSize = in.readLong(); - + int numBlocks = in.readInt(); - BlockInfo[] blocks = new BlockInfo[numBlocks]; + + final BlockInfoContiguous[] blocksContiguous = + new BlockInfoContiguous[numBlocks]; Block blk = new Block(); int i = 0; - for (; i < numBlocks-1; i++) { + for (; i < numBlocks - 1; i++) { blk.readFields(in); - blocks[i] = new BlockInfoContiguous(blk, blockReplication); + blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication); } // last block is UNDER_CONSTRUCTION if(numBlocks > 0) { blk.readFields(in); - blocks[i] = new BlockInfoContiguous(blk, blockReplication); - blocks[i].convertToBlockUnderConstruction(BlockUCState.UNDER_CONSTRUCTION, - null); + blocksContiguous[i] = new BlockInfoContiguous(blk, blockReplication); + blocksContiguous[i].convertToBlockUnderConstruction( + BlockUCState.UNDER_CONSTRUCTION, null); } + PermissionStatus perm = PermissionStatus.read(in); String clientName = readString(in); String clientMachine = readString(in); @@ -152,7 +154,7 @@ static INodeFile readINodeUnderConstruction( // 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); + modificationTime, blocksContiguous, blockReplication, preferredBlockSize); file.toUnderConstruction(clientName, clientMachine); return file; } @@ -305,7 +307,7 @@ private static void writeINodeReference(INodeReference ref, DataOutput out, if (!isWithName) { Preconditions.checkState(ref instanceof INodeReference.DstReference); // dst snapshot id - out.writeInt(((INodeReference.DstReference) ref).getDstSnapshotId()); + out.writeInt(ref.getDstSnapshotId()); } else { out.writeInt(((INodeReference.WithName) ref).getLastSnapshotId()); } 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 4dda27dfbd09d..a94b61ce8b554 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 @@ -57,9 +57,9 @@ import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_EDIT_LOG_AUTOROLL_MULTIPLIER_THRESHOLD_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_ENABLE_RETRY_CACHE_KEY; +import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_LAZY_PERSIST_FILE_SCRUB_INTERVAL_SEC_DEFAULT; -import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_INODE_ATTRIBUTES_PROVIDER_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_DEFAULT; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_MAX_OBJECTS_KEY; import static org.apache.hadoop.hdfs.DFSConfigKeys.DFS_NAMENODE_NAME_DIR_KEY; @@ -134,6 +134,7 @@ import org.apache.hadoop.crypto.CryptoProtocolVersion; import org.apache.hadoop.crypto.key.KeyProvider.Metadata; import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension; +import org.apache.hadoop.crypto.key.KeyProviderCryptoExtension.EncryptedKeyVersion; import org.apache.hadoop.fs.BatchedRemoteIterator.BatchedListEntries; import org.apache.hadoop.fs.CacheFlag; import org.apache.hadoop.fs.ContentSummary; @@ -144,6 +145,7 @@ import org.apache.hadoop.fs.InvalidPathException; import org.apache.hadoop.fs.Options; import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.fs.UnresolvedLinkException; import org.apache.hadoop.fs.XAttr; import org.apache.hadoop.fs.XAttrSetFlag; @@ -152,10 +154,8 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.fs.permission.PermissionStatus; -import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.ha.HAServiceProtocol.HAServiceState; import org.apache.hadoop.ha.ServiceFailedException; -import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DFSUtil; import org.apache.hadoop.hdfs.HAUtil; @@ -163,6 +163,7 @@ import org.apache.hadoop.hdfs.UnknownCryptoProtocolVersionException; import org.apache.hadoop.hdfs.protocol.AlreadyBeingCreatedException; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; import org.apache.hadoop.hdfs.protocol.CacheDirectiveEntry; import org.apache.hadoop.hdfs.protocol.CacheDirectiveInfo; import org.apache.hadoop.hdfs.protocol.CachePoolEntry; @@ -171,15 +172,17 @@ 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.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.HdfsConstants; -import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.protocol.LastBlockWithStatus; 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.RecoveryInProgressException; import org.apache.hadoop.hdfs.protocol.RollingUpgradeException; import org.apache.hadoop.hdfs.protocol.RollingUpgradeInfo; @@ -194,6 +197,8 @@ 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.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockUnderConstructionFeature; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; @@ -409,6 +414,7 @@ private void logAuditEvent(boolean succeeded, private final BlockManager blockManager; private final SnapshotManager snapshotManager; private final CacheManager cacheManager; + private final ErasureCodingPolicyManager ecPolicyManager; private final DatanodeStatistics datanodeStatistics; private String nameserviceId; @@ -582,6 +588,7 @@ void clear() { leaseManager.removeAllLeases(); snapshotManager.clearSnapshottableDirs(); cacheManager.clear(); + ecPolicyManager.clear(); setImageLoaded(false); blockManager.clear(); } @@ -819,6 +826,7 @@ static FSNamesystem loadFromDisk(Configuration conf) throws IOException { this.dir = new FSDirectory(this, conf); this.snapshotManager = new SnapshotManager(dir); this.cacheManager = new CacheManager(this, conf, blockManager); + this.ecPolicyManager = new ErasureCodingPolicyManager(); this.safeMode = new SafeModeInfo(conf); this.topConf = new TopConf(conf); this.auditLoggers = initAuditLoggers(conf); @@ -1740,8 +1748,14 @@ LocatedBlocks getBlockLocations(String clientMachine, String srcArg, LocatedBlocks blocks = res.blocks; if (blocks != null) { + List 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(); @@ -2076,7 +2090,19 @@ private HdfsFileStatus startFileInt(final String src, if (!DFSUtil.isValidName(src)) { throw new InvalidPathException(src); } - blockManager.verifyReplication(src, replication, clientMachine); + + checkOperation(OperationCategory.READ); + readLock(); + try { + checkOperation(OperationCategory.READ); + if (!FSDirErasureCodingOp.hasErasureCodingPolicy(this, src)) { + blockManager.verifyReplication(src, replication, clientMachine); + } + } finally { + readUnlock(); + } + + checkOperation(OperationCategory.WRITE); if (blockSize < minBlockSize) { throw new IOException("Specified block size is less than configured" + " minimum value (" + DFSConfigKeys.DFS_NAMENODE_MIN_BLOCK_SIZE_KEY @@ -2418,6 +2444,7 @@ LocatedBlock getAdditionalDatanode(String src, long fileId, final long preferredblocksize; final byte storagePolicyID; final List chosen; + final boolean isStriped; checkOperation(OperationCategory.READ); byte[][] pathComponents = FSDirectory.getPathComponentsForReservedPath(src); FSPermissionChecker pc = getPermissionChecker(); @@ -2444,6 +2471,7 @@ LocatedBlock getAdditionalDatanode(String src, long fileId, clientnode = blockManager.getDatanodeManager().getDatanodeByHost(clientMachine); preferredblocksize = file.getPreferredBlockSize(); storagePolicyID = file.getStoragePolicyID(); + isStriped = file.isStriped(); //find datanode storages final DatanodeManager dm = blockManager.getDatanodeManager(); @@ -2459,7 +2487,7 @@ LocatedBlock getAdditionalDatanode(String src, long fileId, // choose new datanodes. final DatanodeStorageInfo[] targets = blockManager.chooseTarget4AdditionalDatanode( src, numAdditionalNodes, clientnode, chosen, - excludes, preferredblocksize, storagePolicyID); + excludes, preferredblocksize, storagePolicyID, isStriped); final LocatedBlock lb = BlockManager.newLocatedBlock( blk, targets, -1, false); blockManager.setBlockToken(lb, BlockTokenIdentifier.AccessMode.COPY); @@ -2553,15 +2581,20 @@ boolean completeFile(final String src, String holder, writeUnlock(); } getEditLog().logSync(); + if (success) { + NameNode.stateChangeLog.info("DIR* completeFile: " + src + + " is closed by " + holder); + } return success; } /** * Create new block with a unique block id and a new generation stamp. + * @param isStriped is the file under striping or contiguous layout? */ - Block createNewBlock() throws IOException { + Block createNewBlock(boolean isStriped) throws IOException { assert hasWriteLock(); - Block b = new Block(nextBlockId(), 0, 0); + Block b = new Block(nextBlockId(isStriped), 0, 0); // Increment the generation stamp for every new block. b.setGenerationStamp(nextGenerationStamp(false)); return b; @@ -2755,7 +2788,7 @@ void removeBlocksAndUpdateSafemodeTotal(BlocksMapUpdateInfo blocks) { if (trackBlockCounts) { if (b.isComplete()) { numRemovedComplete++; - if (blockManager.checkMinReplication(b)) { + if (blockManager.hasMinStorage(b, b.numNodes())) { numRemovedSafe++; } } @@ -2987,7 +3020,7 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, curBlock = blocks[nrCompleteBlocks]; if(!curBlock.isComplete()) break; - assert blockManager.checkMinReplication(curBlock) : + assert blockManager.hasMinStorage(curBlock) : "A COMPLETE block is not minimally replicated in " + src; } @@ -3022,8 +3055,8 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, 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.hasMinStorage(penultimateBlock); switch(lastBlockState) { case COMPLETE: @@ -3031,8 +3064,8 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, break; case COMMITTED: // Close file if committed blocks are minimally replicated - if(penultimateBlockMinReplication && - blockManager.checkMinReplication(lastBlock)) { + if(penultimateBlockMinStorage && + blockManager.hasMinStorage(lastBlock)) { finalizeINodeFileUnderConstruction(src, pendingFile, iip.getLatestSnapshotId()); NameNode.stateChangeLog.warn("BLOCK*" @@ -3067,8 +3100,8 @@ boolean internalReleaseLease(Lease lease, String src, INodesInPath iip, // setup the last block locations from the blockManager if not known if (uc.getNumExpectedLocations() == 0) { - uc.setExpectedLocations(lastBlock, - blockManager.getStorages(lastBlock)); + uc.setExpectedLocations(lastBlock, blockManager.getStorages(lastBlock), + lastBlock.isStriped()); } if (uc.getNumExpectedLocations() == 0 && lastBlock.getNumBytes() == 0) { @@ -3133,10 +3166,31 @@ void commitOrCompleteLastBlock( } // Adjust disk space consumption if required - final long diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes(); + final long diff; + final short replicationFactor; + if (fileINode.isStriped()) { + final ErasureCodingPolicy ecPolicy = FSDirErasureCodingOp + .getErasureCodingPolicy(this, iip); + final short numDataUnits = (short) ecPolicy.getNumDataUnits(); + final short numParityUnits = (short) ecPolicy.getNumParityUnits(); + + final long numBlocks = numDataUnits + numParityUnits; + final long fullBlockGroupSize = + fileINode.getPreferredBlockSize() * numBlocks; + + final BlockInfoStriped striped = new BlockInfoStriped(commitBlock, + ecPolicy); + final long actualBlockGroupSize = striped.spaceConsumed(); + + diff = fullBlockGroupSize - actualBlockGroupSize; + replicationFactor = (short) 1; + } else { + diff = fileINode.getPreferredBlockSize() - commitBlock.getNumBytes(); + replicationFactor = fileINode.getFileReplication(); + } if (diff > 0) { try { - dir.updateSpaceConsumed(iip, 0, -diff, fileINode.getFileReplication()); + dir.updateSpaceConsumed(iip, 0, -diff, replicationFactor); } catch (IOException e) { LOG.warn("Unexpected exception while updating disk space.", e); } @@ -3286,7 +3340,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, } truncatedBlock = iFile.getLastBlock(); - long recoveryId = truncatedBlock.getUnderConstructionFeature() + final long recoveryId = truncatedBlock.getUnderConstructionFeature() .getBlockRecoveryId(); copyTruncate = truncatedBlock.getBlockId() != storedBlock.getBlockId(); if(recoveryId != newgenerationstamp) { @@ -3301,8 +3355,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, if (remove) { blockManager.removeBlock(storedBlock); } - } - else { + } else { // update last block if(!copyTruncate) { storedBlock.setGenerationStamp(newgenerationstamp); @@ -3336,9 +3389,9 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, trimmedTargets.get(i).getStorageInfo(trimmedStorages.get(i)); if (storageInfo != null) { if(copyTruncate) { - storageInfo.addBlock(truncatedBlock); + storageInfo.addBlock(truncatedBlock, truncatedBlock); } else { - storageInfo.addBlock(storedBlock); + storageInfo.addBlock(storedBlock, storedBlock); } } } @@ -3354,8 +3407,9 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, } else { iFile.convertLastBlockToUC(storedBlock, trimmedStorageInfos); if (closeFile) { - blockManager.markBlockReplicasAsCorrupt(storedBlock, - oldGenerationStamp, oldNumBytes, trimmedStorageInfos); + blockManager.markBlockReplicasAsCorrupt(oldBlock.getLocalBlock(), + storedBlock, oldGenerationStamp, oldNumBytes, + trimmedStorageInfos); } } } @@ -3363,7 +3417,7 @@ void commitBlockSynchronization(ExtendedBlock oldBlock, if (closeFile) { if(copyTruncate) { src = closeFileCommitBlocks(iFile, truncatedBlock); - if(!iFile.isBlockInLatestSnapshot(storedBlock)) { + if(!iFile.isBlockInLatestSnapshot((BlockInfoContiguous) storedBlock)) { blockManager.removeBlock(storedBlock); } } else { @@ -4353,10 +4407,16 @@ private synchronized void setBlockTotal(int total) { /** * 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).getRealDataBlockNum() : safeReplication; + if (storageNum == safe) { this.blockSafe++; // Report startup progress only if we haven't completed startup yet. @@ -4631,12 +4691,12 @@ public boolean isInStartupSafeMode() { } @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 @@ -5136,11 +5196,13 @@ long nextGenerationStamp(boolean legacyBlock) /** * Increments, logs and then returns the block ID + * @param isStriped is the file under striping or contiguous layout? */ - private long nextBlockId() throws IOException { + private long nextBlockId(boolean isStriped) throws IOException { assert hasWriteLock(); checkNameNodeSafeMode("Cannot get next block ID"); - final long blockId = blockIdManager.nextBlockId(); + final long blockId = isStriped ? + blockIdManager.nextStripedBlockId() : blockIdManager.nextContiguousBlockId(); getEditLog().logAllocateBlockId(blockId); // NB: callers sync the log return blockId; @@ -5245,29 +5307,32 @@ void reportBadBlocks(LocatedBlock[] blocks) throws IOException { * Get a new generation stamp together with an access token for * a block under construction * - * This method is called for recovering a failed pipeline or setting up - * a pipeline to append to a block. + * This method is called for recovering a failed write or setting up + * a block for appended. * * @param block a block * @param clientName the name of a client * @return a located block with a new generation stamp and an access token * @throws IOException if any error occurs */ - LocatedBlock updateBlockForPipeline(ExtendedBlock block, + LocatedBlock bumpBlockGenerationStamp(ExtendedBlock block, String clientName) throws IOException { - LocatedBlock locatedBlock; + final LocatedBlock locatedBlock; checkOperation(OperationCategory.WRITE); writeLock(); try { checkOperation(OperationCategory.WRITE); // check vadility of parameters - checkUCBlock(block, clientName); + final INodeFile file = checkUCBlock(block, clientName); // get a new generation stamp and an access token block.setGenerationStamp(nextGenerationStamp(blockIdManager.isLegacyBlock(block.getLocalBlock()))); - locatedBlock = new LocatedBlock(block, new DatanodeInfo[0]); - blockManager.setBlockToken(locatedBlock, BlockTokenIdentifier.AccessMode.WRITE); + + locatedBlock = BlockManager.newLocatedBlock( + block, file.getLastBlock(), null, -1); + blockManager.setBlockToken(locatedBlock, + BlockTokenIdentifier.AccessMode.WRITE); } finally { writeUnlock(); } @@ -5319,28 +5384,34 @@ private void updatePipelineInternal(String clientName, ExtendedBlock oldBlock, assert hasWriteLock(); // check the vadility of the block and lease holder name final INodeFile pendingFile = checkUCBlock(oldBlock, clientName); - final BlockInfo blockinfo = pendingFile.getLastBlock(); - assert !blockinfo.isComplete(); + final BlockInfo lastBlock = pendingFile.getLastBlock(); + assert !lastBlock.isComplete(); // check new GS & length: this is not expected - if (newBlock.getGenerationStamp() <= blockinfo.getGenerationStamp() || - newBlock.getNumBytes() < blockinfo.getNumBytes()) { - String msg = "Update " + oldBlock + " (len = " + - blockinfo.getNumBytes() + ") to an older state: " + newBlock + - " (len = " + newBlock.getNumBytes() +")"; + if (newBlock.getGenerationStamp() <= lastBlock.getGenerationStamp()) { + final String msg = "Update " + oldBlock + " but the new block " + newBlock + + " does not have a larger generation stamp than the last block " + + lastBlock; + LOG.warn(msg); + throw new IOException(msg); + } + if (newBlock.getNumBytes() < lastBlock.getNumBytes()) { + final String msg = "Update " + oldBlock + " (size=" + + oldBlock.getNumBytes() + ") to a smaller size block " + newBlock + + " (size=" + newBlock.getNumBytes() + ")"; LOG.warn(msg); throw new IOException(msg); } // Update old block with the new generation stamp and new length - blockinfo.setNumBytes(newBlock.getNumBytes()); - blockinfo.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); + lastBlock.setNumBytes(newBlock.getNumBytes()); + lastBlock.setGenerationStampAndVerifyReplicas(newBlock.getGenerationStamp()); // find the DatanodeDescriptor objects final DatanodeStorageInfo[] storages = blockManager.getDatanodeManager() .getDatanodeStorageInfos(newNodes, newStorageIDs); - blockinfo.getUnderConstructionFeature().setExpectedLocations( - blockinfo, storages); + lastBlock.getUnderConstructionFeature().setExpectedLocations(lastBlock, + storages, lastBlock.isStriped()); String src = pendingFile.getFullPathName(); FSDirWriteFileOp.persistBlocks(dir, src, pendingFile, logRetryCache); @@ -6110,17 +6181,24 @@ public BlockIdManager getBlockIdManager() { public FSDirectory getFSDirectory() { return dir; } + /** Set the FSDirectory. */ @VisibleForTesting public void setFSDirectory(FSDirectory dir) { this.dir = dir; } + /** @return the cache manager. */ @Override public CacheManager getCacheManager() { return cacheManager; } + /** @return the ErasureCodingPolicyManager. */ + public ErasureCodingPolicyManager getErasureCodingPolicyManager() { + return ecPolicyManager; + } + @Override public HAContext getHAContext() { return haContext; @@ -7024,6 +7102,68 @@ BatchedListEntries listEncryptionZones(long prevId) } } + /** + * Set an erasure coding policy on the given path. + * @param srcArg The path of the target directory. + * @param ecPolicy The erasure coding policy to set on the target directory. + * @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 setErasureCodingPolicy(final String srcArg, final ErasureCodingPolicy + ecPolicy, final boolean logRetryCache) throws IOException, + UnresolvedLinkException, SafeModeException, AccessControlException { + checkSuperuserPrivilege(); + checkOperation(OperationCategory.WRITE); + HdfsFileStatus resultingStat = null; + boolean success = false; + writeLock(); + try { + checkOperation(OperationCategory.WRITE); + checkNameNodeSafeMode("Cannot set erasure coding policy on " + srcArg); + resultingStat = FSDirErasureCodingOp.setErasureCodingPolicy(this, + srcArg, ecPolicy, logRetryCache); + success = true; + } finally { + writeUnlock(); + if (success) { + getEditLog().logSync(); + } + logAuditEvent(success, "setErasureCodingPolicy", srcArg, null, + resultingStat); + } + } + + /** + * Get the erasure coding policy information for specified path + */ + ErasureCodingPolicy getErasureCodingPolicy(String src) + throws AccessControlException, UnresolvedLinkException, IOException { + checkOperation(OperationCategory.READ); + readLock(); + try { + checkOperation(OperationCategory.READ); + return getErasureCodingPolicyForPath(src); + } finally { + readUnlock(); + } + } + + /** + * Get available erasure coding polices + */ + ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { + checkOperation(OperationCategory.READ); + waitForLoadingFSImage(); + readLock(); + try { + checkOperation(OperationCategory.READ); + return FSDirErasureCodingOp.getErasureCodingPolicies(this); + } finally { + readUnlock(); + } + } + void setXAttr(String src, XAttr xAttr, EnumSet flag, boolean logRetryCache) throws IOException { @@ -7205,7 +7345,6 @@ private static void enableAsyncAuditLog() { logger.addAppender(asyncAppender); } } - /** * Return total number of Sync Operations on FSEditLog. */ @@ -7230,5 +7369,11 @@ public String getTotalSyncTimes() { return ""; } } + + @Override + public ErasureCodingPolicy getErasureCodingPolicyForPath(String src) + throws IOException { + return FSDirErasureCodingOp.getErasureCodingPolicy(this, src); + } } 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 8565522507586..242c456bdb515 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 @@ -34,9 +34,11 @@ import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; 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.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; import org.apache.hadoop.hdfs.server.blockmanagement.BlockStoragePolicySuite; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -79,12 +81,14 @@ public static INodeFile valueOf(INode inode, String path, boolean acceptNull) /** * Bit format: - * [4-bit storagePolicyID][12-bit replication][48-bit preferredBlockSize] + * [4-bit storagePolicyID][1-bit isStriped] + * [11-bit replication][48-bit preferredBlockSize] */ enum HeaderFormat { PREFERRED_BLOCK_SIZE(null, 48, 1), - REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 12, 1), - STORAGE_POLICY_ID(REPLICATION.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH, + REPLICATION(PREFERRED_BLOCK_SIZE.BITS, 11, 0), + IS_STRIPED(REPLICATION.BITS, 1, 0), + STORAGE_POLICY_ID(IS_STRIPED.BITS, BlockStoragePolicySuite.ID_BIT_LENGTH, 0); private final LongBitFormat BITS; @@ -105,14 +109,27 @@ static byte getStoragePolicyID(long header) { return (byte)STORAGE_POLICY_ID.BITS.retrieve(header); } + static boolean isStriped(long header) { + long isStriped = IS_STRIPED.BITS.retrieve(header); + Preconditions.checkState(isStriped == 0 || isStriped == 1); + return isStriped == 1; + } + static long toLong(long preferredBlockSize, short replication, - byte storagePolicyID) { + boolean isStriped, byte storagePolicyID) { long h = 0; if (preferredBlockSize == 0) { preferredBlockSize = PREFERRED_BLOCK_SIZE.BITS.getMin(); } h = PREFERRED_BLOCK_SIZE.BITS.combine(preferredBlockSize, h); - h = REPLICATION.BITS.combine(replication, h); + // Replication factor for striped files is zero + if (isStriped) { + h = REPLICATION.BITS.combine(0L, h); + h = IS_STRIPED.BITS.combine(1L, h); + } else { + h = REPLICATION.BITS.combine(replication, h); + h = IS_STRIPED.BITS.combine(0L, h); + } h = STORAGE_POLICY_ID.BITS.combine(storagePolicyID, h); return h; } @@ -127,15 +144,21 @@ static long toLong(long preferredBlockSize, short replication, long atime, BlockInfo[] blklist, short replication, long preferredBlockSize) { this(id, name, permissions, mtime, atime, blklist, replication, - preferredBlockSize, (byte) 0); + preferredBlockSize, (byte) 0, false); } INodeFile(long id, byte[] name, PermissionStatus permissions, long mtime, long atime, BlockInfo[] blklist, short replication, - long preferredBlockSize, byte storagePolicyID) { + long preferredBlockSize, byte storagePolicyID, boolean isStriped) { super(id, name, permissions, mtime, atime); - header = HeaderFormat.toLong(preferredBlockSize, replication, storagePolicyID); - this.blocks = blklist; + header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped, + storagePolicyID); + if (blklist != null && blklist.length > 0) { + for (BlockInfo b : blklist) { + Preconditions.checkArgument(b.isStriped() == isStriped); + } + } + setBlocks(blklist); } public INodeFile(INodeFile that) { @@ -227,6 +250,7 @@ private void assertAllBlocksComplete() { @Override // BlockCollection public void setBlock(int index, BlockInfo blk) { + Preconditions.checkArgument(blk.isStriped() == this.isStriped()); this.blocks[index] = blk; } @@ -235,7 +259,6 @@ 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."); } @@ -350,6 +373,7 @@ public final short getFileReplication(int snapshot) { /** The same as getFileReplication(null). */ @Override // INodeFileAttributes + // TODO properly handle striped files public final short getFileReplication() { return getFileReplication(CURRENT_STATE_ID); } @@ -364,7 +388,13 @@ public short getPreferredBlockReplication() { } max = maxInSnapshot > max ? maxInSnapshot : max; } - return max; + if(!isStriped()){ + return max; + } + // TODO support more policies based on policyId + ErasureCodingPolicy ecPolicy = + ErasureCodingPolicyManager.getSystemDefaultPolicy(); + return (short) (ecPolicy.getNumDataUnits() + ecPolicy.getNumParityUnits()); } /** Set the replication factor of this file. */ @@ -412,6 +442,29 @@ public final void setStoragePolicyID(byte storagePolicyId, setStoragePolicyID(storagePolicyId); } + + /** + * @return The ID of the erasure coding policy on the file. 0 represents no + * EC policy (file is in contiguous format). 1 represents the system + * default EC policy: + * {@link ErasureCodingPolicyManager#SYS_DEFAULT_POLICY}. + * TODO: support more policies by reusing {@link HeaderFormat#REPLICATION}. + */ + @VisibleForTesting + @Override + public byte getErasureCodingPolicyID() { + return isStriped() ? (byte)1 : (byte)0; + } + + /** + * @return true if the file is in the striping layout. + */ + @VisibleForTesting + @Override + public boolean isStriped() { + return HeaderFormat.isStriped(header); + } + @Override // INodeFileAttributes public long getHeaderLong() { return header; @@ -425,10 +478,12 @@ public BlockInfo[] getBlocks() { /** @return blocks of the file corresponding to the snapshot. */ public BlockInfo[] getBlocks(int snapshot) { - if(snapshot == CURRENT_STATE_ID || getDiffs() == null) { + if (snapshot == CURRENT_STATE_ID || getDiffs() == null) { return getBlocks(); } + // find blocks stored in snapshot diffs (for truncate) FileDiff diff = getDiffs().getDiffById(snapshot); + // note that currently FileDiff can only store contiguous blocks BlockInfo[] snapshotBlocks = diff == null ? getBlocks() : diff.getBlocks(); if (snapshotBlocks != null) { return snapshotBlocks; @@ -446,6 +501,7 @@ void concatBlocks(INodeFile[] inodes, BlockManager bm) { int size = this.blocks.length; int totalAddedBlocks = 0; for(INodeFile f : inodes) { + Preconditions.checkState(f.isStriped() == this.isStriped()); totalAddedBlocks += f.blocks.length; } @@ -473,6 +529,7 @@ void concatBlocks(INodeFile[] inodes, BlockManager bm) { * add a block to the block list */ void addBlock(BlockInfo newblock) { + Preconditions.checkArgument(newblock.isStriped() == this.isStriped()); if (this.blocks == null) { this.setBlocks(new BlockInfo[]{newblock}); } else { @@ -586,6 +643,10 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, final long ssDeltaNoReplication; short replication; + if (isStriped()) { + return computeQuotaUsageWithStriped(bsp, counts); + } + if (last < lastSnapshotId) { ssDeltaNoReplication = computeFileSize(true, false); replication = getFileReplication(); @@ -608,6 +669,18 @@ public final QuotaCounts computeQuotaUsage(BlockStoragePolicySuite bsps, return counts; } + /** + * Compute quota of striped file. Note that currently EC files do not support + * append/hflush/hsync, thus the file length recorded in snapshots should be + * the same with the current file length. + */ + public final QuotaCounts computeQuotaUsageWithStriped( + BlockStoragePolicy bsp, QuotaCounts counts) { + counts.addNameSpace(1); + counts.add(storagespaceConsumed(bsp)); + return counts; + } + @Override public final ContentSummaryComputationContext computeContentSummary( int snapshotId, final ContentSummaryComputationContext summary) { @@ -678,17 +751,21 @@ public final long computeFileSize(boolean includesLastUcBlock, return 0; } final int last = blocks.length - 1; - //check if the last block is under-construction - long size = blocks[last].getNumBytes(); - if (!blocks[last].isComplete()) { + //check if the last block is BlockInfoUnderConstruction + BlockInfo lastBlk = blocks[last]; + long size = lastBlk.getNumBytes(); + if (!lastBlk.isComplete()) { if (!includesLastUcBlock) { size = 0; } else if (usePreferredBlockSize4LastUcBlock) { - size = getPreferredBlockSize(); + size = isStriped()? + getPreferredBlockSize() * + ((BlockInfoStriped)lastBlk).getDataBlockNum() : + getPreferredBlockSize(); } } //sum other blocks - for(int i = 0; i < last; i++) { + for (int i = 0; i < last; i++) { size += blocks[i].getNumBytes(); } return size; @@ -700,6 +777,32 @@ public final long computeFileSize(boolean includesLastUcBlock, * Use preferred block size for the last block if it is under construction. */ public final QuotaCounts storagespaceConsumed(BlockStoragePolicy bsp) { + if (isStriped()) { + return storagespaceConsumedStriped(); + } else { + return storagespaceConsumedContiguous(bsp); + } + } + + // TODO: support EC with heterogeneous storage + public final QuotaCounts storagespaceConsumedStriped() { + QuotaCounts counts = new QuotaCounts.Builder().build(); + if (blocks == null || blocks.length == 0) { + return counts; + } + + for (BlockInfo b : blocks) { + Preconditions.checkState(b.isStriped()); + long blockSize = b.isComplete() ? + ((BlockInfoStriped)b).spaceConsumed() : getPreferredBlockSize() * + ((BlockInfoStriped)b).getTotalBlockNum(); + counts.addStorageSpace(blockSize); + } + return counts; + } + + public final QuotaCounts storagespaceConsumedContiguous( + BlockStoragePolicy bsp) { QuotaCounts counts = new QuotaCounts.Builder().build(); final Iterable blocks; FileWithSnapshotFeature sf = getFileWithSnapshotFeature(); @@ -803,6 +906,7 @@ public long collectBlocksBeyondMax(final long max, /** * compute the quota usage change for a truncate op * @param newLength the length for truncation + * TODO: properly handle striped blocks (HDFS-7622) **/ void computeQuotaDeltaForTruncate( long newLength, BlockStoragePolicy bsps, @@ -866,8 +970,15 @@ void truncateBlocksTo(int n) { setBlocks(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(BlockInfo[] snapshotBlocks, BlocksMapUpdateInfo collectedBlocks) { + Preconditions.checkState(!isStriped()); BlockInfo[] oldBlocks = getBlocks(); if(snapshotBlocks == null || oldBlocks == null) return; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java index 204c8ac6a8b67..8cd481a68e063 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/namenode/INodeFileAttributes.java @@ -20,7 +20,7 @@ import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.fs.permission.PermissionStatus; import org.apache.hadoop.hdfs.server.namenode.INodeFile.HeaderFormat; -import org.apache.hadoop.hdfs.server.namenode.XAttrFeature; + /** * The attributes of a file. */ @@ -29,6 +29,12 @@ public interface INodeFileAttributes extends INodeAttributes { /** @return the file replication. */ public short getFileReplication(); + /** @return whether the file is striped (instead of contiguous) */ + public boolean isStriped(); + + /** @return whether the file is striped (instead of contiguous) */ + public byte getErasureCodingPolicyID(); + /** @return preferred block size in bytes */ public long getPreferredBlockSize(); @@ -47,10 +53,10 @@ public static class SnapshotCopy extends INodeAttributes.SnapshotCopy public SnapshotCopy(byte[] name, PermissionStatus permissions, AclFeature aclFeature, long modificationTime, long accessTime, short replication, long preferredBlockSize, - byte storagePolicyID, XAttrFeature xAttrsFeature) { + byte storagePolicyID, XAttrFeature xAttrsFeature, boolean isStriped) { super(name, permissions, aclFeature, modificationTime, accessTime, xAttrsFeature); - header = HeaderFormat.toLong(preferredBlockSize, replication, + header = HeaderFormat.toLong(preferredBlockSize, replication, isStriped, storagePolicyID); } @@ -69,6 +75,16 @@ public short getFileReplication() { return HeaderFormat.getReplication(header); } + @Override + public boolean isStriped() { + return HeaderFormat.isStriped(header); + } + + @Override + public byte getErasureCodingPolicyID() { + return isStriped() ? (byte)1 : (byte)0; + } + @Override public long getPreferredBlockSize() { return HeaderFormat.getPreferredBlockSize(header); 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 908af45a105c2..cec9313819c1f 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 @@ -121,8 +121,8 @@ synchronized long getNumUnderConstructionBlocks() { for(BlockInfo b : blocks) { if(!b.isComplete()) numUCBlocks++; + } } - } LOG.info("Number of blocks under construction: " + numUCBlocks); return numUCBlocks; } 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 3a5dc12a842ae..2943fc2d7ef99 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 @@ -87,7 +87,8 @@ public static enum Feature implements LayoutFeature { BLOCK_STORAGE_POLICY(-60, -60, "Block Storage policy"), TRUNCATE(-61, -61, "Truncate"), APPEND_NEW_BLOCK(-62, -61, "Support appending to new block"), - QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types"); + QUOTA_BY_STORAGE_TYPE(-63, -61, "Support quota for specific storage types"), + ERASURE_CODING(-64, -61, "Support erasure coding"); private final FeatureInfo info; 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 79a3773a7dbab..e9363b44bb30a 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.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; import org.apache.hadoop.hdfs.protocol.DirectoryListing; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.protocol.EncryptionZone; import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.FSLimitException; @@ -798,7 +799,7 @@ public void reportBadBlocks(LocatedBlock[] blocks) throws IOException { public LocatedBlock updateBlockForPipeline(ExtendedBlock block, String clientName) throws IOException { checkNNStartup(); - return namesystem.updateBlockForPipeline(block, clientName); + return namesystem.bumpBlockGenerationStamp(block, clientName); } @@ -1425,8 +1426,8 @@ public void refreshServiceAcl() throws IOException { @Override // RefreshAuthorizationPolicyProtocol public void refreshUserToGroupsMappings() throws IOException { - LOG.info("Refreshing all user-to-groups mappings. Requested by user: " + - getRemoteUser().getShortUserName()); + LOG.info("Refreshing all user-to-groups mappings. Requested by user: " + + getRemoteUser().getShortUserName()); Groups.getUserToGroupsMappingService().refresh(); } @@ -1559,7 +1560,7 @@ public String createSnapshot(String snapshotRoot, String snapshotName) } namesystem.checkOperation(OperationCategory.WRITE); CacheEntryWithPayload cacheEntry = RetryCache.waitForCompletion(retryCache, - null); + null); if (cacheEntry != null && cacheEntry.isSuccess()) { return (String) cacheEntry.getPayload(); } @@ -1850,6 +1851,23 @@ public BatchedEntries listEncryptionZones( return namesystem.listEncryptionZones(prevId); } + @Override // ClientProtocol + public void setErasureCodingPolicy(String src, ErasureCodingPolicy ecPolicy) + throws IOException { + checkNNStartup(); + final CacheEntry cacheEntry = RetryCache.waitForCompletion(retryCache); + if (cacheEntry != null && cacheEntry.isSuccess()) { + return; + } + boolean success = false; + try { + namesystem.setErasureCodingPolicy(src, ecPolicy, cacheEntry != null); + success = true; + } finally { + RetryCache.setState(cacheEntry, success); + } + } + @Override // ClientProtocol public void setXAttr(String src, XAttr xAttr, EnumSet flag) throws IOException { @@ -2045,4 +2063,16 @@ public void removeSpanReceiver(long id) throws IOException { namesystem.checkSuperuserPrivilege(); nn.tracerConfigurationManager.removeSpanReceiver(id); } + + @Override // ClientProtocol + public ErasureCodingPolicy[] getErasureCodingPolicies() throws IOException { + checkNNStartup(); + return namesystem.getErasureCodingPolicies(); + } + + @Override // ClientProtocol + public ErasureCodingPolicy getErasureCodingPolicy(String src) throws IOException { + checkNNStartup(); + return namesystem.getErasureCodingPolicy(src); + } } 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 6d4089822bb40..c6a2118e26ba4 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 @@ -66,8 +66,9 @@ 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.BlockInfoStriped; import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; -import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicies; import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementStatus; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; @@ -124,6 +125,9 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { private final int totalDatanodes; private final InetAddress remoteAddress; + private long totalDirs = 0L; + private long totalSymlinks = 0L; + private String lostFound = null; private boolean lfInited = false; private boolean lfInitedOk = false; @@ -173,7 +177,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { private final PrintWriter out; private List snapshottableDirs = null; - private final BlockPlacementPolicy bpPolicy; + private final BlockPlacementPolicies bpPolicies; private StoragePolicySummary storageTypeSummary = null; /** @@ -195,7 +199,7 @@ public class NamenodeFsck implements DataEncryptionKeyFactory { this.out = out; this.totalDatanodes = totalDatanodes; this.remoteAddress = remoteAddress; - this.bpPolicy = BlockPlacementPolicy.getInstance(conf, null, + this.bpPolicies = new BlockPlacementPolicies(conf, null, networktopology, namenode.getNamesystem().getBlockManager().getDatanodeManager() .getHost2DatanodeMap()); @@ -258,7 +262,8 @@ public void blockIdCK(String blockId) { NumberReplicas numberReplicas= bm.countNodes(blockInfo); out.println("Block Id: " + blockId); out.println("Block belongs to: "+iNode.getFullPathName()); - out.println("No. of Expected Replica: " + blockInfo.getReplication()); + out.println("No. of Expected Replica: " + + bm.getExpectedReplicaNum(blockInfo)); out.println("No. of live Replica: " + numberReplicas.liveReplicas()); out.println("No. of excess Replica: " + numberReplicas.excessReplicas()); out.println("No. of stale Replica: " + @@ -359,13 +364,21 @@ public void fsck() { namenode.getNamesystem().getBlockManager().getStoragePolicies()); } - Result res = new Result(conf); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); - check(path, file, res); + check(path, file, replRes, ecRes); - out.println(res); - out.println(" Number of data-nodes:\t\t" + totalDatanodes); + out.print("\nStatus: "); + out.println(replRes.isHealthy() && ecRes.isHealthy() ? "HEALTHY" : "CORRUPT"); + out.println(" Number of data-nodes:\t" + totalDatanodes); out.println(" Number of racks:\t\t" + networktopology.getNumOfRacks()); + out.println(" Total dirs:\t\t\t" + totalDirs); + out.println(" Total symlinks:\t\t" + totalSymlinks); + out.println("\nReplicated Blocks:"); + out.println(replRes); + out.println("\nErasure Coded Block Groups:"); + out.println(ecRes); if (this.showStoragePolcies) { out.print(storageTypeSummary.toString()); @@ -385,7 +398,7 @@ public void fsck() { // of file system and return appropriate code. Changing the output // string might break testcases. Also note this must be the last line // of the report. - if (res.isHealthy()) { + if (replRes.isHealthy() && ecRes.isHealthy()) { out.print("\n\nThe filesystem under path '" + path + "' " + HEALTHY_STATUS); } else { out.print("\n\nThe filesystem under path '" + path + "' " + CORRUPT_STATUS); @@ -428,42 +441,49 @@ private void listCorruptFileBlocks() throws IOException { } @VisibleForTesting - void check(String parent, HdfsFileStatus file, Result res) throws IOException { + void check(String parent, HdfsFileStatus file, Result replRes, Result ecRes) + throws IOException { String path = file.getFullName(parent); if (file.isDir()) { - checkDir(path, res); + checkDir(path, replRes, ecRes); return; } if (file.isSymlink()) { if (showFiles) { out.println(path + " "); } - res.totalSymlinks++; + totalSymlinks++; return; } LocatedBlocks blocks = getBlockLocations(path, file); if (blocks == null) { // the file is deleted return; } - collectFileSummary(path, file, res, blocks); - collectBlocksSummary(parent, file, res, blocks); + + final Result r = file.getErasureCodingPolicy() != null ? ecRes: replRes; + collectFileSummary(path, file, r, blocks); + if (showprogress && (replRes.totalFiles + ecRes.totalFiles) % 100 == 0) { + out.println(); + out.flush(); + } + collectBlocksSummary(parent, file, r, blocks); } - private void checkDir(String path, Result res) throws IOException { + private void checkDir(String path, Result replRes, Result ecRes) throws IOException { if (snapshottableDirs != null && snapshottableDirs.contains(path)) { String snapshotPath = (path.endsWith(Path.SEPARATOR) ? path : path + Path.SEPARATOR) + HdfsConstants.DOT_SNAPSHOT_DIR; HdfsFileStatus snapshotFileInfo = namenode.getRpcServer().getFileInfo( snapshotPath); - check(snapshotPath, snapshotFileInfo, res); + check(snapshotPath, snapshotFileInfo, replRes, ecRes); } byte[] lastReturnedName = HdfsFileStatus.EMPTY_NAME; DirectoryListing thisListing; if (showFiles) { out.println(path + " "); } - res.totalDirs++; + totalDirs++; do { assert lastReturnedName != null; thisListing = namenode.getRpcServer().getListing( @@ -473,7 +493,7 @@ private void checkDir(String path, Result res) throws IOException { } HdfsFileStatus[] files = thisListing.getPartialListing(); for (int i = 0; i < files.length; i++) { - check(path, files[i], res); + check(path, files[i], replRes, ecRes); } lastReturnedName = thisListing.getLastName(); } while (thisListing.hasMore()); @@ -521,10 +541,6 @@ private void collectFileSummary(String path, HdfsFileStatus file, Result res, } else if (showprogress) { out.print('.'); } - if ((showprogress) && res.totalFiles % 100 == 0) { - out.println(); - out.flush(); - } } private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res, @@ -555,9 +571,10 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res final BlockInfo storedBlock = bm.getStoredBlock( block.getLocalBlock()); + final int minReplication = bm.getMinStorageNum(storedBlock); // count decommissionedReplicas / decommissioningReplicas NumberReplicas numberReplicas = bm.countNodes(storedBlock); - int decommissionedReplicas = numberReplicas.decommissioned();; + int decommissionedReplicas = numberReplicas.decommissioned(); int decommissioningReplicas = numberReplicas.decommissioning(); res.decommissionedReplicas += decommissionedReplicas; res.decommissioningReplicas += decommissioningReplicas; @@ -569,11 +586,18 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res res.totalReplicas += totalReplicasPerBlock; // count expected replicas - short targetFileReplication = file.getReplication(); + short targetFileReplication; + if (file.getErasureCodingPolicy() != null) { + assert storedBlock instanceof BlockInfoStriped; + targetFileReplication = ((BlockInfoStriped) storedBlock) + .getRealTotalBlockNum(); + } else { + targetFileReplication = file.getReplication(); + } res.numExpectedReplicas += targetFileReplication; // count under min repl'd blocks - if(totalReplicasPerBlock < res.minReplication){ + if(totalReplicasPerBlock < minReplication){ res.numUnderMinReplicatedBlocks++; } @@ -594,7 +618,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res } // count minimally replicated blocks - if (totalReplicasPerBlock >= res.minReplication) + if (totalReplicasPerBlock >= minReplication) res.numMinReplicatedBlocks++; // count missing replicas / under replicated blocks @@ -614,7 +638,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res } // count mis replicated blocks - BlockPlacementStatus blockPlacementStatus = bpPolicy + BlockPlacementStatus blockPlacementStatus = bpPolicies.getPolicy(false) .verifyBlockPlacement(path, lBlk, targetFileReplication); if (!blockPlacementStatus.isPlacementPolicySatisfied()) { res.numMisReplicatedBlocks++; @@ -649,9 +673,9 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res report.append(" Live_repl=" + liveReplicas); if (showLocations || showRacks || showReplicaDetails) { StringBuilder sb = new StringBuilder("["); - Iterable storages = bm.getStorages(block.getLocalBlock()); - for (Iterator iterator = storages.iterator(); iterator.hasNext();) { - DatanodeStorageInfo storage = iterator.next(); + DatanodeStorageInfo[] storages = bm.getStorages(storedBlock); + for (int i = 0; i < storages.length; i++) { + DatanodeStorageInfo storage = storages[i]; DatanodeDescriptor dnDesc = storage.getDatanodeDescriptor(); if (showRacks) { sb.append(NodeBase.getPath(dnDesc)); @@ -660,7 +684,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res .getStorageType())); } if (showReplicaDetails) { - LightWeightLinkedSet blocksExcess = + LightWeightLinkedSet blocksExcess = bm.excessReplicateMap.get(dnDesc.getDatanodeUuid()); Collection corruptReplicas = bm.getCorruptReplicas(block.getLocalBlock()); @@ -681,7 +705,7 @@ private void collectBlocksSummary(String parent, HdfsFileStatus file, Result res sb.append("LIVE)"); } } - if (iterator.hasNext()) { + if (i < storages.length - 1) { sb.append(", "); } } @@ -996,7 +1020,7 @@ static class Result { long missingReplicas = 0L; long decommissionedReplicas = 0L; long decommissioningReplicas = 0L; - long numUnderMinReplicatedBlocks=0L; + long numUnderMinReplicatedBlocks = 0L; long numOverReplicatedBlocks = 0L; long numUnderReplicatedBlocks = 0L; long numMisReplicatedBlocks = 0L; // blocks that do not satisfy block placement policy @@ -1006,22 +1030,10 @@ static class Result { long totalOpenFilesBlocks = 0L; long totalFiles = 0L; long totalOpenFiles = 0L; - long totalDirs = 0L; - long totalSymlinks = 0L; long totalSize = 0L; long totalOpenFilesSize = 0L; long totalReplicas = 0L; - final short replication; - final int minReplication; - - Result(Configuration conf) { - this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, - DFSConfigKeys.DFS_REPLICATION_DEFAULT); - this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, - DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT); - } - /** * DFS is considered healthy if there are no missing blocks. */ @@ -1047,19 +1059,29 @@ float getReplicationFactor() { return 0.0f; return (float) (totalReplicas) / (float) totalBlocks; } + } + + @VisibleForTesting + static class ReplicationResult extends Result { + final short replication; + final short minReplication; + + ReplicationResult(Configuration conf) { + this.replication = (short)conf.getInt(DFSConfigKeys.DFS_REPLICATION_KEY, + DFSConfigKeys.DFS_REPLICATION_DEFAULT); + this.minReplication = (short)conf.getInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_KEY, + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MIN_DEFAULT); + } @Override public String toString() { StringBuilder res = new StringBuilder(); - res.append("Status: ").append((isHealthy() ? "HEALTHY" : "CORRUPT")) - .append("\n Total size:\t").append(totalSize).append(" B"); + res.append(" Total size:\t").append(totalSize).append(" B"); if (totalOpenFilesSize != 0) { res.append(" (Total open files size: ").append(totalOpenFilesSize) .append(" B)"); } - res.append("\n Total dirs:\t").append(totalDirs).append( - "\n Total files:\t").append(totalFiles); - res.append("\n Total symlinks:\t\t").append(totalSymlinks); + res.append("\n Total files:\t").append(totalFiles); if (totalOpenFiles != 0) { res.append(" (Files currently being written: ").append(totalOpenFiles) .append(")"); @@ -1149,4 +1171,110 @@ public String toString() { return res.toString(); } } + + @VisibleForTesting + static class ErasureCodingResult extends Result { + final String defaultECPolicy; + + ErasureCodingResult(Configuration conf) { + defaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy() + .getName(); + } + + @Override + public String toString() { + StringBuilder res = new StringBuilder(); + res.append(" Total size:\t").append(totalSize).append(" B"); + if (totalOpenFilesSize != 0) { + res.append(" (Total open files size: ").append(totalOpenFilesSize) + .append(" B)"); + } + res.append("\n Total files:\t").append(totalFiles); + if (totalOpenFiles != 0) { + res.append(" (Files currently being written: ").append(totalOpenFiles) + .append(")"); + } + res.append("\n Total block groups (validated):\t").append(totalBlocks); + if (totalBlocks > 0) { + res.append(" (avg. block group size ").append((totalSize / totalBlocks)) + .append(" B)"); + } + if (totalOpenFilesBlocks != 0) { + res.append(" (Total open file block groups (not validated): ").append( + totalOpenFilesBlocks).append(")"); + } + if (corruptFiles > 0 || numUnderMinReplicatedBlocks > 0) { + res.append("\n ********************************"); + if(numUnderMinReplicatedBlocks>0){ + res.append("\n UNRECOVERABLE BLOCK GROUPS:\t").append(numUnderMinReplicatedBlocks); + if(totalBlocks>0){ + res.append(" (").append( + ((float) (numUnderMinReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + } + if(corruptFiles>0) { + res.append( + "\n CORRUPT FILES:\t").append(corruptFiles); + if (missingSize > 0) { + res.append("\n MISSING BLOCK GROUPS:\t").append(missingIds.size()).append( + "\n MISSING SIZE:\t\t").append(missingSize).append(" B"); + } + if (corruptBlocks > 0) { + res.append("\n CORRUPT BLOCK GROUPS: \t").append(corruptBlocks).append( + "\n CORRUPT SIZE:\t\t").append(corruptSize).append(" B"); + } + } + res.append("\n ********************************"); + } + res.append("\n Minimally erasure-coded block groups:\t").append( + numMinReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numMinReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Over-erasure-coded block groups:\t") + .append(numOverReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numOverReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Under-erasure-coded block groups:\t").append( + numUnderReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numUnderReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Unsatisfactory placement block groups:\t") + .append(numMisReplicatedBlocks); + if (totalBlocks > 0) { + res.append(" (").append( + ((float) (numMisReplicatedBlocks * 100) / (float) totalBlocks)) + .append(" %)"); + } + res.append("\n Default ecPolicy:\t\t").append(defaultECPolicy) + .append("\n Average block group size:\t").append( + getReplicationFactor()).append("\n Missing block groups:\t\t").append( + missingIds.size()).append("\n Corrupt block groups:\t\t").append( + corruptBlocks).append("\n Missing internal blocks:\t").append( + missingReplicas); + if (totalReplicas > 0) { + res.append(" (").append( + ((float) (missingReplicas * 100) / (float) numExpectedReplicas)).append( + " %)"); + } + if (decommissionedReplicas > 0) { + res.append("\n Decommissioned internal blocks:\t").append( + decommissionedReplicas); + } + if (decommissioningReplicas > 0) { + res.append("\n Decommissioning internal blocks:\t").append( + decommissioningReplicas); + } + return res.toString(); + } + } } 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 5bc4033e562b4..b1012c26e6db1 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 @@ -17,8 +17,11 @@ */ package org.apache.hadoop.hdfs.server.namenode; +import java.io.IOException; + import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.blockmanagement.BlockCollection; import org.apache.hadoop.hdfs.server.namenode.NameNode.OperationCategory; @@ -49,8 +52,19 @@ public interface Namesystem extends RwLock, SafeMode { void checkOperation(OperationCategory read) throws StandbyException; + /** + * Gets the erasure coding policy for the path + * @param src + * - path + * @return {@link ErasureCodingPolicy} + * @throws IOException + */ + ErasureCodingPolicy getErasureCodingPolicyForPath(String src) + throws IOException; + boolean isInSnapshot(BlockInfo blockUC); CacheManager getCacheManager(); + HAContext getHAContext(); -} \ No newline at end of file +} 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 06a8219f4ab11..98deed2787644 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 @@ -41,9 +41,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); 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 cf21411330dcb..ae23783456632 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 @@ -236,19 +236,22 @@ private void loadFileDiffList(InputStream in, INodeFile file, int size) .toByteArray(), permission, acl, fileInPb.getModificationTime(), fileInPb.getAccessTime(), (short) fileInPb.getReplication(), fileInPb.getPreferredBlockSize(), - (byte)fileInPb.getStoragePolicyID(), xAttrs); + (byte)fileInPb.getStoragePolicyID(), xAttrs, + fileInPb.getIsStriped()); } FileDiff diff = new FileDiff(pbf.getSnapshotId(), copy, null, pbf.getFileSize()); List bpl = pbf.getBlocksList(); + // in file diff there can only be contiguous blocks BlockInfo[] blocks = new BlockInfo[bpl.size()]; for(int j = 0, e = bpl.size(); j < e; ++j) { Block blk = PBHelperClient.convert(bpl.get(j)); BlockInfo storedBlock = bm.getStoredBlock(blk); if(storedBlock == null) { - storedBlock = bm.addBlockCollection( - new BlockInfoContiguous(blk, copy.getFileReplication()), file); + storedBlock = (BlockInfoContiguous) fsn.getBlockManager() + .addBlockCollectionWithCheck(new BlockInfoContiguous(blk, + copy.getFileReplication()), file); } blocks[j] = storedBlock; } 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 fc27ed872d004..9dcd4d887f80f 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 @@ -22,6 +22,7 @@ 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.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.namenode.INode; import org.apache.hadoop.hdfs.server.namenode.INode.BlocksMapUpdateInfo; @@ -54,7 +55,9 @@ public void saveSelf2Snapshot(int latestSnapshotId, INodeFile iNodeFile, final FileDiff diff = super.saveSelf2Snapshot(latestSnapshotId, iNodeFile, snapshotCopy); if (withBlocks) { // Store blocks if this is the first update - diff.setBlocks(iNodeFile.getBlocks()); + BlockInfo[] blks = iNodeFile.getBlocks(); + assert blks != null; + diff.setBlocks(blks); } } @@ -114,10 +117,10 @@ void combineAndCollectSnapshotBlocks( earlierDiff.setBlocks(removedBlocks); } BlockInfo[] earlierBlocks = - (earlierDiff == null ? new BlockInfo[]{} : earlierDiff.getBlocks()); + (earlierDiff == null ? new BlockInfoContiguous[]{} : earlierDiff.getBlocks()); // Find later snapshot (or file itself) with blocks BlockInfo[] laterBlocks = findLaterSnapshotBlocks(removed.getSnapshotId()); - laterBlocks = (laterBlocks==null) ? file.getBlocks() : laterBlocks; + laterBlocks = (laterBlocks == null) ? file.getBlocks() : laterBlocks; // Skip blocks, which belong to either the earlier or the later lists int i = 0; for(; i < removedBlocks.length; i++) { 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 0000000000000..55ae7b665a2a7 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlockECRecoveryCommand.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.protocol; + +import com.google.common.base.Joiner; + +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +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.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; + +import java.util.Arrays; +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 ecTasks; + + /** + * Create BlockECRecoveryCommand from a collection of + * {@link BlockECRecoveryInfo}, each representing a recovery task + */ + public BlockECRecoveryCommand(int action, + Collection 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(); + } + + /** Block and targets pair */ + @InterfaceAudience.Private + @InterfaceStability.Evolving + public static class BlockECRecoveryInfo { + private final ExtendedBlock block; + private final DatanodeInfo[] sources; + private DatanodeInfo[] targets; + private String[] targetStorageIDs; + private StorageType[] targetStorageTypes; + private final short[] liveBlockIndices; + private final ErasureCodingPolicy ecPolicy; + + public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, + DatanodeStorageInfo[] targetDnStorageInfo, short[] liveBlockIndices, + ErasureCodingPolicy ecPolicy) { + this(block, sources, DatanodeStorageInfo + .toDatanodeInfos(targetDnStorageInfo), DatanodeStorageInfo + .toStorageIDs(targetDnStorageInfo), DatanodeStorageInfo + .toStorageTypes(targetDnStorageInfo), liveBlockIndices, ecPolicy); + } + + public BlockECRecoveryInfo(ExtendedBlock block, DatanodeInfo[] sources, + DatanodeInfo[] targets, String[] targetStorageIDs, + StorageType[] targetStorageTypes, short[] liveBlockIndices, + ErasureCodingPolicy ecPolicy) { + this.block = block; + this.sources = sources; + this.targets = targets; + this.targetStorageIDs = targetStorageIDs; + this.targetStorageTypes = targetStorageTypes; + this.liveBlockIndices = liveBlockIndices; + this.ecPolicy = ecPolicy; + } + + public ExtendedBlock getExtendedBlock() { + return block; + } + + public DatanodeInfo[] getSourceDnInfos() { + return sources; + } + + public DatanodeInfo[] getTargetDnInfos() { + return targets; + } + + public String[] getTargetStorageIDs() { + return targetStorageIDs; + } + + public StorageType[] getTargetStorageTypes() { + return targetStorageTypes; + } + + public short[] getLiveBlockIndices() { + return liveBlockIndices; + } + + public ErasureCodingPolicy getErasureCodingPolicy() { + return ecPolicy; + } + + @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") + .append(" Block Indices: ").append(Arrays.asList(liveBlockIndices)) + .toString(); + } + } + + public Collection getECTasks() { + return this.ecTasks; + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java index a985dbdc8c0e3..b6eddb61579da 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/server/protocol/BlocksWithLocations.java @@ -17,6 +17,7 @@ */ package org.apache.hadoop.hdfs.server.protocol; +import com.google.common.base.Preconditions; import org.apache.hadoop.classification.InterfaceAudience; import org.apache.hadoop.classification.InterfaceStability; import org.apache.hadoop.fs.StorageType; @@ -91,6 +92,35 @@ private StringBuilder appendString(int i, StringBuilder b) { } } + public static class StripedBlockWithLocations extends BlockWithLocations { + final byte[] indices; + final short dataBlockNum; + final int cellSize; + + public StripedBlockWithLocations(BlockWithLocations blk, byte[] indices, + short dataBlockNum, int cellSize) { + super(blk.getBlock(), blk.getDatanodeUuids(), blk.getStorageIDs(), + blk.getStorageTypes()); + Preconditions.checkArgument( + blk.getDatanodeUuids().length == indices.length); + this.indices = indices; + this.dataBlockNum = dataBlockNum; + this.cellSize = cellSize; + } + + public byte[] getIndices() { + return indices; + } + + public short getDataBlockNum() { + return dataBlockNum; + } + + public int getCellSize() { + return cellSize; + } + } + private final BlockWithLocations[] blocks; /** Constructor with one parameter */ 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 dfe081382ca84..add4e736ed147 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_ERASURE_CODING_RECOVERY = 11; // erasure coding recovery command /** * Register Datanode. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java new file mode 100644 index 0000000000000..4ed9d0ae85916 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCli.java @@ -0,0 +1,48 @@ +/** + * 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.tools.erasurecode; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FsShell; +import org.apache.hadoop.fs.shell.CommandFactory; +import org.apache.hadoop.hdfs.HdfsConfiguration; +import org.apache.hadoop.util.ToolRunner; + +/** + * CLI for the erasure code encoding operations. + */ +public class ECCli extends FsShell { + + private final static String usagePrefix = + "Usage: hdfs erasurecode [generic options]"; + + @Override + protected String getUsagePrefix() { + return usagePrefix; + } + + @Override + protected void registerCommands(CommandFactory factory) { + factory.registerCommands(ECCommand.class); + } + + public static void main(String[] args) throws Exception { + Configuration conf = new HdfsConfiguration(); + int res = ToolRunner.run(conf, new ECCli(), args); + System.exit(res); + } +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java new file mode 100644 index 0000000000000..978fe47423ca4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/java/org/apache/hadoop/hdfs/tools/erasurecode/ECCommand.java @@ -0,0 +1,214 @@ +/** + * 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.tools.erasurecode; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.LinkedList; +import java.util.List; + +import org.apache.hadoop.HadoopIllegalArgumentException; +import org.apache.hadoop.classification.InterfaceAudience; +import org.apache.hadoop.classification.InterfaceStability; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.shell.Command; +import org.apache.hadoop.fs.shell.CommandFactory; +import org.apache.hadoop.fs.shell.PathData; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.server.namenode.UnsupportedActionException; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.util.StringUtils; + +/** + * Erasure Coding CLI commands + */ +@InterfaceAudience.Private +@InterfaceStability.Evolving +public abstract class ECCommand extends Command { + + public static void registerCommands(CommandFactory factory) { + // Register all commands of Erasure CLI, with a '-' at the beginning in name + // of the command. + factory.addClass(SetECPolicyCommand.class, "-" + SetECPolicyCommand.NAME); + factory.addClass(GetECPolicyCommand.class, "-" + + GetECPolicyCommand.NAME); + factory.addClass(ListPolicies.class, "-" + ListPolicies.NAME); + } + + @Override + public String getCommandName() { + return getName(); + } + + @Override + protected void run(Path path) throws IOException { + throw new RuntimeException("Not suppose to get here"); + } + + @Deprecated + @Override + public int runAll() { + return run(args); + } + + @Override + protected void processPath(PathData item) throws IOException { + if (!(item.fs instanceof DistributedFileSystem)) { + throw new UnsupportedActionException( + "Erasure commands are only supported for the HDFS paths"); + } + } + + /** + * A command to set the erasure coding policy for a directory, with the name + * of the policy. + */ + static class SetECPolicyCommand extends ECCommand { + public static final String NAME = "setPolicy"; + public static final String USAGE = "[-p ] "; + public static final String DESCRIPTION = + "Set a specified erasure coding policy to a directory\n" + + "Options :\n" + + " -p : erasure coding policy name to encode files. " + + "If not passed the default policy will be used\n" + + " : Path to a directory. Under this directory " + + "files will be encoded using specified erasure coding policy"; + private String ecPolicyName; + private ErasureCodingPolicy ecPolicy = null; + + @Override + protected void processOptions(LinkedList args) throws IOException { + ecPolicyName = StringUtils.popOptionWithArgument("-p", args); + if (args.isEmpty()) { + throw new HadoopIllegalArgumentException(" is missing"); + } + if (args.size() > 1) { + throw new HadoopIllegalArgumentException("Too many arguments"); + } + } + + @Override + protected void processPath(PathData item) throws IOException { + super.processPath(item); + DistributedFileSystem dfs = (DistributedFileSystem) item.fs; + try { + if (ecPolicyName != null) { + ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies(); + for (ErasureCodingPolicy ecPolicy : ecPolicies) { + if (ecPolicyName.equals(ecPolicy.getName())) { + this.ecPolicy = ecPolicy; + break; + } + } + if (ecPolicy == null) { + StringBuilder sb = new StringBuilder(); + sb.append("Policy '"); + sb.append(ecPolicyName); + sb.append("' does not match any of the supported policies."); + sb.append(" Please select any one of "); + List ecPolicyNames = new ArrayList(); + for (ErasureCodingPolicy ecPolicy : ecPolicies) { + ecPolicyNames.add(ecPolicy.getName()); + } + sb.append(ecPolicyNames); + throw new HadoopIllegalArgumentException(sb.toString()); + } + } + dfs.setErasureCodingPolicy(item.path, ecPolicy); + out.println("EC policy set successfully at " + item.path); + } catch (IOException e) { + throw new IOException("Unable to set EC policy for the path " + + item.path + ". " + e.getMessage()); + } + } + } + + /** + * Get the erasure coding policy of a file or directory + */ + static class GetECPolicyCommand extends ECCommand { + public static final String NAME = "getPolicy"; + public static final String USAGE = ""; + public static final String DESCRIPTION = + "Get erasure coding policy information about at specified path\n"; + + @Override + protected void processOptions(LinkedList args) throws IOException { + if (args.isEmpty()) { + throw new HadoopIllegalArgumentException(" is missing"); + } + if (args.size() > 1) { + throw new HadoopIllegalArgumentException("Too many arguments"); + } + } + + @Override + protected void processPath(PathData item) throws IOException { + super.processPath(item); + DistributedFileSystem dfs = (DistributedFileSystem) item.fs; + try { + ErasureCodingPolicy ecPolicy = dfs.getErasureCodingPolicy(item.path); + if (ecPolicy != null) { + out.println(ecPolicy.toString()); + } else { + out.println("Path " + item.path + " is not erasure coded."); + } + } catch (IOException e) { + throw new IOException("Unable to get EC policy for the path " + + item.path + ". " + e.getMessage()); + } + } + } + + /** + * List all supported erasure coding policies + */ + static class ListPolicies extends ECCommand { + public static final String NAME = "listPolicies"; + public static final String USAGE = ""; + public static final String DESCRIPTION = + "Get the list of erasure coding policies supported\n"; + + @Override + protected void processOptions(LinkedList args) throws IOException { + if (!args.isEmpty()) { + throw new HadoopIllegalArgumentException("Too many parameters"); + } + + FileSystem fs = FileSystem.get(getConf()); + if (fs instanceof DistributedFileSystem == false) { + throw new UnsupportedActionException( + "Erasure commands are only supported for the HDFS"); + } + DistributedFileSystem dfs = (DistributedFileSystem) fs; + + ErasureCodingPolicy[] ecPolicies = dfs.getClient().getErasureCodingPolicies(); + StringBuilder sb = new StringBuilder(); + int i = 0; + while (i < ecPolicies.length) { + ErasureCodingPolicy ecPolicy = ecPolicies[i]; + sb.append(ecPolicy.getName()); + i++; + if (i < ecPolicies.length) { + sb.append(", "); + } + } + out.println(sb.toString()); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto index 727259f99b857..6dd7b89a22f5e 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/DatanodeProtocol.proto @@ -33,6 +33,7 @@ package hadoop.hdfs.datanode; import "HAServiceProtocol.proto"; import "hdfs.proto"; +import "erasurecoding.proto"; import "HdfsServer.proto"; /** @@ -59,6 +60,7 @@ message DatanodeCommandProto { UnusedUpgradeCommand = 6; NullDatanodeCommand = 7; BlockIdCommand = 8; + BlockECRecoveryCommand = 9; } required Type cmdType = 1; // Type of the command @@ -72,6 +74,7 @@ message DatanodeCommandProto { optional KeyUpdateCommandProto keyUpdateCmd = 6; optional RegisterCommandProto registerCmd = 7; optional BlockIdCommandProto blkIdCmd = 8; + optional BlockECRecoveryCommandProto blkECRecoveryCmd = 9; } /** @@ -145,6 +148,13 @@ message RegisterCommandProto { // void } +/** + * Block Erasure coding recovery command + */ +message BlockECRecoveryCommandProto { + repeated BlockECRecoveryInfoProto blockECRecoveryinfo = 1; +} + /** * registration - Information of the datanode registering with the namenode */ diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto index 3b60e51a64b3f..66b2a33ef68ca 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/proto/HdfsServer.proto @@ -33,13 +33,6 @@ package hadoop.hdfs; import "hdfs.proto"; -/** - * A list of storage IDs. - */ -message StorageUuidsProto { - repeated string storageUuids = 1; -} - /** * Block access token information */ @@ -68,6 +61,10 @@ message BlockWithLocationsProto { repeated string datanodeUuids = 2; // Datanodes with replicas of the block repeated string storageUuids = 3; // Storages with replicas of the block repeated StorageTypeProto storageTypes = 4; + + optional bytes indices = 5; + optional uint32 dataBlockNum = 6; + optional uint32 cellSize = 7; } /** 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 3bd1d9160d415..43a1fc5fc502d 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; } /** @@ -139,6 +140,7 @@ message INodeSection { optional AclFeatureProto acl = 8; optional XAttrFeatureProto xAttrs = 9; optional uint32 storagePolicyID = 10; + optional bool isStriped = 11; } message QuotaByStorageTypeEntryProto { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml index f1b855e554247..96ffec0c41f74 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml +++ b/hadoop-hdfs-project/hadoop-hdfs/src/main/resources/hdfs-default.xml @@ -2413,11 +2413,11 @@ - - dfs.datanode.block-pinning.enabled - false - Whether pin blocks on favored DataNode. - + + dfs.datanode.block-pinning.enabled + false + Whether pin blocks on favored DataNode. + dfs.client.block.write.locateFollowingBlock.initial.delay.ms @@ -2446,6 +2446,27 @@ + + dfs.datanode.stripedread.timeout.millis + 5000 + Datanode striped read timeout in milliseconds. + + + + + dfs.datanode.stripedread.threads + 20 + Number of threads used by the Datanode for background recovery work. + + + + + dfs.datanode.stripedread.buffer.size + 262144 + Datanode striped read buffer size. + + + dfs.namenode.quota.init-threads 4 diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md index 718463ad7048f..604fde47c1d55 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSCommands.md @@ -36,6 +36,7 @@ HDFS Commands Guide * [crypto](#crypto) * [datanode](#datanode) * [dfsadmin](#dfsadmin) + * [erasurecode](#erasurecode) * [haadmin](#haadmin) * [journalnode](#journalnode) * [mover](#mover) @@ -383,6 +384,25 @@ Usage: Runs a HDFS dfsadmin client. +### `erasurecode` + +Usage: + + hdfs erasurecode [generic options] + [-setPolicy [-s ] ] + [-getPolicy ] + [-listPolicies] + [-usage [cmd ...]] + [-help [cmd ...]] + +| COMMAND\_OPTION | Description | +|:---- |:---- | +|-setPolicy| Set a specified ErasureCoding policy to a directory| +|-getPolicy| Get ErasureCoding policy information about a specified path| +|-listPolicies| Lists all supported ErasureCoding policies| + +Runs the ErasureCoding CLI. See [HDFS ErasureCoding](./HDFSErasureCoding.html#Administrative_commands) for more information on this command. + ### `haadmin` Usage: diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md new file mode 100644 index 0000000000000..5b3aa34833adc --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/site/markdown/HDFSErasureCoding.md @@ -0,0 +1,135 @@ + + +HDFS Erasure Coding +=================== + +* [HDFS Erasure Coding](#HDFS_Erasure_Coding) + * [Purpose](#Purpose) + * [Background](#Background) + * [Architecture](#Architecture) + * [Deployment](#Deployment) + * [Cluster and hardware configuration](#Cluster_and_hardware_configuration) + * [Configuration keys](#Configuration_keys) + * [Administrative commands](#Administrative_commands) + +Purpose +------- + Replication is expensive -- the default 3x replication scheme in HDFS has 200% overhead in storage space and other resources (e.g., network bandwidth). + However, for warm and cold datasets with relatively low I/O activities, additional block replicas are rarely accessed during normal operations, but still consume the same amount of resources as the first replica. + + Therefore, a natural improvement is to use Erasure Coding (EC) in place of replication, which provides the same level of fault-tolerance with much less storage space. In typical Erasure Coding (EC) setups, the storage overhead is no more than 50%. + +Background +---------- + + In storage systems, the most notable usage of EC is Redundant Array of Inexpensive Disks (RAID). RAID implements EC through striping, which divides logically sequential data (such as a file) into smaller units (such as bit, byte, or block) and stores consecutive units on different disks. In the rest of this guide this unit of striping distribution is termed a striping cell (or cell). For each stripe of original data cells, a certain number of parity cells are calculated and stored -- the process of which is called encoding. The error on any striping cell can be recovered through decoding calculation based on surviving data and parity cells. + + Integrating EC with HDFS can improve storage efficiency while still providing similar data durability as traditional replication-based HDFS deployments. + As an example, a 3x replicated file with 6 blocks will consume 6*3 = 18 blocks of disk space. But with EC (6 data, 3 parity) deployment, it will only consume 9 blocks of disk space. + +Architecture +------------ + In the context of EC, striping has several critical advantages. First, it enables online EC (writing data immediately in EC format), avoiding a conversion phase and immediately saving storage space. Online EC also enhances sequential I/O performance by leveraging multiple disk spindles in parallel; this is especially desirable in clusters with high end networking. Second, it naturally distributes a small file to multiple DataNodes and eliminates the need to bundle multiple files into a single coding group. This greatly simplifies file operations such as deletion, quota reporting, and migration between federated namespaces. + + In typical HDFS clusters, small files can account for over 3/4 of total storage consumption. To better support small files, in this first phase of work HDFS supports EC with striping. In the future, HDFS will also support a contiguous EC layout. See the design doc and discussion on [HDFS-7285](https://issues.apache.org/jira/browse/HDFS-7285) for more information. + + * **NameNode Extensions** - Striped HDFS files are logically composed of block groups, each of which contains a certain number of internal blocks. + To reduce NameNode memory consumption from these additional blocks, a new hierarchical block naming protocol was introduced. The ID of a block group can be inferred from the ID of any of its internal blocks. This allows management at the level of the block group rather than the block. + + * **Client Extensions** - The client read and write paths were enhanced to work on multiple internal blocks in a block group in parallel. + On the output / write path, DFSStripedOutputStream manages a set of data streamers, one for each DataNode storing an internal block in the current block group. The streamers mostly + work asynchronously. A coordinator takes charge of operations on the entire block group, including ending the current block group, allocating a new block group, and so forth. + On the input / read path, DFSStripedInputStream translates a requested logical byte range of data as ranges into internal blocks stored on DataNodes. It then issues read requests in + parallel. Upon failures, it issues additional read requests for decoding. + + * **DataNode Extensions** - The DataNode runs an additional ErasureCodingWorker (ECWorker) task for background recovery of failed erasure coded blocks. Failed EC blocks are detected by the NameNode, which then chooses a DataNode to do the recovery work. The recovery task is passed as a heartbeat response. This process is similar to how replicated blocks are re-replicated on failure. Reconstruction performs three key tasks: + + 1. _Read the data from source nodes:_ Input data is read in parallel from source nodes using a dedicated thread pool. + Based on the EC policy, it schedules the read requests to all source targets and reads only the minimum number of input blocks for reconstruction. + + 1. _Decode the data and generate the output data:_ New data and parity blocks are decoded from the input data. All missing data and parity blocks are decoded together. + + 1. _Transfer the generated data blocks to target nodes:_ Once decoding is finished, the recovered blocks are transferred to target DataNodes. + + * **ErasureCoding policy** + To accommodate heterogeneous workloads, we allow files and directories in an HDFS cluster to have different replication and EC policies. + Information on how to encode/decode a file is encapsulated in an ErasureCodingPolicy class. Each policy is defined by the following 2 pieces of information: + + 1. _The ECSchema:_ This includes the numbers of data and parity blocks in an EC group (e.g., 6+3), as well as the codec algorithm (e.g., Reed-Solomon). + + 1. _The size of a striping cell._ This determines the granularity of striped reads and writes, including buffer sizes and encoding work. + + Currently, HDFS supports the Reed-Solomon and XOR erasure coding algorithms. Additional algorithms are planned as future work. + The system default scheme is Reed-Solomon (6, 3) with a cell size of 64KB. + + +Deployment +---------- + +### Cluster and hardware configuration + + Erasure coding places additional demands on the cluster in terms of CPU and network. + + Encoding and decoding work consumes additional CPU on both HDFS clients and DataNodes. + + Erasure coded files are also spread across racks for rack fault-tolerance. + This means that when reading and writing striped files, most operations are off-rack. + Network bisection bandwidth is thus very important. + + For rack fault-tolerance, it is also important to have at least as many racks as the configured EC stripe width. + For the default EC policy of RS (6,3), this means minimally 9 racks, and ideally 10 or 11 to handle planned and unplanned outages. + For clusters with fewer racks than the stripe width, HDFS cannot maintain rack fault-tolerance, but will still attempt + to spread a striped file across multiple nodes to preserve node-level fault-tolerance. + +### Configuration keys + + The codec implementation for Reed-Solomon and XOR can be configured with the following client and DataNode configuration keys: + `io.erasurecode.codec.rs.rawcoder` and `io.erasurecode.codec.xor.rawcoder`. + The default implementations for both of these codecs are pure Java. + + Erasure coding background recovery work on the DataNodes can also be tuned via the following configuration parameters: + + 1. `dfs.datanode.stripedread.timeout.millis` - Timeout for striped reads. Default value is 5000 ms. + 1. `dfs.datanode.stripedread.threads` - Number of concurrent reader threads. Default value is 20 threads. + 1. `dfs.datanode.stripedread.buffer.size` - Buffer size for reader service. Default value is 256KB. + +### Administrative commands + + HDFS provides an `erasurecode` subcommand to perform administrative commands related to erasure coding. + + hdfs erasurecode [generic options] + [-setPolicy [-p ] ] + [-getPolicy ] + [-listPolicies] + [-usage [cmd ...]] + [-help [cmd ...]] + +Below are the details about each command. + + * `[-setPolicy [-p ] ]` + + Sets an ErasureCoding policy on a directory at the specified path. + + `path`: An directory in HDFS. This is a mandatory parameter. Setting a policy only affects newly created files, and does not affect existing files. + + `policyName`: The ErasureCoding policy to be used for files under this directory. This is an optional parameter, specified using ‘-s’ flag. If no policy is specified, the system default ErasureCodingPolicy will be used. + + * `[-getPolicy ]` + + Get details of the ErasureCoding policy of a file or directory at the specified path. + + * `[-listPolicies]` + + Lists all supported ErasureCoding policies. These names are suitable for use with the `setPolicy` command. diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.java new file mode 100644 index 0000000000000..0499a2b1e757b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/CLITestCmdErasureCoding.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.cli; + +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli; +import org.apache.hadoop.cli.util.CLICommandTypes; +import org.apache.hadoop.cli.util.CLITestCmd; +import org.apache.hadoop.cli.util.CommandExecutor; +import org.apache.hadoop.cli.util.ErasureCodingCliCmdExecutor; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.tools.erasurecode.ECCli; + +public class CLITestCmdErasureCoding extends CLITestCmd { + public CLITestCmdErasureCoding(String str, CLICommandTypes type) { + super(str, type); + } + + @Override + public CommandExecutor getExecutor(String tag, Configuration conf) throws IllegalArgumentException { + if (getType() instanceof CLICommandErasureCodingCli) + return new ErasureCodingCliCmdExecutor(tag, new ECCli()); + return super.getExecutor(tag, conf); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java new file mode 100644 index 0000000000000..29ec98ea55a14 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/TestErasureCodingCLI.java @@ -0,0 +1,115 @@ + +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.hadoop.cli; + +import org.apache.hadoop.cli.util.CLICommand; +import org.apache.hadoop.cli.util.CLICommandErasureCodingCli; +import org.apache.hadoop.cli.util.CommandExecutor.Result; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; +import org.xml.sax.SAXException; + +public class TestErasureCodingCLI extends CLITestHelper { + private final int NUM_OF_DATANODES = 3; + private MiniDFSCluster dfsCluster = null; + private FileSystem fs = null; + private String namenode = null; + + @Before + @Override + public void setUp() throws Exception { + super.setUp(); + + dfsCluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(NUM_OF_DATANODES).build(); + dfsCluster.waitClusterUp(); + namenode = conf.get(DFSConfigKeys.FS_DEFAULT_NAME_KEY, "file:///"); + + username = System.getProperty("user.name"); + + fs = dfsCluster.getFileSystem(); + } + + @Override + protected String getTestFile() { + return "testErasureCodingConf.xml"; + } + + @After + @Override + public void tearDown() throws Exception { + if (fs != null) { + fs.close(); + } + if (dfsCluster != null) { + dfsCluster.shutdown(); + } + Thread.sleep(2000); + super.tearDown(); + } + + @Override + protected String expandCommand(final String cmd) { + String expCmd = cmd; + expCmd = expCmd.replaceAll("NAMENODE", namenode); + expCmd = expCmd.replaceAll("#LF#", System.getProperty("line.separator")); + expCmd = super.expandCommand(expCmd); + return expCmd; + } + + @Override + protected TestConfigFileParser getConfigParser() { + return new TestErasureCodingAdmin(); + } + + private class TestErasureCodingAdmin extends + CLITestHelper.TestConfigFileParser { + @Override + public void endElement(String uri, String localName, String qName) + throws SAXException { + if (qName.equals("ec-admin-command")) { + if (testCommands != null) { + testCommands.add(new CLITestCmdErasureCoding(charString, + new CLICommandErasureCodingCli())); + } else if (cleanupCommands != null) { + cleanupCommands.add(new CLITestCmdErasureCoding(charString, + new CLICommandErasureCodingCli())); + } + } else { + super.endElement(uri, localName, qName); + } + } + } + + @Override + protected Result execute(CLICommand cmd) throws Exception { + return cmd.getExecutor(namenode, conf).executeCommand(cmd.getCmd()); + } + + @Test + @Override + public void testAll() { + super.testAll(); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java new file mode 100644 index 0000000000000..aafcd9fa77458 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/CLICommandErasureCodingCli.java @@ -0,0 +1,21 @@ +/** + * 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.cli.util; + +public class CLICommandErasureCodingCli implements CLICommandTypes { +} \ No newline at end of file diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java new file mode 100644 index 0000000000000..e993313a8fbc6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/cli/util/ErasureCodingCliCmdExecutor.java @@ -0,0 +1,37 @@ +/** + * 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.cli.util; + +import org.apache.hadoop.hdfs.tools.erasurecode.ECCli; +import org.apache.hadoop.util.ToolRunner; + +public class ErasureCodingCliCmdExecutor extends CommandExecutor { + protected String namenode = null; + protected ECCli admin = null; + + public ErasureCodingCliCmdExecutor(String namenode, ECCli admin) { + this.namenode = namenode; + this.admin = admin; + } + + @Override + protected void execute(final String cmd) throws Exception { + String[] args = getCommandAsArgs(cmd, "NAMENODE", this.namenode); + ToolRunner.run(admin, args); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java index ba25d97aa329e..1ca1ca5f61b99 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/BlockReaderTestUtil.java @@ -165,20 +165,19 @@ public void readAndCheckEOS(BlockReader reader, int length, boolean expectEof) */ public BlockReader getBlockReader(LocatedBlock testBlock, int offset, int lenToRead) throws IOException { - return getBlockReader(cluster, testBlock, offset, lenToRead); + return getBlockReader(cluster.getFileSystem(), testBlock, offset, lenToRead); } /** * Get a BlockReader for the given block. */ - public static BlockReader getBlockReader(MiniDFSCluster cluster, - LocatedBlock testBlock, int offset, int lenToRead) throws IOException { + public static BlockReader getBlockReader(final DistributedFileSystem fs, + LocatedBlock testBlock, int offset, long lenToRead) throws IOException { InetSocketAddress targetAddr = null; ExtendedBlock block = testBlock.getBlock(); DatanodeInfo[] nodes = testBlock.getLocations(); targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr()); - final DistributedFileSystem fs = cluster.getFileSystem(); return new BlockReaderFactory(fs.getClient().getConf()). setInetSocketAddress(targetAddr). setBlock(block). 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 bce8b64c98d15..155abd01c6d80 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 @@ -66,6 +66,12 @@ import java.util.UUID; import java.util.concurrent.TimeoutException; import java.util.concurrent.atomic.AtomicBoolean; +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.lang.UnhandledException; @@ -106,6 +112,7 @@ import org.apache.hadoop.hdfs.protocol.LayoutVersion; 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.datatransfer.Sender; import org.apache.hadoop.hdfs.protocol.proto.DataTransferProtos.BlockOpResponseProto; import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; @@ -125,14 +132,19 @@ import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; 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; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocol; +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo; +import org.apache.hadoop.hdfs.server.protocol.ReceivedDeletedBlockInfo.BlockStatus; +import org.apache.hadoop.hdfs.server.protocol.StorageReceivedDeletedBlocks; import org.apache.hadoop.hdfs.tools.DFSAdmin; import org.apache.hadoop.hdfs.tools.JMXGet; import org.apache.hadoop.io.IOUtils; @@ -154,12 +166,8 @@ import org.mockito.internal.util.reflection.Whitebox; import com.google.common.annotations.VisibleForTesting; -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.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS; /** Utilities for HDFS tests */ public class DFSTestUtil { @@ -809,15 +817,21 @@ public static byte[] readFileAsBytes(File f) throws IOException { return os.toByteArray(); } - /* Write the given string to the given file */ - public static void writeFile(FileSystem fs, Path p, String s) + /* Write the given bytes to the given file */ + public static void writeFile(FileSystem fs, Path p, byte[] bytes) throws IOException { if (fs.exists(p)) { fs.delete(p, true); } - InputStream is = new ByteArrayInputStream(s.getBytes()); + InputStream is = new ByteArrayInputStream(bytes); FSDataOutputStream os = fs.create(p); - IOUtils.copyBytes(is, os, s.length(), true); + IOUtils.copyBytes(is, os, bytes.length, true); + } + + /* Write the given string to the given file */ + public static void writeFile(FileSystem fs, Path p, String s) + throws IOException { + writeFile(fs, p, s.getBytes()); } /* Append the given string to the given file */ @@ -1834,7 +1848,7 @@ public static void resetLastUpdatesWithOffset(DatanodeInfo dn, long offset) { dn.setLastUpdate(Time.now() + offset); dn.setLastUpdateMonotonic(Time.monotonicNow() + offset); } - + /** * This method takes a set of block locations and fills the provided buffer * with expected bytes based on simulated content from @@ -1858,6 +1872,132 @@ public static void fillExpectedBuf(LocatedBlocks lbs, byte[] expected) { } } + 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; + } + + /** + * Creates the metadata of a file in striped layout. This method only + * manipulates the NameNode state without injecting data to DataNode. + * You should disable periodical heartbeat before use this. + * @param file Path of the file to create + * @param dir Parent path of the file + * @param numBlocks Number of striped block groups to add to the file + * @param numStripesPerBlk Number of striped cells in each block + * @param toMkdir + */ + public static void createStripedFile(MiniDFSCluster cluster, Path file, Path dir, + int numBlocks, int numStripesPerBlk, boolean toMkdir) throws Exception { + DistributedFileSystem dfs = cluster.getFileSystem(); + // If outer test already set EC policy, dir should be left as null + if (toMkdir) { + assert dir != null; + dfs.mkdirs(dir); + try { + dfs.getClient().setErasureCodingPolicy(dir.toString(), null); + } catch (IOException e) { + if (!e.getMessage().contains("non-empty directory")) { + throw e; + } + } + } + + 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 = addStripedBlockToFile(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); + } + } + + /** + * Adds a striped block group to a file. This method only manipulates NameNode + * states of the file and the block without injecting data to DataNode. + * It does mimic block reports. + * You should disable periodical heartbeat before use this. + * @param dataNodes List DataNodes to host the striped block group + * @param previous Previous block in the file + * @param numStripes Number of stripes in each block group + * @return The added block group + */ + public static Block addStripedBlockToFile(List 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.getPreferredBlockReplication(); + assert dataNodes.size() >= groupSize; + // 1. RECEIVING_BLOCK IBR + for (int i = 0; i < groupSize; i++) { + DataNode dn = dataNodes.get(i); + 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 + for (int i = 0; i < groupSize; i++) { + DataNode dn = dataNodes.get(i); + 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; + } + + /** + * Because currently DFSStripedOutputStream does not support hflush/hsync, + * tests can use this method to flush all the buffered data to DataNodes. + */ + public static ExtendedBlock flushInternal(DFSStripedOutputStream out) + throws IOException { + out.flushAllInternals(); + return out.getBlock(); + } + + public static ExtendedBlock flushBuffer(DFSStripedOutputStream out) + throws IOException { + out.flush(); + return out.getBlock(); + } + public static void waitForMetric(final JMXGet jmx, final String metricName, final int expectedValue) throws TimeoutException, InterruptedException { GenericTestUtils.waitFor(new Supplier() { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java index 40414c85be7f8..71a4bd2dc03bc 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/MiniDFSCluster.java @@ -121,6 +121,7 @@ import org.apache.hadoop.util.ExitUtil; import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.StringUtils; +import org.apache.hadoop.util.ShutdownHookManager; import org.apache.hadoop.util.ToolRunner; import com.google.common.base.Joiner; @@ -2115,8 +2116,6 @@ public synchronized DataNodeProperties stopDataNode(String dnName) { int node = -1; for (int i = 0; i < dataNodes.size(); i++) { DataNode dn = dataNodes.get(i).datanode; - LOG.info("DN name=" + dnName + " found DN=" + dn + - " with name=" + dn.getDisplayName()); if (dnName.equals(dn.getDatanodeId().getXferAddr())) { node = i; break; diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java new file mode 100644 index 0000000000000..12453fafb3c9a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/StripedFileTestUtil.java @@ -0,0 +1,491 @@ +/** + * 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.base.Joiner; +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataInputStream; +import org.apache.hadoop.fs.FileStatus; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +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.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.hdfs.web.ByteRangeInputStream; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureEncoder; +import org.junit.Assert; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.Collections; +import java.util.HashSet; +import java.util.List; +import java.util.concurrent.TimeoutException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.junit.Assert.assertEquals; + +public class StripedFileTestUtil { + public static final Log LOG = LogFactory.getLog(StripedFileTestUtil.class); + /* + * These values correspond to the values used by the system default erasure + * coding policy. + */ + public static final short NUM_DATA_BLOCKS = (short) 6; + public static final short NUM_PARITY_BLOCKS = (short) 3; + public static final int BLOCK_STRIPED_CELL_SIZE = 64 * 1024; + + static final int stripesPerBlock = 4; + static final int blockSize = BLOCK_STRIPED_CELL_SIZE * stripesPerBlock; + static final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS + 2; + static final int BLOCK_GROUP_SIZE = blockSize * NUM_DATA_BLOCKS; + + + static byte[] generateBytes(int cnt) { + byte[] bytes = new byte[cnt]; + for (int i = 0; i < cnt; i++) { + bytes[i] = getByte(i); + } + return bytes; + } + + static byte getByte(long pos) { + final int mod = 29; + return (byte) (pos % mod + 1); + } + + static int readAll(FSDataInputStream in, byte[] buf) throws IOException { + int readLen = 0; + int ret; + while ((ret = in.read(buf, readLen, buf.length - readLen)) >= 0 && + readLen <= buf.length) { + readLen += ret; + } + return readLen; + } + + static void verifyLength(FileSystem fs, Path srcPath, int fileLength) + throws IOException { + FileStatus status = fs.getFileStatus(srcPath); + assertEquals("File length should be the same", fileLength, status.getLen()); + } + + static void verifyPread(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, byte[] buf) throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + int[] startOffsets = {0, 1, BLOCK_STRIPED_CELL_SIZE - 102, BLOCK_STRIPED_CELL_SIZE, BLOCK_STRIPED_CELL_SIZE + 102, + BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1), BLOCK_STRIPED_CELL_SIZE * (NUM_DATA_BLOCKS - 1) + 102, + BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS, fileLength - 102, fileLength - 1}; + for (int startOffset : startOffsets) { + startOffset = Math.max(0, Math.min(startOffset, fileLength - 1)); + int remaining = fileLength - startOffset; + int offset = startOffset; + final byte[] result = new byte[remaining]; + while (remaining > 0) { + int target = Math.min(remaining, buf.length); + in.readFully(offset, buf, 0, target); + System.arraycopy(buf, 0, result, offset - startOffset, target); + remaining -= target; + offset += target; + } + for (int i = 0; i < fileLength - startOffset; i++) { + assertEquals("Byte at " + (startOffset + i) + " is different, " + "the startOffset is " + startOffset, expected[startOffset + i], result[i]); + } + } + } + } + + static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, byte[] buf) throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + final byte[] result = new byte[fileLength]; + int readLen = 0; + int ret; + while ((ret = in.read(buf, 0, buf.length)) >= 0) { + System.arraycopy(buf, 0, result, readLen, ret); + readLen += ret; + } + assertEquals("The length of file should be the same to write size", fileLength, readLen); + Assert.assertArrayEquals(expected, result); + } + } + + static void verifyStatefulRead(FileSystem fs, Path srcPath, int fileLength, + byte[] expected, ByteBuffer buf) throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + ByteBuffer result = ByteBuffer.allocate(fileLength); + int readLen = 0; + int ret; + while ((ret = in.read(buf)) >= 0) { + readLen += ret; + buf.flip(); + result.put(buf); + buf.clear(); + } + assertEquals("The length of file should be the same to write size", fileLength, readLen); + Assert.assertArrayEquals(expected, result.array()); + } + } + + static void verifySeek(FileSystem fs, Path srcPath, int fileLength) + throws IOException { + try (FSDataInputStream in = fs.open(srcPath)) { + // seek to 1/2 of content + int pos = fileLength / 2; + assertSeekAndRead(in, pos, fileLength); + + // seek to 1/3 of content + pos = fileLength / 3; + assertSeekAndRead(in, pos, fileLength); + + // seek to 0 pos + pos = 0; + assertSeekAndRead(in, pos, fileLength); + + if (fileLength > BLOCK_STRIPED_CELL_SIZE) { + // seek to cellSize boundary + pos = BLOCK_STRIPED_CELL_SIZE - 1; + assertSeekAndRead(in, pos, fileLength); + } + + if (fileLength > BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS) { + // seek to striped cell group boundary + pos = BLOCK_STRIPED_CELL_SIZE * NUM_DATA_BLOCKS - 1; + assertSeekAndRead(in, pos, fileLength); + } + + if (fileLength > blockSize * NUM_DATA_BLOCKS) { + // seek to striped block group boundary + pos = blockSize * NUM_DATA_BLOCKS - 1; + assertSeekAndRead(in, pos, fileLength); + } + + if (!(in.getWrappedStream() instanceof ByteRangeInputStream)) { + try { + in.seek(-1); + Assert.fail("Should be failed if seek to negative offset"); + } catch (EOFException e) { + // expected + } + + try { + in.seek(fileLength + 1); + Assert.fail("Should be failed if seek after EOF"); + } catch (EOFException e) { + // expected + } + } + } + } + + static void assertSeekAndRead(FSDataInputStream fsdis, int pos, + int writeBytes) throws IOException { + fsdis.seek(pos); + byte[] buf = new byte[writeBytes]; + int readLen = StripedFileTestUtil.readAll(fsdis, buf); + assertEquals(readLen, writeBytes - pos); + for (int i = 0; i < readLen; i++) { + assertEquals("Byte at " + i + " should be the same", StripedFileTestUtil.getByte(pos + i), buf[i]); + } + } + + static void killDatanode(MiniDFSCluster cluster, DFSStripedOutputStream out, + final int dnIndex, final AtomicInteger pos) { + final StripedDataStreamer s = out.getStripedDataStreamer(dnIndex); + final DatanodeInfo datanode = getDatanodes(s); + assert datanode != null; + LOG.info("killDatanode " + dnIndex + ": " + datanode + ", pos=" + pos); + cluster.stopDataNode(datanode.getXferAddr()); + } + + static DatanodeInfo getDatanodes(StripedDataStreamer streamer) { + for(;;) { + final DatanodeInfo[] datanodes = streamer.getNodes(); + if (datanodes != null) { + assertEquals(1, datanodes.length); + Assert.assertNotNull(datanodes[0]); + return datanodes[0]; + } + try { + Thread.sleep(100); + } catch (InterruptedException ignored) { + return null; + } + } + } + + /** + * If the length of blockGroup is less than a full stripe, it returns the the + * number of actual data internal blocks. Otherwise returns NUM_DATA_BLOCKS. + */ + public static short getRealDataBlockNum(int numBytes) { + return (short) Math.min(NUM_DATA_BLOCKS, + (numBytes - 1) / BLOCK_STRIPED_CELL_SIZE + 1); + } + + public static short getRealTotalBlockNum(int numBytes) { + return (short) (getRealDataBlockNum(numBytes) + NUM_PARITY_BLOCKS); + } + + /** + * Wait for all the internalBlocks of the blockGroups of the given file to be reported. + */ + public static void waitBlockGroupsReported(DistributedFileSystem fs, String src) + throws IOException, InterruptedException, TimeoutException { + boolean success; + final int ATTEMPTS = 40; + int count = 0; + + do { + success = true; + count++; + LocatedBlocks lbs = fs.getClient().getLocatedBlocks(src, 0); + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + short expected = getRealTotalBlockNum((int) lb.getBlockSize()); + int reported = lb.getLocations().length; + if (reported != expected){ + success = false; + System.out.println("blockGroup " + lb.getBlock() + " of file " + src + + " has reported internalBlocks " + reported + + " (desired " + expected + "); locations " + + Joiner.on(' ').join(lb.getLocations())); + Thread.sleep(1000); + break; + } + } + if (success) { + System.out.println("All blockGroups of file " + src + + " verified to have all internalBlocks."); + } + } while (!success && count < ATTEMPTS); + + if (count == ATTEMPTS) { + throw new TimeoutException("Timed out waiting for " + src + + " to have all the internalBlocks"); + } + } + + /** + * Generate n random and different numbers within + * specified non-negative integer range + * @param min minimum of the range + * @param max maximum of the range + * @param n number to be generated + */ + public static int[] randomArray(int min, int max, int n){ + if (n > (max - min + 1) || max < min || min < 0 || max < 0) { + return null; + } + int[] result = new int[n]; + for (int i = 0; i < n; i++) { + result[i] = -1; + } + + int count = 0; + while(count < n) { + int num = (int) (Math.random() * (max - min)) + min; + boolean flag = true; + for (int j = 0; j < n; j++) { + if(num == result[j]){ + flag = false; + break; + } + } + if(flag){ + result[count] = num; + count++; + } + } + return result; + } + + /** + * Verify that blocks in striped block group are on different nodes, and every + * internal blocks exists. + */ + public static void verifyLocatedStripedBlocks(LocatedBlocks lbs, int groupSize) { + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + assert lb instanceof LocatedStripedBlock; + HashSet locs = new HashSet<>(); + Collections.addAll(locs, lb.getLocations()); + assertEquals(groupSize, lb.getLocations().length); + assertEquals(groupSize, locs.size()); + + // verify that every internal blocks exists + int[] blockIndices = ((LocatedStripedBlock) lb).getBlockIndices(); + assertEquals(groupSize, blockIndices.length); + HashSet found = new HashSet<>(); + for (int index : blockIndices) { + assert index >=0; + found.add(index); + } + assertEquals(groupSize, found.size()); + } + } + + static void checkData(DistributedFileSystem dfs, Path srcPath, int length, + int[] killedDnIndex, long oldGS) throws IOException { + + StripedFileTestUtil.verifyLength(dfs, srcPath, length); + Arrays.sort(killedDnIndex); + List> blockGroupList = new ArrayList<>(); + LocatedBlocks lbs = dfs.getClient().getLocatedBlocks(srcPath.toString(), 0L, + Long.MAX_VALUE); + int expectedNumGroup = 0; + if (length > 0) { + expectedNumGroup = (length - 1) / BLOCK_GROUP_SIZE + 1; + } + assertEquals(expectedNumGroup, lbs.getLocatedBlocks().size()); + + for (LocatedBlock firstBlock : lbs.getLocatedBlocks()) { + Assert.assertTrue(firstBlock instanceof LocatedStripedBlock); + + final long gs = firstBlock.getBlock().getGenerationStamp(); + final String s = "gs=" + gs + ", oldGS=" + oldGS; + LOG.info(s); + Assert.assertTrue(s, gs >= oldGS); + + LocatedBlock[] blocks = StripedBlockUtil.parseStripedBlockGroup( + (LocatedStripedBlock) firstBlock, BLOCK_STRIPED_CELL_SIZE, + NUM_DATA_BLOCKS, NUM_PARITY_BLOCKS); + blockGroupList.add(Arrays.asList(blocks)); + } + + // test each block group + for (int group = 0; group < blockGroupList.size(); group++) { + final boolean isLastGroup = group == blockGroupList.size() - 1; + final int groupSize = !isLastGroup? BLOCK_GROUP_SIZE + : length - (blockGroupList.size() - 1)*BLOCK_GROUP_SIZE; + final int numCellInGroup = (groupSize - 1)/BLOCK_STRIPED_CELL_SIZE + 1; + final int lastCellIndex = (numCellInGroup - 1) % NUM_DATA_BLOCKS; + final int lastCellSize = groupSize - (numCellInGroup - 1)*BLOCK_STRIPED_CELL_SIZE; + + //get the data of this block + List blockList = blockGroupList.get(group); + byte[][] dataBlockBytes = new byte[NUM_DATA_BLOCKS][]; + byte[][] parityBlockBytes = new byte[NUM_PARITY_BLOCKS][]; + + // for each block, use BlockReader to read data + for (int i = 0; i < blockList.size(); i++) { + final int j = i >= NUM_DATA_BLOCKS? 0: i; + final int numCellInBlock = (numCellInGroup - 1)/NUM_DATA_BLOCKS + + (j <= lastCellIndex? 1: 0); + final int blockSize = numCellInBlock*BLOCK_STRIPED_CELL_SIZE + + (isLastGroup && j == lastCellIndex? lastCellSize - BLOCK_STRIPED_CELL_SIZE: 0); + + final byte[] blockBytes = new byte[blockSize]; + if (i < NUM_DATA_BLOCKS) { + dataBlockBytes[i] = blockBytes; + } else { + parityBlockBytes[i - NUM_DATA_BLOCKS] = blockBytes; + } + + final LocatedBlock lb = blockList.get(i); + LOG.info("i,j=" + i + ", " + j + ", numCellInBlock=" + numCellInBlock + + ", blockSize=" + blockSize + ", lb=" + lb); + if (lb == null) { + continue; + } + final ExtendedBlock block = lb.getBlock(); + assertEquals(blockSize, block.getNumBytes()); + + if (block.getNumBytes() == 0) { + continue; + } + + if (Arrays.binarySearch(killedDnIndex, i) < 0) { + final BlockReader blockReader = BlockReaderTestUtil.getBlockReader( + dfs, lb, 0, block.getNumBytes()); + blockReader.readAll(blockBytes, 0, (int) block.getNumBytes()); + blockReader.close(); + } + } + + // check data + final int groupPosInFile = group*BLOCK_GROUP_SIZE; + for (int i = 0; i < dataBlockBytes.length; i++) { + boolean killed = false; + if (Arrays.binarySearch(killedDnIndex, i) >= 0){ + killed = true; + } + final byte[] actual = dataBlockBytes[i]; + for (int posInBlk = 0; posInBlk < actual.length; posInBlk++) { + final long posInFile = StripedBlockUtil.offsetInBlkToOffsetInBG( + BLOCK_STRIPED_CELL_SIZE, NUM_DATA_BLOCKS, posInBlk, i) + groupPosInFile; + Assert.assertTrue(posInFile < length); + final byte expected = getByte(posInFile); + + if (killed) { + actual[posInBlk] = expected; + } else { + if(expected != actual[posInBlk]){ + String s = "expected=" + expected + " but actual=" + actual[posInBlk] + + ", posInFile=" + posInFile + ", posInBlk=" + posInBlk + + ". group=" + group + ", i=" + i; + Assert.fail(s); + } + } + } + } + + // check parity + verifyParityBlocks(dfs.getConf(), lbs.getLocatedBlocks().get(group) + .getBlockSize(), + BLOCK_STRIPED_CELL_SIZE, dataBlockBytes, parityBlockBytes, killedDnIndex); + } + } + + static void verifyParityBlocks(Configuration conf, final long size, final int cellSize, + byte[][] dataBytes, byte[][] parityBytes, int[] killedDnIndex) { + Arrays.sort(killedDnIndex); + // verify the parity blocks + int parityBlkSize = (int) StripedBlockUtil.getInternalBlockLength( + size, cellSize, dataBytes.length, dataBytes.length); + final byte[][] expectedParityBytes = new byte[parityBytes.length][]; + for (int i = 0; i < parityBytes.length; i++) { + expectedParityBytes[i] = new byte[parityBlkSize]; + } + for (int i = 0; i < dataBytes.length; i++) { + if (dataBytes[i] == null) { + dataBytes[i] = new byte[dataBytes[0].length]; + } else if (dataBytes[i].length < dataBytes[0].length) { + final byte[] tmp = dataBytes[i]; + dataBytes[i] = new byte[dataBytes[0].length]; + System.arraycopy(tmp, 0, dataBytes[i], 0, tmp.length); + } + } + final RawErasureEncoder encoder = + CodecUtil.createRSRawEncoder(conf, dataBytes.length, parityBytes.length); + encoder.encode(dataBytes, expectedParityBytes); + for (int i = 0; i < parityBytes.length; i++) { + if (Arrays.binarySearch(killedDnIndex, dataBytes.length + i) < 0){ + Assert.assertArrayEquals("i=" + i + ", killedDnIndex=" + Arrays.toString(killedDnIndex), + expectedParityBytes[i], parityBytes[i]); + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java index a8ca9c7bf52f0..a392c6c6c576c 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestBlockReaderFactory.java @@ -250,8 +250,8 @@ public void run() { LocatedBlock lblock = locatedBlocks.get(0); // first block BlockReader blockReader = null; try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); Assert.fail("expected getBlockReader to fail the first time."); } catch (Throwable t) { Assert.assertTrue("expected to see 'TCP reads were disabled " + @@ -265,8 +265,8 @@ public void run() { // Second time should succeed. try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); } catch (Throwable t) { LOG.error("error trying to retrieve a block reader " + "the second time.", t); @@ -474,8 +474,8 @@ public void run() { while (true) { BlockReader blockReader = null; try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); sem.release(); try { blockReader.readAll(buf, 0, TEST_FILE_LEN); @@ -514,8 +514,8 @@ public void run() { // getting a ClosedChannelException. BlockReader blockReader = null; try { - blockReader = BlockReaderTestUtil. - getBlockReader(cluster, lblock, 0, TEST_FILE_LEN); + blockReader = BlockReaderTestUtil.getBlockReader( + cluster.getFileSystem(), lblock, 0, TEST_FILE_LEN); blockReader.readFully(buf, 0, TEST_FILE_LEN); } finally { if (blockReader != null) blockReader.close(); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java index 6a224f9233e78..e41c06a3a6cd3 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSClientRetries.java @@ -256,12 +256,12 @@ public Object answer(InvocationOnMock invocation) Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)).when(mockNN).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null)).when(mockNN).getFileInfo(anyString()); Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)) + 1010, 0, null, (byte) 0, null)) .when(mockNN) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), @@ -549,7 +549,7 @@ private LocatedBlocks makeBadBlockList(LocatedBlocks goodBlockList) { badBlocks.add(badLocatedBlock); return new LocatedBlocks(goodBlockList.getFileLength(), false, badBlocks, null, true, - null); + null, null); } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java new file mode 100644 index 0000000000000..549a744bc7de5 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedInputStream.java @@ -0,0 +1,335 @@ +/** + * 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 static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_DEFAULT; +import static org.apache.hadoop.fs.CommonConfigurationKeysPublic.IO_FILE_BUFFER_SIZE_KEY; +import org.apache.hadoop.fs.Path; +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.LocatedStripedBlock; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.datanode.DataNodeTestUtils; +import org.apache.hadoop.hdfs.server.datanode.SimulatedFSDataset; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.apache.hadoop.io.erasurecode.CodecUtil; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.rawcoder.RawErasureDecoder; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Arrays; +import java.util.List; + +public class TestDFSStripedInputStream { + + public static final Log LOG = LogFactory.getLog(TestDFSStripedInputStream.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 ErasureCodingPolicy ecPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final int NUM_STRIPE_PER_BLOCK = 2; + private final int INTERNAL_BLOCK_SIZE = NUM_STRIPE_PER_BLOCK * CELLSIZE; + private final int BLOCK_GROUP_SIZE = DATA_BLK_NUM * INTERNAL_BLOCK_SIZE; + + @Before + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, INTERNAL_BLOCK_SIZE); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + SimulatedFSDataset.setFactory(conf); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes( + DATA_BLK_NUM + PARITY_BLK_NUM).build(); + cluster.waitActive(); + for (DataNode dn : cluster.getDataNodes()) { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); + } + fs = cluster.getFileSystem(); + fs.mkdirs(dirPath); + fs.getClient().setErasureCodingPolicy(dirPath.toString(), null); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Test {@link DFSStripedInputStream#getBlockAt(long)} + */ + @Test + public void testRefreshBlock() throws Exception { + final int numBlocks = 4; + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); + final DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), + filePath.toString(), false, ecPolicy, null); + + List lbList = lbs.getLocatedBlocks(); + for (LocatedBlock aLbList : lbList) { + LocatedStripedBlock lsb = (LocatedStripedBlock) aLbList; + LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(lsb, + CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); + for (int j = 0; j < DATA_BLK_NUM; j++) { + LocatedBlock refreshed = in.refreshLocatedBlock(blks[j]); + 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 = 2; + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCK_GROUP_SIZE * numBlocks); + int fileLen = BLOCK_GROUP_SIZE * numBlocks; + + byte[] expected = new byte[fileLen]; + assertEquals(numBlocks, lbs.getLocatedBlocks().size()); + for (int bgIdx = 0; bgIdx < numBlocks; bgIdx++) { + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(bgIdx)); + for (int i = 0; i < DATA_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + bg.getBlock().getGenerationStamp()); + blk.setGenerationStamp(bg.getBlock().getGenerationStamp()); + cluster.injectBlocks(i, Arrays.asList(blk), + bg.getBlock().getBlockPoolId()); + } + + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[bgIdx*BLOCK_GROUP_SIZE + posInFile] = + SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } + } + } + } + DFSStripedInputStream in = new DFSStripedInputStream(fs.getClient(), + filePath.toString(), false, ecPolicy, null); + + int[] startOffsets = {0, 1, CELLSIZE - 102, CELLSIZE, CELLSIZE + 102, + CELLSIZE*DATA_BLK_NUM, CELLSIZE*DATA_BLK_NUM + 102, + BLOCK_GROUP_SIZE - 102, BLOCK_GROUP_SIZE, BLOCK_GROUP_SIZE + 102, + fileLen - 1}; + for (int startOffset : startOffsets) { + startOffset = Math.max(0, Math.min(startOffset, fileLen - 1)); + int remaining = fileLen - startOffset; + byte[] buf = new byte[fileLen]; + int ret = in.read(startOffset, buf, 0, fileLen); + assertEquals(remaining, ret); + for (int i = 0; i < remaining; i++) { + Assert.assertEquals("Byte at " + (startOffset + i) + " should be the " + + "same", + expected[startOffset + i], buf[i]); + } + } + in.close(); + } + + @Test + public void testPreadWithDNFailure() throws Exception { + final int numBlocks = 4; + final int failedDNIdx = 2; + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations( + filePath.toString(), 0, BLOCK_GROUP_SIZE); + + assert lbs.get(0) instanceof LocatedStripedBlock; + LocatedStripedBlock bg = (LocatedStripedBlock)(lbs.get(0)); + for (int i = 0; i < DATA_BLK_NUM + PARITY_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + 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, + ErasureCodingPolicyManager.getSystemDefaultPolicy(), null); + int readSize = BLOCK_GROUP_SIZE; + byte[] readBuffer = new byte[readSize]; + byte[] expected = new byte[readSize]; + cluster.stopDataNode(failedDNIdx); + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[posInFile] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } + } + } + + // Update the expected content for decoded data + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + byte[][] decodeInputs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][CELLSIZE]; + int[] missingBlkIdx = new int[]{failedDNIdx + PARITY_BLK_NUM, 1, 2}; + byte[][] decodeOutputs = new byte[PARITY_BLK_NUM][CELLSIZE]; + for (int j = 0; j < DATA_BLK_NUM; j++) { + int posInBuf = i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE; + if (j != failedDNIdx) { + System.arraycopy(expected, posInBuf, decodeInputs[j + PARITY_BLK_NUM], + 0, CELLSIZE); + } + } + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + decodeInputs[0][k] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + DATA_BLK_NUM), posInBlk); + } + for (int m : missingBlkIdx) { + decodeInputs[m] = null; + } + RawErasureDecoder rawDecoder = CodecUtil.createRSRawDecoder(conf, + DATA_BLK_NUM, PARITY_BLK_NUM); + rawDecoder.decode(decodeInputs, missingBlkIdx, decodeOutputs); + int posInBuf = i * CELLSIZE * DATA_BLK_NUM + failedDNIdx * CELLSIZE; + System.arraycopy(decodeOutputs[0], 0, expected, posInBuf, CELLSIZE); + } + int delta = 10; + int done = 0; + // read a small delta, shouldn't trigger decode + // |cell_0 | + // |10 | + done += in.read(0, readBuffer, 0, delta); + assertEquals(delta, done); + // both head and trail cells are partial + // |c_0 |c_1 |c_2 |c_3 |c_4 |c_5 | + // |256K - 10|missing|256K|256K|256K - 10|not in range| + done += in.read(delta, readBuffer, delta, + CELLSIZE * (DATA_BLK_NUM - 1) - 2 * delta); + assertEquals(CELLSIZE * (DATA_BLK_NUM - 1) - delta, done); + // read the rest + done += in.read(done, readBuffer, done, readSize - done); + assertEquals(readSize, done); + assertArrayEquals(expected, readBuffer); + } + + @Test + public void testStatefulRead() throws Exception { + testStatefulRead(false, false); + testStatefulRead(true, false); + testStatefulRead(true, true); + } + + private void testStatefulRead(boolean useByteBuffer, boolean cellMisalignPacket) + throws Exception { + final int numBlocks = 2; + final int fileSize = numBlocks * BLOCK_GROUP_SIZE; + if (cellMisalignPacket) { + conf.setInt(IO_FILE_BUFFER_SIZE_KEY, IO_FILE_BUFFER_SIZE_DEFAULT + 1); + tearDown(); + setup(); + } + DFSTestUtil.createStripedFile(cluster, filePath, null, numBlocks, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = fs.getClient().namenode.getBlockLocations(filePath.toString(), 0, fileSize); + + assert lbs.getLocatedBlocks().size() == numBlocks; + for (LocatedBlock lb : lbs.getLocatedBlocks()) { + assert lb instanceof LocatedStripedBlock; + LocatedStripedBlock bg = (LocatedStripedBlock)(lb); + for (int i = 0; i < DATA_BLK_NUM; i++) { + Block blk = new Block(bg.getBlock().getBlockId() + i, + NUM_STRIPE_PER_BLOCK * CELLSIZE, + 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, ecPolicy, null); + + byte[] expected = new byte[fileSize]; + + for (LocatedBlock bg : lbs.getLocatedBlocks()) { + /** A variation of {@link DFSTestUtil#fillExpectedBuf} for striped blocks */ + for (int i = 0; i < NUM_STRIPE_PER_BLOCK; i++) { + for (int j = 0; j < DATA_BLK_NUM; j++) { + for (int k = 0; k < CELLSIZE; k++) { + int posInBlk = i * CELLSIZE + k; + int posInFile = (int) bg.getStartOffset() + + i * CELLSIZE * DATA_BLK_NUM + j * CELLSIZE + k; + expected[posInFile] = SimulatedFSDataset.simulatedByte( + new Block(bg.getBlock().getBlockId() + j), posInBlk); + } + } + } + } + + if (useByteBuffer) { + ByteBuffer readBuffer = ByteBuffer.allocate(fileSize); + int done = 0; + while (done < fileSize) { + int ret = in.read(readBuffer); + assertTrue(ret > 0); + done += ret; + } + assertArrayEquals(expected, readBuffer.array()); + } else { + byte[] readBuffer = new byte[fileSize]; + int done = 0; + while (done < fileSize) { + int ret = in.read(readBuffer, done, fileSize - done); + assertTrue(ret > 0); + done += ret; + } + assertArrayEquals(expected, readBuffer); + } + fs.delete(filePath, true); + } +} 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 0000000000000..d78e88b197220 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStream.java @@ -0,0 +1,156 @@ +/** + * 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 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.test.GenericTestUtils; +import org.apache.log4j.Level; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestDFSStripedOutputStream { + public static final Log LOG = LogFactory.getLog( + TestDFSStripedOutputStream.class); + + static { + GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL); + GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); + } + + private int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private Configuration conf; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final int stripesPerBlock = 4; + private final int blockSize = cellSize * stripesPerBlock; + + @Before + public void setup() throws IOException { + int numDNs = dataBlocks + parityBlocks + 2; + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testFileEmpty() throws Exception { + testOneFile("/EmptyFile", 0); + } + + @Test + public void testFileSmallerThanOneCell1() throws Exception { + testOneFile("/SmallerThanOneCell", 1); + } + + @Test + public void testFileSmallerThanOneCell2() throws Exception { + testOneFile("/SmallerThanOneCell", cellSize - 1); + } + + @Test + public void testFileEqualsWithOneCell() throws Exception { + testOneFile("/EqualsWithOneCell", cellSize); + } + + @Test + public void testFileSmallerThanOneStripe1() throws Exception { + testOneFile("/SmallerThanOneStripe", cellSize * dataBlocks - 1); + } + + @Test + public void testFileSmallerThanOneStripe2() throws Exception { + testOneFile("/SmallerThanOneStripe", cellSize + 123); + } + + @Test + public void testFileEqualsWithOneStripe() throws Exception { + testOneFile("/EqualsWithOneStripe", cellSize * dataBlocks); + } + + @Test + public void testFileMoreThanOneStripe1() throws Exception { + testOneFile("/MoreThanOneStripe1", cellSize * dataBlocks + 123); + } + + @Test + public void testFileMoreThanOneStripe2() throws Exception { + testOneFile("/MoreThanOneStripe2", cellSize * dataBlocks + + cellSize * dataBlocks + 123); + } + + @Test + public void testFileLessThanFullBlockGroup() throws Exception { + testOneFile("/LessThanFullBlockGroup", + cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); + } + + @Test + public void testFileFullBlockGroup() throws Exception { + testOneFile("/FullBlockGroup", blockSize * dataBlocks); + } + + @Test + public void testFileMoreThanABlockGroup1() throws Exception { + testOneFile("/MoreThanABlockGroup1", blockSize * dataBlocks + 123); + } + + @Test + public void testFileMoreThanABlockGroup2() throws Exception { + testOneFile("/MoreThanABlockGroup2", + blockSize * dataBlocks + cellSize+ 123); + } + + + @Test + public void testFileMoreThanABlockGroup3() throws Exception { + testOneFile("/MoreThanABlockGroup3", + blockSize * dataBlocks * 3 + cellSize * dataBlocks + + cellSize + 123); + } + + private void testOneFile(String src, int writeBytes) throws Exception { + src += "_" + writeBytes; + Path testPath = new Path(src); + + byte[] bytes = StripedFileTestUtil.generateBytes(writeBytes); + DFSTestUtil.writeFile(fs, testPath, new String(bytes)); + StripedFileTestUtil.waitBlockGroupsReported(fs, src); + + StripedFileTestUtil.checkData(fs, testPath, writeBytes, + new int[]{}, 0); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java new file mode 100644 index 0000000000000..f6c25661df68f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDFSStripedOutputStreamWithFailure.java @@ -0,0 +1,473 @@ +/** + * 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.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; +import java.util.ArrayList; +import java.util.Arrays; +import java.util.List; +import java.util.concurrent.atomic.AtomicInteger; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.CommonConfigurationKeysPublic; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.client.HdfsClientConfigKeys; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.DatanodeReportType; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenIdentifier; +import org.apache.hadoop.hdfs.security.token.block.BlockTokenSecretManager; +import org.apache.hadoop.hdfs.security.token.block.SecurityTestUtil; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManager; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.security.token.Token; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.hadoop.util.StringUtils; +import org.apache.log4j.Level; +import org.junit.Assert; +import org.junit.Test; + +import com.google.common.base.Preconditions; + + +public class TestDFSStripedOutputStreamWithFailure { + public static final Log LOG = LogFactory.getLog( + TestDFSStripedOutputStreamWithFailure.class); + static { + GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL); + GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); + } + + private static final int NUM_DATA_BLOCKS = StripedFileTestUtil.NUM_DATA_BLOCKS; + private static final int NUM_PARITY_BLOCKS = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private static final int CELL_SIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private static final int STRIPES_PER_BLOCK = 4; + private static final int BLOCK_SIZE = CELL_SIZE * STRIPES_PER_BLOCK; + private static final int BLOCK_GROUP_SIZE = BLOCK_SIZE * NUM_DATA_BLOCKS; + + private static final int FLUSH_POS + = 9*DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT + 1; + + static { + System.out.println("NUM_DATA_BLOCKS = " + NUM_DATA_BLOCKS); + System.out.println("NUM_PARITY_BLOCKS= " + NUM_PARITY_BLOCKS); + System.out.println("CELL_SIZE = " + CELL_SIZE + + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(CELL_SIZE, "B", 2) + ")"); + System.out.println("BLOCK_SIZE = " + BLOCK_SIZE + + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_SIZE, "B", 2) + ")"); + System.out.println("BLOCK_GROUP_SIZE = " + BLOCK_GROUP_SIZE + + " (=" + StringUtils.TraditionalBinaryPrefix.long2String(BLOCK_GROUP_SIZE, "B", 2) + ")"); + } + + static List newLengths() { + final List lengths = new ArrayList<>(); + lengths.add(FLUSH_POS + 2); + for(int b = 0; b <= 2; b++) { + for(int c = 0; c < STRIPES_PER_BLOCK*NUM_DATA_BLOCKS; c++) { + for(int delta = -1; delta <= 1; delta++) { + final int length = b*BLOCK_GROUP_SIZE + c*CELL_SIZE + delta; + System.out.println(lengths.size() + ": length=" + length + + ", (b, c, d) = (" + b + ", " + c + ", " + delta + ")"); + lengths.add(length); + } + } + } + return lengths; + } + + private static final int[][] dnIndexSuite = { + {0, 1}, + {0, 5}, + {0, 6}, + {0, 8}, + {1, 5}, + {1, 6}, + {6, 8}, + {0, 1, 2}, + {3, 4, 5}, + {0, 1, 6}, + {0, 5, 6}, + {0, 5, 8}, + {0, 6, 7}, + {5, 6, 7}, + {6, 7, 8}, + }; + + private int[] getKillPositions(int fileLen, int num) { + int[] positions = new int[num]; + for (int i = 0; i < num; i++) { + positions[i] = fileLen * (i + 1) / (num + 1); + } + return positions; + } + + private static final List LENGTHS = newLengths(); + + static int getLength(int i) { + return LENGTHS.get(i); + } + + private MiniDFSCluster cluster; + private DistributedFileSystem dfs; + private final Path dir = new Path("/" + + TestDFSStripedOutputStreamWithFailure.class.getSimpleName()); + + private void setup(Configuration conf) throws IOException { + final int numDNs = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + dfs = cluster.getFileSystem(); + dfs.mkdirs(dir); + dfs.setErasureCodingPolicy(dir, null); + } + + private void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + private HdfsConfiguration newHdfsConfiguration() { + final HdfsConfiguration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + conf.setLong(HdfsClientConfigKeys.DFS_CLIENT_SOCKET_TIMEOUT_KEY, 6000L); + conf.setInt(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + return conf; + } + + @Test(timeout=240000) + public void testDatanodeFailure56() throws Exception { + runTest(getLength(56)); + } + + @Test(timeout=240000) + public void testMultipleDatanodeFailure56() throws Exception { + runTestWithMultipleFailure(getLength(56)); + } + + @Test(timeout=240000) + public void testBlockTokenExpired() throws Exception { + final int length = NUM_DATA_BLOCKS * (BLOCK_SIZE - CELL_SIZE); + final HdfsConfiguration conf = newHdfsConfiguration(); + + conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); + conf.setInt(CommonConfigurationKeysPublic.IPC_CLIENT_CONNECT_MAX_RETRIES_KEY, 0); + // Set short retry timeouts so this test runs faster + conf.setInt(HdfsClientConfigKeys.Retry.WINDOW_BASE_KEY, 10); + for (int dn = 0; dn < 9; dn += 2) { + try { + setup(conf); + runTest(length, new int[]{length/2}, new int[]{dn}, true); + } catch (Exception e) { + LOG.error("failed, dn=" + dn + ", length=" + length); + throw e; + } finally { + tearDown(); + } + } + } + + @Test(timeout = 90000) + public void testAddBlockWhenNoSufficientDataBlockNumOfNodes() + throws IOException { + HdfsConfiguration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + try { + setup(conf); + ArrayList dataNodes = cluster.getDataNodes(); + // shutdown few datanodes to avoid getting sufficient data blocks number + // of datanodes + int killDns = dataNodes.size() / 2; + int numDatanodes = dataNodes.size() - killDns; + for (int i = 0; i < killDns; i++) { + cluster.stopDataNode(i); + } + cluster.restartNameNodes(); + cluster.triggerHeartbeats(); + DatanodeInfo[] info = dfs.getClient().datanodeReport( + DatanodeReportType.LIVE); + assertEquals("Mismatches number of live Dns ", numDatanodes, info.length); + final Path dirFile = new Path(dir, "ecfile"); + FSDataOutputStream out = null; + try { + out = dfs.create(dirFile, true); + out.write("something".getBytes()); + out.flush(); + out.close(); + Assert.fail("Failed to validate available dns against blkGroupSize"); + } catch (IOException ioe) { + // expected + GenericTestUtils.assertExceptionContains("Failed to get 6 nodes from" + + " namenode: blockGroupSize= 9, blocks.length= 5", ioe); + } + } finally { + tearDown(); + } + } + + @Test(timeout = 90000) + public void testAddBlockWhenNoSufficientParityNumOfNodes() throws IOException { + HdfsConfiguration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + try { + setup(conf); + ArrayList dataNodes = cluster.getDataNodes(); + // shutdown few data nodes to avoid writing parity blocks + int killDns = (NUM_PARITY_BLOCKS - 1); + int numDatanodes = dataNodes.size() - killDns; + for (int i = 0; i < killDns; i++) { + cluster.stopDataNode(i); + } + cluster.restartNameNodes(); + cluster.triggerHeartbeats(); + DatanodeInfo[] info = dfs.getClient().datanodeReport( + DatanodeReportType.LIVE); + assertEquals("Mismatches number of live Dns ", numDatanodes, info.length); + Path srcPath = new Path(dir, "testAddBlockWhenNoSufficientParityNodes"); + int fileLength = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE - 1000; + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + DFSTestUtil.writeFile(dfs, srcPath, new String(expected)); + LOG.info("writing finished. Seek and read the file to verify."); + StripedFileTestUtil.verifySeek(dfs, srcPath, fileLength); + } finally { + tearDown(); + } + } + + void runTest(final int length) { + final HdfsConfiguration conf = newHdfsConfiguration(); + for (int dn = 0; dn < 9; dn++) { + try { + setup(conf); + runTest(length, new int[]{length/2}, new int[]{dn}, false); + } catch (Throwable e) { + final String err = "failed, dn=" + dn + ", length=" + length + + StringUtils.stringifyException(e); + LOG.error(err); + Assert.fail(err); + } finally { + tearDown(); + } + } + } + + void runTestWithMultipleFailure(final int length) throws Exception { + final HdfsConfiguration conf = newHdfsConfiguration(); + for(int i=0;i ls = Arrays.asList(l1, l2); - LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null); + LocatedBlocks lbs = new LocatedBlocks(10, false, ls, l2, true, null, null); BlockLocation[] bs = DFSUtilClient.locatedBlocks2Locations(lbs); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java index 9cdb763f0323d..989e9fc0a5583 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestDatanodeConfig.java @@ -81,7 +81,7 @@ private static void clearBaseDir() throws IOException { public void testDataDirectories() throws IOException { File dataDir = new File(BASE_DIR, "data").getCanonicalFile(); Configuration conf = cluster.getConfiguration(0); - // 1. Test unsupported schema. Only "file:" is supported. + // 1. Test unsupported ecPolicy. Only "file:" is supported. String dnDir = makeURI("shv", null, fileAsURI(dataDir).getPath()); conf.set(DFSConfigKeys.DFS_DATANODE_DATA_DIR_KEY, dnDir); DataNode dn = null; @@ -97,7 +97,7 @@ public void testDataDirectories() throws IOException { } assertNull("Data-node startup should have failed.", dn); - // 2. Test "file:" schema and no schema (path-only). Both should work. + // 2. Test "file:" ecPolicy and no ecPolicy (path-only). Both should work. String dnDir1 = fileAsURI(dataDir).toString() + "1"; String dnDir2 = makeURI("file", "localhost", fileAsURI(dataDir).getPath() + "2"); 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 c1fdd2527eca4..3ae9e25b036fc 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 @@ -674,8 +674,7 @@ public void testRecommission() throws Exception { GenericTestUtils.waitFor(new Supplier() { @Override public Boolean get() { - BlockInfo 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++) { diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java index 567a70aa51425..64daeb3b96f6b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestEncryptionZones.java @@ -743,7 +743,7 @@ private static void mockCreate(ClientProtocol mcp, version, new byte[suite.getAlgorithmBlockSize()], new byte[suite.getAlgorithmBlockSize()], "fakeKey", "fakeVersion"), - (byte) 0)) + (byte) 0, null)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java new file mode 100644 index 0000000000000..0ababed63eb52 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestErasureCodingPolicies.java @@ -0,0 +1,249 @@ +/** + * 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.CommonConfigurationKeys; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.permission.FsPermission; +import org.apache.hadoop.hdfs.protocol.HdfsFileStatus; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.hdfs.server.namenode.INode; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.io.erasurecode.ECSchema; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.Collection; + +import static org.apache.hadoop.test.GenericTestUtils.assertExceptionContains; +import static org.junit.Assert.*; + +public class TestErasureCodingPolicies { + 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(1).build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + namesystem = cluster.getNamesystem(); + } + + @After + public void shutdownCluster() throws IOException { + cluster.shutdown(); + } + + @Test + public void testBasicSetECPolicy() + throws IOException, InterruptedException { + final Path testDir = new Path("/ec"); + fs.mkdir(testDir, FsPermission.getDirDefault()); + + /* Normal creation of an erasure coding directory */ + fs.getClient().setErasureCodingPolicy(testDir.toString(), null); + + /* Verify files under the directory 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 setting EC policy on non-empty directory only affects + * newly created files under the directory. + */ + final Path notEmpty = new Path("/nonEmpty"); + fs.mkdir(notEmpty, FsPermission.getDirDefault()); + final Path oldFile = new Path(notEmpty, "old"); + fs.create(oldFile); + fs.getClient().setErasureCodingPolicy(notEmpty.toString(), null); + final Path newFile = new Path(notEmpty, "new"); + fs.create(newFile); + INode oldInode = namesystem.getFSDirectory().getINode(oldFile.toString()); + assertFalse(oldInode.asFile().isStriped()); + INode newInode = namesystem.getFSDirectory().getINode(newFile.toString()); + assertTrue(newInode.asFile().isStriped()); + + /* Verify that nested EC policies not supported */ + final Path dir1 = new Path("/dir1"); + final Path dir2 = new Path(dir1, "dir2"); + fs.mkdir(dir1, FsPermission.getDirDefault()); + fs.getClient().setErasureCodingPolicy(dir1.toString(), null); + fs.mkdir(dir2, FsPermission.getDirDefault()); + try { + fs.getClient().setErasureCodingPolicy(dir2.toString(), null); + fail("Nested erasure coding policies"); + } catch (IOException e) { + assertExceptionContains("already has an erasure coding policy", e); + } + + /* Verify that EC policy cannot be set on a file */ + final Path fPath = new Path("/file"); + fs.create(fPath); + try { + fs.getClient().setErasureCodingPolicy(fPath.toString(), null); + fail("Erasure coding policy on file"); + } catch (IOException e) { + assertExceptionContains("erasure coding policy 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().setErasureCodingPolicy(srcECDir.toString(), null); + fs.getClient().setErasureCodingPolicy(dstECDir.toString(), null); + final Path srcFile = new Path(srcECDir, "foo"); + fs.create(srcFile); + + // Test move dir + // Move EC dir under non-EC dir + final Path newDir = new Path("/srcEC_new"); + fs.rename(srcECDir, newDir); + fs.rename(newDir, srcECDir); // move back + + // Move EC dir under another EC dir + fs.rename(srcECDir, dstECDir); + fs.rename(new Path("/dstEC/srcEC"), srcECDir); // move back + + // Test move file + /* Verify that a file can be moved between 2 EC dirs */ + fs.rename(srcFile, dstECDir); + fs.rename(new Path(dstECDir, "foo"), srcECDir); // move back + + /* Verify that a file can be moved from a non-EC dir to an EC dir */ + final Path nonECDir = new Path("/nonEC"); + fs.mkdir(nonECDir, FsPermission.getDirDefault()); + fs.rename(srcFile, nonECDir); + + /* Verify that a file can be moved from an EC dir to a non-EC dir */ + final Path nonECFile = new Path(nonECDir, "nonECFile"); + fs.create(nonECFile); + fs.rename(nonECFile, dstECDir); + } + + @Test + public void testReplication() throws IOException { + final Path testDir = new Path("/ec"); + fs.mkdir(testDir, FsPermission.getDirDefault()); + fs.setErasureCodingPolicy(testDir, null); + final Path fooFile = new Path(testDir, "foo"); + // create ec file with replication=0 + fs.create(fooFile, FsPermission.getFileDefault(), true, + conf.getInt(CommonConfigurationKeys.IO_FILE_BUFFER_SIZE_KEY, 4096), + (short)0, fs.getDefaultBlockSize(fooFile), null); + // set replication should be a no-op + fs.setReplication(fooFile, (short) 3); + } + + @Test + public void testGetErasureCodingPolicyWithSystemDefaultECPolicy() throws Exception { + String src = "/ec"; + final Path ecDir = new Path(src); + fs.mkdir(ecDir, FsPermission.getDirDefault()); + // dir EC policy should be null + assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy()); + // dir EC policy after setting + fs.getClient().setErasureCodingPolicy(src, null); //Default one will be used. + ErasureCodingPolicy sysDefaultECPolicy = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + verifyErasureCodingInfo(src, sysDefaultECPolicy); + fs.create(new Path(ecDir, "child1")).close(); + // verify for the files in ec dir + verifyErasureCodingInfo(src + "/child1", sysDefaultECPolicy); + } + + @Test + public void testGetErasureCodingPolicy() throws Exception { + ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager.getSystemPolices(); + assertTrue("System ecPolicies should be of only 1 for now", + sysECPolicies.length == 1); + + ErasureCodingPolicy usingECPolicy = sysECPolicies[0]; + String src = "/ec2"; + final Path ecDir = new Path(src); + fs.mkdir(ecDir, FsPermission.getDirDefault()); + // dir ECInfo before being set + assertNull(fs.getClient().getFileInfo(src).getErasureCodingPolicy()); + // dir ECInfo after set + fs.getClient().setErasureCodingPolicy(src, usingECPolicy); + verifyErasureCodingInfo(src, usingECPolicy); + fs.create(new Path(ecDir, "child1")).close(); + // verify for the files in ec dir + verifyErasureCodingInfo(src + "/child1", usingECPolicy); + } + + private void verifyErasureCodingInfo( + String src, ErasureCodingPolicy usingECPolicy) throws IOException { + HdfsFileStatus hdfsFileStatus = fs.getClient().getFileInfo(src); + ErasureCodingPolicy ecPolicy = hdfsFileStatus.getErasureCodingPolicy(); + assertNotNull(ecPolicy); + assertEquals("Actually used ecPolicy should be equal with target ecPolicy", + usingECPolicy, ecPolicy); + } + + @Test + public void testCreationErasureCodingZoneWithInvalidPolicy() + throws IOException { + ECSchema rsSchema = new ECSchema("rs", 4, 2); + String policyName = "RS-4-2-128k"; + int cellSize = 128 * 1024; + ErasureCodingPolicy ecPolicy= + new ErasureCodingPolicy(policyName,rsSchema,cellSize); + String src = "/ecZone4-2"; + final Path ecDir = new Path(src); + try { + fs.mkdir(ecDir, FsPermission.getDirDefault()); + fs.getClient().setErasureCodingPolicy(src, ecPolicy); + fail("HadoopIllegalArgumentException should be thrown for" + + "setting an invalid erasure coding policy"); + } catch (Exception e) { + assertExceptionContains("Policy [ RS-4-2-128k ] does not match " + + "any of the supported policies",e); + } + } + + @Test + public void testGetAllErasureCodingPolicies() throws Exception { + ErasureCodingPolicy[] sysECPolicies = ErasureCodingPolicyManager + .getSystemPolices(); + assertTrue("System ecPolicies should be of only 1 for now", + sysECPolicies.length == 1); + + Collection allECPolicies = fs + .getAllErasureCodingPolicies(); + assertTrue("All ecPolicies should be of only 1 for now", + allECPolicies.size() == 1); + assertEquals("Erasure coding policy mismatches", + sysECPolicies[0], allECPolicies.iterator().next()); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java new file mode 100644 index 0000000000000..64063d20eb09b --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestFileStatusWithECPolicy.java @@ -0,0 +1,65 @@ +package org.apache.hadoop.hdfs; + +import static org.junit.Assert.assertNotNull; +import static org.junit.Assert.assertNull; +import static org.junit.Assert.assertTrue; + +import java.io.IOException; + +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.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +public class TestFileStatusWithECPolicy { + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private DFSClient client; + + @Before + public void before() throws IOException { + cluster = + new MiniDFSCluster.Builder(new Configuration()).numDataNodes(1).build(); + cluster.waitActive(); + fs = cluster.getFileSystem(); + client = fs.getClient(); + } + + @After + public void after() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testFileStatusWithECPolicy() throws Exception { + // test directory doesn't have an EC policy + final Path dir = new Path("/foo"); + assertTrue(fs.mkdir(dir, FsPermission.getDirDefault())); + assertNull(client.getFileInfo(dir.toString()).getErasureCodingPolicy()); + // test file doesn't have an EC policy + final Path file = new Path(dir, "foo"); + fs.create(file).close(); + assertNull(client.getFileInfo(file.toString()).getErasureCodingPolicy()); + fs.delete(file, true); + + final ErasureCodingPolicy ecPolicy1 = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + // set EC policy on dir + fs.setErasureCodingPolicy(dir, ecPolicy1); + final ErasureCodingPolicy ecPolicy2 = client.getFileInfo(dir.toUri().getPath()).getErasureCodingPolicy(); + assertNotNull(ecPolicy2); + assertTrue(ecPolicy1.equals(ecPolicy2)); + + // test file doesn't have an EC policy + fs.create(file).close(); + final ErasureCodingPolicy ecPolicy3 = + fs.getClient().getFileInfo(file.toUri().getPath()).getErasureCodingPolicy(); + assertNotNull(ecPolicy3); + assertTrue(ecPolicy1.equals(ecPolicy3)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java index bacdc9946ac59..20596c56b9516 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestLease.java @@ -354,12 +354,12 @@ public void testFactory() throws Exception { Mockito.doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)).when(mcp).getFileInfo(anyString()); + 1010, 0, null, (byte) 0, null)).when(mcp).getFileInfo(anyString()); Mockito .doReturn( new HdfsFileStatus(0, false, 1, 1024, 0, 0, new FsPermission( (short) 777), "owner", "group", new byte[0], new byte[0], - 1010, 0, null, (byte) 0)) + 1010, 0, null, (byte) 0, null)) .when(mcp) .create(anyString(), (FsPermission) anyObject(), anyString(), (EnumSetWritable) anyObject(), anyBoolean(), diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java new file mode 100644 index 0000000000000..9cf970b7f4d97 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithDecoding.java @@ -0,0 +1,362 @@ +/** + * 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.BlockLocation; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.Block; +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.protocol.LocatedStripedBlock; +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.datanode.DataNode; +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.util.StripedBlockUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.File; +import java.io.FileOutputStream; +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; + +public class TestReadStripedFileWithDecoding { + static final Log LOG = LogFactory.getLog(TestReadStripedFileWithDecoding.class); + + private MiniDFSCluster cluster; + private DistributedFileSystem fs; + private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final int smallFileLength = blockSize * dataBlocks - 123; + private final int largeFileLength = blockSize * dataBlocks + 123; + private final int[] fileLengths = {smallFileLength, largeFileLength}; + private final int[] dnFailureNums = {1, 2, 3}; + + @Before + public void setup() throws IOException { + Configuration conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + cluster = new MiniDFSCluster.Builder(new HdfsConfiguration()) + .numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Shutdown tolerable number of Datanode before reading. + * Verify the decoding works correctly. + */ + @Test(timeout=300000) + public void testReadWithDNFailure() throws Exception { + for (int fileLength : fileLengths) { + for (int dnFailureNum : dnFailureNums) { + try { + // setup a new cluster with no dead datanode + setup(); + testReadWithDNFailure(fileLength, dnFailureNum); + } catch (IOException ioe) { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + LOG.error("Failed to read file with DN failure:" + + " fileType = "+ fileType + + ", dnFailureNum = " + dnFailureNum); + } finally { + // tear down the cluster + tearDown(); + } + } + } + } + + /** + * Corrupt tolerable number of block before reading. + * Verify the decoding works correctly. + */ + @Test(timeout=300000) + public void testReadCorruptedData() throws IOException { + for (int fileLength : fileLengths) { + for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) { + for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) { + String src = "/corrupted_" + dataDelNum + "_" + parityDelNum; + testReadWithBlockCorrupted(src, fileLength, + dataDelNum, parityDelNum, false); + } + } + } + } + + /** + * Delete tolerable number of block before reading. + * Verify the decoding works correctly. + */ + @Test(timeout=300000) + public void testReadCorruptedDataByDeleting() throws IOException { + for (int fileLength : fileLengths) { + for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) { + for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) { + String src = "/deleted_" + dataDelNum + "_" + parityDelNum; + testReadWithBlockCorrupted(src, fileLength, + dataDelNum, parityDelNum, true); + } + } + } + } + + private int findFirstDataNode(Path file, long length) throws IOException { + BlockLocation[] locs = fs.getFileBlockLocations(file, 0, length); + String name = (locs[0].getNames())[0]; + int dnIndex = 0; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + return dnIndex; + } + dnIndex++; + } + return -1; + } + + private void verifyRead(Path testPath, int length, byte[] expected) + throws IOException { + byte[] buffer = new byte[length + 100]; + StripedFileTestUtil.verifyLength(fs, testPath, length); + StripedFileTestUtil.verifyPread(fs, testPath, length, expected, buffer); + StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, buffer); + StripedFileTestUtil.verifyStatefulRead(fs, testPath, length, expected, + ByteBuffer.allocate(length + 100)); + StripedFileTestUtil.verifySeek(fs, testPath, length); + } + + private void testReadWithDNFailure(int fileLength, int dnFailureNum) + throws Exception { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + String src = "/dnFailure_" + dnFailureNum + "_" + fileType; + LOG.info("testReadWithDNFailure: file = " + src + + ", fileSize = " + fileLength + + ", dnFailureNum = " + dnFailureNum); + + Path testPath = new Path(src); + final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength); + DFSTestUtil.writeFile(fs, testPath, bytes); + StripedFileTestUtil.waitBlockGroupsReported(fs, src); + + // shut down the DN that holds an internal data block + BlockLocation[] locs = fs.getFileBlockLocations(testPath, cellSize * 5, + cellSize); + for (int failedDnIdx = 0; failedDnIdx < dnFailureNum; failedDnIdx++) { + String name = (locs[0].getNames())[failedDnIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + } + } + } + + // check file length, pread, stateful read and seek + verifyRead(testPath, fileLength, bytes); + } + + /** + * After reading a corrupted block, make sure the client can correctly report + * the corruption to the NameNode. + */ + @Test + public void testReportBadBlock() throws IOException { + // create file + final Path file = new Path("/corrupted"); + final int length = 10; // length of "corruption" + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(fs, file, bytes); + + // corrupt the first data block + int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); + Assert.assertNotEquals(-1, dnIndex); + LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() + .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, + cellSize, dataBlocks, parityBlocks); + // find the first block file + File storageDir = cluster.getInstanceStorageDir(dnIndex, 0); + File blkFile = MiniDFSCluster.getBlockFile(storageDir, blks[0].getBlock()); + Assert.assertTrue("Block file does not exist", blkFile.exists()); + // corrupt the block file + LOG.info("Deliberately corrupting file " + blkFile.getName()); + try (FileOutputStream out = new FileOutputStream(blkFile)) { + out.write("corruption".getBytes()); + } + + // disable the heartbeat from DN so that the corrupted block record is kept + // in NameNode + for (DataNode dn : cluster.getDataNodes()) { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); + } + + try { + // do stateful read + StripedFileTestUtil.verifyStatefulRead(fs, file, length, bytes, + ByteBuffer.allocate(1024)); + + // check whether the corruption has been reported to the NameNode + final FSNamesystem ns = cluster.getNamesystem(); + final BlockManager bm = ns.getBlockManager(); + BlockInfo blockInfo = (ns.getFSDirectory().getINode4Write(file.toString()) + .asFile().getBlocks())[0]; + Assert.assertEquals(1, bm.getCorruptReplicas(blockInfo).size()); + } finally { + for (DataNode dn : cluster.getDataNodes()) { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false); + } + } + } + + @Test + public void testInvalidateBlock() throws IOException { + final Path file = new Path("/invalidate"); + final int length = 10; + final byte[] bytes = StripedFileTestUtil.generateBytes(length); + DFSTestUtil.writeFile(fs, file, bytes); + + int dnIndex = findFirstDataNode(file, cellSize * dataBlocks); + Assert.assertNotEquals(-1, dnIndex); + LocatedStripedBlock slb = (LocatedStripedBlock)fs.getClient() + .getLocatedBlocks(file.toString(), 0, cellSize * dataBlocks).get(0); + final LocatedBlock[] blks = StripedBlockUtil.parseStripedBlockGroup(slb, + cellSize, dataBlocks, parityBlocks); + final Block b = blks[0].getBlock().getLocalBlock(); + + DataNode dn = cluster.getDataNodes().get(dnIndex); + // disable the heartbeat from DN so that the invalidated block record is kept + // in NameNode until heartbeat expires and NN mark the dn as dead + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, true); + + try { + // delete the file + fs.delete(file, true); + // check the block is added to invalidateBlocks + final FSNamesystem fsn = cluster.getNamesystem(); + final BlockManager bm = fsn.getBlockManager(); + DatanodeDescriptor dnd = NameNodeAdapter.getDatanode(fsn, dn.getDatanodeId()); + Assert.assertTrue(bm.containsInvalidateBlock( + blks[0].getLocations()[0], b) || dnd.containsInvalidateBlock(b)); + } finally { + DataNodeTestUtils.setHeartbeatsDisabledForTests(dn, false); + } + } + + /** + * Test reading a file with some blocks(data blocks or parity blocks or both) + * deleted or corrupted. + * @param src file path + * @param fileLength file length + * @param dataBlkDelNum the deleted or corrupted number of data blocks. + * @param parityBlkDelNum the deleted or corrupted number of parity blocks. + * @param deleteBlockFile whether block file is deleted or corrupted. + * true is to delete the block file. + * false is to corrupt the content of the block file. + * @throws IOException + */ + private void testReadWithBlockCorrupted(String src, int fileLength, + int dataBlkDelNum, int parityBlkDelNum, boolean deleteBlockFile) + throws IOException { + LOG.info("testReadWithBlockCorrupted: file = " + src + + ", dataBlkDelNum = " + dataBlkDelNum + + ", parityBlkDelNum = " + parityBlkDelNum + + ", deleteBlockFile? " + deleteBlockFile); + int recoverBlkNum = dataBlkDelNum + parityBlkDelNum; + Assert.assertTrue("dataBlkDelNum and parityBlkDelNum should be positive", + dataBlkDelNum >= 0 && parityBlkDelNum >= 0); + Assert.assertTrue("The sum of dataBlkDelNum and parityBlkDelNum " + + "should be between 1 ~ " + parityBlocks, recoverBlkNum <= parityBlocks); + + // write a file with the length of writeLen + Path srcPath = new Path(src); + final byte[] bytes = StripedFileTestUtil.generateBytes(fileLength); + DFSTestUtil.writeFile(fs, srcPath, bytes); + + // delete or corrupt some blocks + corruptBlocks(srcPath, dataBlkDelNum, parityBlkDelNum, deleteBlockFile); + + // check the file can be read after some blocks were deleted + verifyRead(srcPath, fileLength, bytes); + } + + private void corruptBlocks(Path srcPath, int dataBlkDelNum, + int parityBlkDelNum, boolean deleteBlockFile) throws IOException { + int recoverBlkNum = dataBlkDelNum + parityBlkDelNum; + + LocatedBlocks locatedBlocks = getLocatedBlocks(srcPath); + LocatedStripedBlock lastBlock = + (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + + int[] delDataBlkIndices = StripedFileTestUtil.randomArray(0, dataBlocks, + dataBlkDelNum); + Assert.assertNotNull(delDataBlkIndices); + int[] delParityBlkIndices = StripedFileTestUtil.randomArray(dataBlocks, + dataBlocks + parityBlocks, parityBlkDelNum); + Assert.assertNotNull(delParityBlkIndices); + + int[] delBlkIndices = new int[recoverBlkNum]; + System.arraycopy(delDataBlkIndices, 0, + delBlkIndices, 0, delDataBlkIndices.length); + System.arraycopy(delParityBlkIndices, 0, + delBlkIndices, delDataBlkIndices.length, delParityBlkIndices.length); + + ExtendedBlock[] delBlocks = new ExtendedBlock[recoverBlkNum]; + for (int i = 0; i < recoverBlkNum; i++) { + delBlocks[i] = StripedBlockUtil + .constructInternalBlock(lastBlock.getBlock(), + cellSize, dataBlocks, delBlkIndices[i]); + if (deleteBlockFile) { + // delete the block file + cluster.corruptBlockOnDataNodesByDeletingBlockFile(delBlocks[i]); + } else { + // corrupt the block file + cluster.corruptBlockOnDataNodes(delBlocks[i]); + } + } + } + + private LocatedBlocks getLocatedBlocks(Path filePath) throws IOException { + return fs.getClient().getLocatedBlocks(filePath.toString(), + 0, Long.MAX_VALUE); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.java new file mode 100644 index 0000000000000..f521d8edb1839 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestReadStripedFileWithMissingBlocks.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.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; + +/** + * Test reading a striped file when some of its blocks are missing (not included + * in the block locations returned by the NameNode). + */ +public class TestReadStripedFileWithMissingBlocks { + public static final Log LOG = LogFactory + .getLog(TestReadStripedFileWithMissingBlocks.class); + private static MiniDFSCluster cluster; + private static DistributedFileSystem fs; + private static Configuration conf = new HdfsConfiguration(); + private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final int fileLength = blockSize * dataBlocks + 123; + + @Before + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_KEY, 0); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testReadFileWithMissingBlocks1() throws Exception { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 0); + } + + @Test + public void testReadFileWithMissingBlocks2() throws Exception { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 1); + } + + @Test + public void testReadFileWithMissingBlocks3() throws Exception { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 1, 2); + } + + @Test + public void testReadFileWithMissingBlocks4() throws Exception { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 0); + } + + @Test + public void testReadFileWithMissingBlocks5() throws Exception { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 2, 1); + } + + @Test + public void testReadFileWithMissingBlocks6() throws Exception { + readFileWithMissingBlocks(new Path("/foo"), fileLength, 3, 0); + } + + private void readFileWithMissingBlocks(Path srcPath, int fileLength, + int missingDataNum, int missingParityNum) + throws Exception { + LOG.info("readFileWithMissingBlocks: (" + missingDataNum + "," + + missingParityNum + ")"); + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + StripedFileTestUtil.waitBlockGroupsReported(fs, srcPath.toUri().getPath()); + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); + int dataBlocks = (fileLength - 1) / cellSize + 1; + BlockLocation[] locs = fs.getFileBlockLocations(srcPath, 0, cellSize); + + int[] missingDataNodes = new int[missingDataNum + missingParityNum]; + for (int i = 0; i < missingDataNum; i++) { + missingDataNodes[i] = i; + } + for (int i = 0; i < missingParityNum; i++) { + missingDataNodes[i + missingDataNum] = i + + Math.min(StripedFileTestUtil.NUM_DATA_BLOCKS, dataBlocks); + } + stopDataNodes(locs, missingDataNodes); + + // make sure there are missing block locations + BlockLocation[] newLocs = fs.getFileBlockLocations(srcPath, 0, cellSize); + Assert.assertTrue(newLocs[0].getNames().length < locs[0].getNames().length); + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + smallBuf); + StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + // delete the file + fs.delete(srcPath, true); + } + + private void stopDataNodes(BlockLocation[] locs, int[] datanodes) + throws IOException { + if (locs != null && locs.length > 0) { + for (int failedDNIdx : datanodes) { + String name = (locs[0].getNames())[failedDNIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + cluster.setDataNodeDead(dn.getDatanodeId()); + LOG.info("stop datanode " + failedDNIdx); + break; + } + } + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java new file mode 100644 index 0000000000000..75dc6a02ac42a --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestRecoverStripedFile.java @@ -0,0 +1,418 @@ +/** + * 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.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.ByteArrayOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.concurrent.ThreadLocalRandom; + +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.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.LocatedBlocks; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +public class TestRecoverStripedFile { + public static final Log LOG = LogFactory.getLog(TestRecoverStripedFile.class); + + private static final int dataBlkNum = StripedFileTestUtil.NUM_DATA_BLOCKS; + private static final int parityBlkNum = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private static final int blockSize = cellSize * 3; + private static final int groupSize = dataBlkNum + parityBlkNum; + private static final int dnNum = groupSize + parityBlkNum; + + private MiniDFSCluster cluster; + private Configuration conf; + private DistributedFileSystem fs; + // Map: DatanodeID -> datanode index in cluster + private Map dnMap = new HashMap(); + + @Before + public void setup() throws IOException { + conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setInt(DFSConfigKeys.DFS_DATANODE_STRIPED_READ_BUFFER_SIZE_KEY, cellSize - 1); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(dnNum).build();; + cluster.waitActive(); + + fs = cluster.getFileSystem(); + fs.getClient().setErasureCodingPolicy("/", null); + + List datanodes = cluster.getDataNodes(); + for (int i = 0; i < dnNum; i++) { + dnMap.put(datanodes.get(i).getDatanodeId(), i); + } + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test(timeout = 120000) + public void testRecoverOneParityBlock() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverOneParityBlock", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneParityBlock1() throws Exception { + int fileLen = cellSize + cellSize/10; + assertFileBlocksRecovery("/testRecoverOneParityBlock1", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneParityBlock2() throws Exception { + int fileLen = 1; + assertFileBlocksRecovery("/testRecoverOneParityBlock2", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneParityBlock3() throws Exception { + int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverOneParityBlock3", fileLen, 0, 1); + } + + @Test(timeout = 120000) + public void testRecoverThreeParityBlocks() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverThreeParityBlocks", fileLen, 0, 3); + } + + @Test(timeout = 120000) + public void testRecoverThreeDataBlocks() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverThreeDataBlocks", fileLen, 1, 3); + } + + @Test(timeout = 120000) + public void testRecoverThreeDataBlocks1() throws Exception { + int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverThreeDataBlocks1", fileLen, 1, 3); + } + + @Test(timeout = 120000) + public void testRecoverOneDataBlock() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverOneDataBlock", fileLen, 1, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneDataBlock1() throws Exception { + int fileLen = cellSize + cellSize/10; + assertFileBlocksRecovery("/testRecoverOneDataBlock1", fileLen, 1, 1); + } + + @Test(timeout = 120000) + public void testRecoverOneDataBlock2() throws Exception { + int fileLen = 1; + assertFileBlocksRecovery("/testRecoverOneDataBlock2", fileLen, 1, 1); + } + + @Test(timeout = 120000) + public void testRecoverAnyBlocks() throws Exception { + int fileLen = 3 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverAnyBlocks", fileLen, 2, 2); + } + + @Test(timeout = 120000) + public void testRecoverAnyBlocks1() throws Exception { + int fileLen = 10 * blockSize + blockSize/10; + assertFileBlocksRecovery("/testRecoverAnyBlocks1", fileLen, 2, 3); + } + + /** + * Test the file blocks recovery. + * 1. Check the replica is recovered in the target datanode, + * and verify the block replica length, generationStamp and content. + * 2. Read the file and verify content. + */ + private void assertFileBlocksRecovery(String fileName, int fileLen, + int recovery, int toRecoverBlockNum) throws Exception { + if (recovery != 0 && recovery != 1 && recovery != 2) { + Assert.fail("Invalid recovery: 0 is to recovery parity blocks," + + "1 is to recovery data blocks, 2 is any."); + } + if (toRecoverBlockNum < 1 || toRecoverBlockNum > parityBlkNum) { + Assert.fail("toRecoverBlockNum should be between 1 ~ " + parityBlkNum); + } + + Path file = new Path(fileName); + + final byte[] data = new byte[fileLen]; + ThreadLocalRandom.current().nextBytes(data); + DFSTestUtil.writeFile(fs, file, data); + StripedFileTestUtil.waitBlockGroupsReported(fs, fileName); + + LocatedBlocks locatedBlocks = getLocatedBlocks(file); + assertEquals(locatedBlocks.getFileLength(), fileLen); + + LocatedStripedBlock lastBlock = + (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + + DatanodeInfo[] storageInfos = lastBlock.getLocations(); + int[] indices = lastBlock.getBlockIndices(); + + BitSet bitset = new BitSet(dnNum); + for (DatanodeInfo storageInfo : storageInfos) { + bitset.set(dnMap.get(storageInfo)); + } + + int[] toDead = new int[toRecoverBlockNum]; + int n = 0; + for (int i = 0; i < indices.length; i++) { + if (n < toRecoverBlockNum) { + if (recovery == 0) { + if (indices[i] >= dataBlkNum) { + toDead[n++] = i; + } + } else if (recovery == 1) { + if (indices[i] < dataBlkNum) { + toDead[n++] = i; + } + } else { + toDead[n++] = i; + } + } else { + break; + } + } + + DatanodeInfo[] dataDNs = new DatanodeInfo[toRecoverBlockNum]; + int[] deadDnIndices = new int[toRecoverBlockNum]; + ExtendedBlock[] blocks = new ExtendedBlock[toRecoverBlockNum]; + File[] replicas = new File[toRecoverBlockNum]; + File[] metadatas = new File[toRecoverBlockNum]; + byte[][] replicaContents = new byte[toRecoverBlockNum][]; + for (int i = 0; i < toRecoverBlockNum; i++) { + dataDNs[i] = storageInfos[toDead[i]]; + deadDnIndices[i] = dnMap.get(dataDNs[i]); + + // Check the block replica file on deadDn before it dead. + blocks[i] = StripedBlockUtil.constructInternalBlock( + lastBlock.getBlock(), cellSize, dataBlkNum, indices[toDead[i]]); + replicas[i] = cluster.getBlockFile(deadDnIndices[i], blocks[i]); + metadatas[i] = cluster.getBlockMetadataFile(deadDnIndices[i], blocks[i]); + // the block replica on the datanode should be the same as expected + assertEquals(replicas[i].length(), + StripedBlockUtil.getInternalBlockLength( + lastBlock.getBlockSize(), cellSize, dataBlkNum, indices[toDead[i]])); + assertTrue(metadatas[i].getName(). + endsWith(blocks[i].getGenerationStamp() + ".meta")); + replicaContents[i] = readReplica(replicas[i]); + } + + int cellsNum = (fileLen - 1) / cellSize + 1; + int groupSize = Math.min(cellsNum, dataBlkNum) + parityBlkNum; + + try { + DatanodeID[] dnIDs = new DatanodeID[toRecoverBlockNum]; + for (int i = 0; i < toRecoverBlockNum; i++) { + /* + * Kill the datanode which contains one replica + * We need to make sure it dead in namenode: clear its update time and + * trigger NN to check heartbeat. + */ + DataNode dn = cluster.getDataNodes().get(deadDnIndices[i]); + dn.shutdown(); + dnIDs[i] = dn.getDatanodeId(); + } + setDataNodesDead(dnIDs); + + // Check the locatedBlocks of the file again + locatedBlocks = getLocatedBlocks(file); + lastBlock = (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + storageInfos = lastBlock.getLocations(); + assertEquals(storageInfos.length, groupSize - toRecoverBlockNum); + + int[] targetDNs = new int[dnNum - groupSize]; + n = 0; + for (int i = 0; i < dnNum; i++) { + if (!bitset.get(i)) { // not contain replica of the block. + targetDNs[n++] = i; + } + } + + waitForRecoveryFinished(file, groupSize); + + targetDNs = sortTargetsByReplicas(blocks, targetDNs); + + // Check the replica on the new target node. + for (int i = 0; i < toRecoverBlockNum; i++) { + File replicaAfterRecovery = cluster.getBlockFile(targetDNs[i], blocks[i]); + File metadataAfterRecovery = + cluster.getBlockMetadataFile(targetDNs[i], blocks[i]); + assertEquals(replicaAfterRecovery.length(), replicas[i].length()); + assertTrue(metadataAfterRecovery.getName(). + endsWith(blocks[i].getGenerationStamp() + ".meta")); + byte[] replicaContentAfterRecovery = readReplica(replicaAfterRecovery); + + Assert.assertArrayEquals(replicaContents[i], replicaContentAfterRecovery); + } + } finally { + for (int i = 0; i < toRecoverBlockNum; i++) { + restartDataNode(toDead[i]); + } + cluster.waitActive(); + } + fs.delete(file, true); + } + + private void setDataNodesDead(DatanodeID[] dnIDs) throws IOException { + for (DatanodeID dn : dnIDs) { + DatanodeDescriptor dnd = + NameNodeAdapter.getDatanode(cluster.getNamesystem(), dn); + DFSTestUtil.setDatanodeDead(dnd); + } + + BlockManagerTestUtil.checkHeartbeat(cluster.getNamesystem().getBlockManager()); + } + + private void restartDataNode(int dn) { + try { + cluster.restartDataNode(dn, true, true); + } catch (IOException e) { + } + } + + private int[] sortTargetsByReplicas(ExtendedBlock[] blocks, int[] targetDNs) { + int[] result = new int[blocks.length]; + for (int i = 0; i < blocks.length; i++) { + result[i] = -1; + for (int j = 0; j < targetDNs.length; j++) { + if (targetDNs[j] != -1) { + File replica = cluster.getBlockFile(targetDNs[j], blocks[i]); + if (replica != null) { + result[i] = targetDNs[j]; + targetDNs[j] = -1; + break; + } + } + } + if (result[i] == -1) { + Assert.fail("Failed to recover striped block: " + blocks[i].getBlockId()); + } + } + return result; + } + + private byte[] readReplica(File replica) throws IOException { + int length = (int)replica.length(); + ByteArrayOutputStream content = new ByteArrayOutputStream(length); + FileInputStream in = new FileInputStream(replica); + try { + byte[] buffer = new byte[1024]; + int total = 0; + while (total < length) { + int n = in.read(buffer); + if (n <= 0) { + break; + } + content.write(buffer, 0, n); + total += n; + } + if (total < length) { + Assert.fail("Failed to read all content of replica"); + } + return content.toByteArray(); + } finally { + in.close(); + } + } + + private LocatedBlocks waitForRecoveryFinished(Path file, int groupSize) + throws Exception { + final int ATTEMPTS = 60; + for (int i = 0; i < ATTEMPTS; i++) { + LocatedBlocks locatedBlocks = getLocatedBlocks(file); + LocatedStripedBlock lastBlock = + (LocatedStripedBlock)locatedBlocks.getLastLocatedBlock(); + DatanodeInfo[] storageInfos = lastBlock.getLocations(); + if (storageInfos.length >= groupSize) { + return locatedBlocks; + } + Thread.sleep(1000); + } + throw new IOException ("Time out waiting for EC block recovery."); + } + + private LocatedBlocks getLocatedBlocks(Path file) throws IOException { + return fs.getClient().getLocatedBlocks(file.toString(), 0, Long.MAX_VALUE); + } + + /* + * Tests that processErasureCodingTasks should not throw exceptions out due to + * invalid ECTask submission. + */ + @Test + public void testProcessErasureCodingTasksSubmitionShouldSucceed() + throws Exception { + DataNode dataNode = cluster.dataNodes.get(0).datanode; + + // Pack invalid(dummy) parameters in ecTasks. Irrespective of parameters, each task + // thread pool submission should succeed, so that it will not prevent + // processing other tasks in the list if any exceptions. + int size = cluster.dataNodes.size(); + short[] liveIndices = new short[size]; + DatanodeInfo[] dataDNs = new DatanodeInfo[size + 1]; + DatanodeStorageInfo targetDnInfos_1 = BlockManagerTestUtil + .newDatanodeStorageInfo(DFSTestUtil.getLocalDatanodeDescriptor(), + new DatanodeStorage("s01")); + DatanodeStorageInfo[] dnStorageInfo = new DatanodeStorageInfo[] { + targetDnInfos_1 }; + + BlockECRecoveryInfo invalidECInfo = new BlockECRecoveryInfo( + new ExtendedBlock("bp-id", 123456), dataDNs, dnStorageInfo, liveIndices, + ErasureCodingPolicyManager.getSystemDefaultPolicy()); + List ecTasks = new ArrayList(); + ecTasks.add(invalidECInfo); + dataNode.getErasureCodingWorker().processErasureCodingTasks(ecTasks); + } +} 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 6cea7e89517c3..6b4e46a94923b 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,7 @@ public void testSafeModeWhenZeroBlockLocations() throws IOException { if(cluster!= null) cluster.shutdown(); } } - + void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOException { FileStatus stat = fs.getFileStatus(fileName); try { @@ -560,7 +560,7 @@ void checkGetBlockLocationsWorks(FileSystem fs, Path fileName) throws IOExceptio } 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); } } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java new file mode 100644 index 0000000000000..9853b8a59fd05 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestSafeModeWithStripedFile.java @@ -0,0 +1,154 @@ +/** + * 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.collect.Lists; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.protocol.DatanodeInfo; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.protocol.LocatedBlocks; +import org.apache.hadoop.hdfs.server.namenode.NameNode; +import org.apache.hadoop.hdfs.server.namenode.NameNodeAdapter; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.util.List; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; +import static org.junit.Assert.assertTrue; + +public class TestSafeModeWithStripedFile { + + static final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS; + static final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS; + static final int numDNs = DATA_BLK_NUM + PARITY_BLK_NUM; + static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + static final int blockSize = cellSize * 2; + + static MiniDFSCluster cluster; + static Configuration conf; + + @Before + public void setup() throws IOException { + conf = new HdfsConfiguration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 100); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); + cluster.waitActive(); + + } + + @After + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testStripedFile0() throws IOException { + doTest(cellSize, 1); + } + + @Test + public void testStripedFile1() throws IOException { + doTest(cellSize * 5, 5); + } + + /** + * This util writes a small block group whose size is given by caller. + * Then write another 2 full stripe blocks. + * Then shutdown all DNs and start again one by one. and verify the safemode + * status accordingly. + * + * @param smallSize file size of the small block group + * @param minStorages minimum replicas needed by the block so it can be safe + */ + private void doTest(int smallSize, int minStorages) throws IOException { + FileSystem fs = cluster.getFileSystem(); + // add 1 block + byte[] data = StripedFileTestUtil.generateBytes(smallSize); + Path smallFilePath = new Path("/testStripedFile_" + smallSize); + DFSTestUtil.writeFile(fs, smallFilePath, data); + + // If we only have 1 block, NN won't enter safemode in the first place + // because the threshold is 0 blocks. + // So we need to add another 2 blocks. + int bigSize = blockSize * DATA_BLK_NUM * 2; + Path bigFilePath = new Path("/testStripedFile_" + bigSize); + data = StripedFileTestUtil.generateBytes(bigSize); + DFSTestUtil.writeFile(fs, bigFilePath, data); + // now we have 3 blocks. NN needs 2 blocks to reach the threshold 0.9 of + // total blocks 3. + + // stopping all DNs + List dnprops = Lists.newArrayList(); + LocatedBlocks lbs = cluster.getNameNodeRpc() + .getBlockLocations(smallFilePath.toString(), 0, smallSize); + DatanodeInfo[] locations = lbs.get(0).getLocations(); + for (DatanodeInfo loc : locations) { + // keep the DNs that have smallFile in the head of dnprops + dnprops.add(cluster.stopDataNode(loc.getName())); + } + for (int i = 0; i < numDNs - locations.length; i++) { + dnprops.add(cluster.stopDataNode(0)); + } + + cluster.restartNameNode(0); + NameNode nn = cluster.getNameNode(); + assertTrue(cluster.getNameNode().isInSafeMode()); + assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn)); + + // the block of smallFile doesn't reach minStorages, + // so the safe blocks count doesn't increment. + for (int i = 0; i < minStorages - 1; i++) { + cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); + cluster.triggerBlockReports(); + assertEquals(0, NameNodeAdapter.getSafeModeSafeBlocks(nn)); + } + + // the block of smallFile reaches minStorages, + // so the safe blocks count increment. + cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); + cluster.triggerBlockReports(); + assertEquals(1, NameNodeAdapter.getSafeModeSafeBlocks(nn)); + + // the 2 blocks of bigFile need DATA_BLK_NUM storages to be safe + for (int i = minStorages; i < DATA_BLK_NUM - 1; i++) { + cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); + cluster.triggerBlockReports(); + assertTrue(nn.isInSafeMode()); + } + + cluster.restartDataNode(dnprops.remove(0)); + cluster.waitActive(); + cluster.triggerBlockReports(); + assertFalse(nn.isInSafeMode()); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java new file mode 100644 index 0000000000000..3d51f0c725c7d --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteReadStripedFile.java @@ -0,0 +1,251 @@ +/** + * 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.commons.logging.impl.Log4JLogger; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.BlockLocation; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockPlacementPolicy; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.web.WebHdfsConstants; +import org.apache.hadoop.hdfs.web.WebHdfsTestUtil; +import org.apache.log4j.Level; +import org.junit.After; +import org.junit.Before; +import org.junit.Test; + +import java.io.IOException; +import java.nio.ByteBuffer; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.stripesPerBlock; + +public class TestWriteReadStripedFile { + public static final Log LOG = LogFactory.getLog(TestWriteReadStripedFile.class); + private static MiniDFSCluster cluster; + private static DistributedFileSystem fs; + private static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private static short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private static Configuration conf = new HdfsConfiguration(); + + static { + ((Log4JLogger)LogFactory.getLog(BlockPlacementPolicy.class)) + .getLogger().setLevel(Level.ALL); + } + + @Before + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); + fs = cluster.getFileSystem(); + } + + @After + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testFileEmpty() throws Exception { + testOneFileUsingDFSStripedInputStream("/EmptyFile", 0); + testOneFileUsingDFSStripedInputStream("/EmptyFile2", 0, true); + } + + @Test + public void testFileSmallerThanOneCell1() throws Exception { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", 1); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", 1, true); + } + + @Test + public void testFileSmallerThanOneCell2() throws Exception { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell", cellSize - 1); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneCell2", cellSize - 1, + true); + } + + @Test + public void testFileEqualsWithOneCell() throws Exception { + testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell", cellSize); + testOneFileUsingDFSStripedInputStream("/EqualsWithOneCell2", cellSize, true); + } + + @Test + public void testFileSmallerThanOneStripe1() throws Exception { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", + cellSize * dataBlocks - 1); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2", + cellSize * dataBlocks - 1, true); + } + + @Test + public void testFileSmallerThanOneStripe2() throws Exception { + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe", + cellSize + 123); + testOneFileUsingDFSStripedInputStream("/SmallerThanOneStripe2", + cellSize + 123, true); + } + + @Test + public void testFileEqualsWithOneStripe() throws Exception { + testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe", + cellSize * dataBlocks); + testOneFileUsingDFSStripedInputStream("/EqualsWithOneStripe2", + cellSize * dataBlocks, true); + } + + @Test + public void testFileMoreThanOneStripe1() throws Exception { + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe1", + cellSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe12", + cellSize * dataBlocks + 123, true); + } + + @Test + public void testFileMoreThanOneStripe2() throws Exception { + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe2", + cellSize * dataBlocks + cellSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanOneStripe22", + cellSize * dataBlocks + cellSize * dataBlocks + 123, true); + } + + @Test + public void testLessThanFullBlockGroup() throws Exception { + testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup", + cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize); + testOneFileUsingDFSStripedInputStream("/LessThanFullBlockGroup2", + cellSize * dataBlocks * (stripesPerBlock - 1) + cellSize, true); + } + + @Test + public void testFileFullBlockGroup() throws Exception { + testOneFileUsingDFSStripedInputStream("/FullBlockGroup", + blockSize * dataBlocks); + testOneFileUsingDFSStripedInputStream("/FullBlockGroup2", + blockSize * dataBlocks, true); + } + + @Test + public void testFileMoreThanABlockGroup1() throws Exception { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup1", + blockSize * dataBlocks + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup12", + blockSize * dataBlocks + 123, true); + } + + @Test + public void testFileMoreThanABlockGroup2() throws Exception { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup2", + blockSize * dataBlocks + cellSize + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup22", + blockSize * dataBlocks + cellSize + 123, true); + } + + + @Test + public void testFileMoreThanABlockGroup3() throws Exception { + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup3", + blockSize * dataBlocks * 3 + cellSize * dataBlocks + + cellSize + 123); + testOneFileUsingDFSStripedInputStream("/MoreThanABlockGroup32", + blockSize * dataBlocks * 3 + cellSize * dataBlocks + + cellSize + 123, true); + } + + private void testOneFileUsingDFSStripedInputStream(String src, int fileLength) + throws Exception { + testOneFileUsingDFSStripedInputStream(src, fileLength, false); + } + + private void testOneFileUsingDFSStripedInputStream(String src, int fileLength, + boolean withDataNodeFailure) throws Exception { + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + Path srcPath = new Path(src); + DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + StripedFileTestUtil.waitBlockGroupsReported(fs, src); + + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); + + if (withDataNodeFailure) { + int dnIndex = 1; // TODO: StripedFileTestUtil.random.nextInt(dataBlocks); + LOG.info("stop DataNode " + dnIndex); + stopDataNode(srcPath, dnIndex); + } + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + largeBuf); + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + ByteBuffer.allocate(fileLength + 100)); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + smallBuf); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + ByteBuffer.allocate(1024)); + } + + private void stopDataNode(Path path, int failedDNIdx) + throws IOException { + BlockLocation[] locs = fs.getFileBlockLocations(path, 0, cellSize); + if (locs != null && locs.length > 0) { + String name = (locs[0].getNames())[failedDNIdx]; + for (DataNode dn : cluster.getDataNodes()) { + int port = dn.getXferPort(); + if (name.contains(Integer.toString(port))) { + dn.shutdown(); + break; + } + } + } + } + + @Test + public void testWriteReadUsingWebHdfs() throws Exception { + int fileLength = blockSize * dataBlocks + cellSize + 123; + + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + FileSystem fs = WebHdfsTestUtil.getWebHdfsFileSystem(conf, + WebHdfsConstants.WEBHDFS_SCHEME); + Path srcPath = new Path("/testWriteReadUsingWebHdfs"); + DFSTestUtil.writeFile(fs, srcPath, new String(expected)); + + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + // TODO: HDFS-8797 + //StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, largeBuf); + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, smallBuf); + // webhdfs doesn't support bytebuffer read + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java new file mode 100644 index 0000000000000..764527d907429 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/TestWriteStripedFileWithFailure.java @@ -0,0 +1,171 @@ +/** + * 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.FSDataOutputStream; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.test.GenericTestUtils; +import org.apache.log4j.Level; +import org.junit.Assert; +import org.junit.Ignore; +import org.junit.Test; + +import java.io.IOException; +import java.util.concurrent.atomic.AtomicInteger; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.blockSize; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.numDNs; + +public class TestWriteStripedFileWithFailure { + public static final Log LOG = LogFactory + .getLog(TestWriteStripedFileWithFailure.class); + private static MiniDFSCluster cluster; + private static FileSystem fs; + private static Configuration conf = new HdfsConfiguration(); + + static { + GenericTestUtils.setLogLevel(DFSOutputStream.LOG, Level.ALL); + GenericTestUtils.setLogLevel(DataStreamer.LOG, Level.ALL); + } + + private final short dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final int smallFileLength = blockSize * dataBlocks - 123; + private final int largeFileLength = blockSize * dataBlocks + 123; + private final int[] fileLengths = {smallFileLength, largeFileLength}; + + public void setup() throws IOException { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); + fs = cluster.getFileSystem(); + } + + public void tearDown() throws IOException { + if (cluster != null) { + cluster.shutdown(); + } + } + + // Test writing file with some Datanodes failure + // TODO: enable this test after HDFS-8704 and HDFS-9040 + @Ignore + @Test(timeout = 300000) + public void testWriteStripedFileWithDNFailure() throws IOException { + for (int fileLength : fileLengths) { + for (int dataDelNum = 1; dataDelNum < 4; dataDelNum++) { + for (int parityDelNum = 0; (dataDelNum+parityDelNum) < 4; parityDelNum++) { + try { + // setup a new cluster with no dead datanode + setup(); + writeFileWithDNFailure(fileLength, dataDelNum, parityDelNum); + } catch (IOException ioe) { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + LOG.error("Failed to write file with DN failure:" + + " fileType = "+ fileType + + ", dataDelNum = " + dataDelNum + + ", parityDelNum = " + parityDelNum); + throw ioe; + } finally { + // tear down the cluster + tearDown(); + } + } + } + } + } + + /** + * Test writing a file with shutting down some DNs(data DNs or parity DNs or both). + * @param fileLength file length + * @param dataDNFailureNum the shutdown number of data DNs + * @param parityDNFailureNum the shutdown number of parity DNs + * @throws IOException + */ + private void writeFileWithDNFailure(int fileLength, + int dataDNFailureNum, int parityDNFailureNum) throws IOException { + String fileType = fileLength < (blockSize * dataBlocks) ? + "smallFile" : "largeFile"; + String src = "/dnFailure_" + dataDNFailureNum + "_" + parityDNFailureNum + + "_" + fileType; + LOG.info("writeFileWithDNFailure: file = " + src + + ", fileType = " + fileType + + ", dataDNFailureNum = " + dataDNFailureNum + + ", parityDNFailureNum = " + parityDNFailureNum); + + Path srcPath = new Path(src); + final AtomicInteger pos = new AtomicInteger(); + final FSDataOutputStream out = fs.create(srcPath); + final DFSStripedOutputStream stripedOut + = (DFSStripedOutputStream)out.getWrappedStream(); + + int[] dataDNFailureIndices = StripedFileTestUtil.randomArray(0, dataBlocks, + dataDNFailureNum); + Assert.assertNotNull(dataDNFailureIndices); + int[] parityDNFailureIndices = StripedFileTestUtil.randomArray(dataBlocks, + dataBlocks + parityBlocks, parityDNFailureNum); + Assert.assertNotNull(parityDNFailureIndices); + + int[] failedDataNodes = new int[dataDNFailureNum + parityDNFailureNum]; + System.arraycopy(dataDNFailureIndices, 0, failedDataNodes, + 0, dataDNFailureIndices.length); + System.arraycopy(parityDNFailureIndices, 0, failedDataNodes, + dataDNFailureIndices.length, parityDNFailureIndices.length); + + final int killPos = fileLength/2; + for (; pos.get() < fileLength; ) { + final int i = pos.getAndIncrement(); + if (i == killPos) { + for(int failedDn : failedDataNodes) { + StripedFileTestUtil.killDatanode(cluster, stripedOut, failedDn, pos); + } + } + write(out, i); + } + out.close(); + + // make sure the expected number of Datanode have been killed + int dnFailureNum = dataDNFailureNum + parityDNFailureNum; + Assert.assertEquals(cluster.getDataNodes().size(), numDNs - dnFailureNum); + + byte[] smallBuf = new byte[1024]; + byte[] largeBuf = new byte[fileLength + 100]; + final byte[] expected = StripedFileTestUtil.generateBytes(fileLength); + StripedFileTestUtil.verifyLength(fs, srcPath, fileLength); + StripedFileTestUtil.verifySeek(fs, srcPath, fileLength); + StripedFileTestUtil.verifyStatefulRead(fs, srcPath, fileLength, expected, + smallBuf); + StripedFileTestUtil.verifyPread(fs, srcPath, fileLength, expected, largeBuf); + + // delete the file + fs.delete(srcPath, true); + } + + void write(FSDataOutputStream out, int i) throws IOException { + try { + out.write(StripedFileTestUtil.getByte(i)); + } catch (IOException e) { + throw new IOException("Failed at i=" + i, e); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java index 9f8aef55dee83..e944b814d41ec 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/protocol/TestLayoutVersion.java @@ -126,7 +126,8 @@ public void testNameNodeFeatureMinimumCompatibleLayoutVersions() { EnumSet compatibleFeatures = EnumSet.of( NameNodeLayoutVersion.Feature.TRUNCATE, NameNodeLayoutVersion.Feature.APPEND_NEW_BLOCK, - NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE); + NameNodeLayoutVersion.Feature.QUOTA_BY_STORAGE_TYPE, + NameNodeLayoutVersion.Feature.ERASURE_CODING); for (LayoutFeature f : compatibleFeatures) { assertEquals(String.format("Expected minimum compatible layout version " + "%d for feature %s.", baseLV, f), baseLV, 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 2bfba98cf123c..4171d5c78c21a 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 @@ -24,6 +24,7 @@ import java.util.ArrayList; import java.util.Arrays; +import java.util.Iterator; import java.util.List; import org.apache.hadoop.fs.permission.AclEntry; @@ -33,6 +34,7 @@ import org.apache.hadoop.fs.permission.FsAction; import org.apache.hadoop.fs.StorageType; import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.DatanodeID; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -40,6 +42,7 @@ import org.apache.hadoop.hdfs.protocol.ExtendedBlock; import org.apache.hadoop.hdfs.protocol.LocatedBlock; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockCommandProto; +import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockECRecoveryCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.BlockRecoveryCommandProto; import org.apache.hadoop.hdfs.protocol.proto.DatanodeProtocolProtos.DatanodeRegistrationProto; import org.apache.hadoop.hdfs.protocol.proto.HdfsProtos; @@ -63,15 +66,21 @@ import org.apache.hadoop.hdfs.security.token.block.BlockKey; 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.BlockManagerTestUtil; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NamenodeRole; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.NodeType; import org.apache.hadoop.hdfs.server.common.StorageInfo; import org.apache.hadoop.hdfs.server.namenode.CheckpointSignature; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; import org.apache.hadoop.hdfs.server.protocol.BlockCommand; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand; import org.apache.hadoop.hdfs.server.protocol.BlockRecoveryCommand.RecoveringBlock; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations; import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.BlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlocksWithLocations.StripedBlockWithLocations; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand; import org.apache.hadoop.hdfs.server.protocol.DatanodeProtocol; import org.apache.hadoop.hdfs.server.protocol.DatanodeRegistration; import org.apache.hadoop.hdfs.server.protocol.DatanodeStorage; @@ -80,6 +89,7 @@ import org.apache.hadoop.hdfs.server.protocol.RemoteEditLog; import org.apache.hadoop.hdfs.server.protocol.RemoteEditLogManifest; import org.apache.hadoop.io.Text; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.apache.hadoop.security.proto.SecurityProtos.TokenProto; import org.apache.hadoop.security.token.Token; import org.apache.hadoop.util.DataChecksum; @@ -181,40 +191,59 @@ public void testConvertBlock() { assertEquals(b, b2); } - private static BlockWithLocations getBlockWithLocations(int bid) { + private static BlockWithLocations getBlockWithLocations( + int bid, boolean isStriped) { final String[] datanodeUuids = {"dn1", "dn2", "dn3"}; final String[] storageIDs = {"s1", "s2", "s3"}; final StorageType[] storageTypes = { StorageType.DISK, StorageType.DISK, StorageType.DISK}; - return new BlockWithLocations(new Block(bid, 0, 1), + final byte[] indices = {0, 1, 2}; + final short dataBlkNum = 6; + BlockWithLocations blkLocs = new BlockWithLocations(new Block(bid, 0, 1), datanodeUuids, storageIDs, storageTypes); + if (isStriped) { + blkLocs = new StripedBlockWithLocations(blkLocs, indices, dataBlkNum, + StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE); + } + return blkLocs; } private void compare(BlockWithLocations locs1, BlockWithLocations locs2) { assertEquals(locs1.getBlock(), locs2.getBlock()); assertTrue(Arrays.equals(locs1.getStorageIDs(), locs2.getStorageIDs())); + if (locs1 instanceof StripedBlockWithLocations) { + assertTrue(Arrays.equals(((StripedBlockWithLocations) locs1).getIndices(), + ((StripedBlockWithLocations) locs2).getIndices())); + } } @Test public void testConvertBlockWithLocations() { - BlockWithLocations locs = getBlockWithLocations(1); - BlockWithLocationsProto locsProto = PBHelper.convert(locs); - BlockWithLocations locs2 = PBHelper.convert(locsProto); - compare(locs, locs2); + boolean[] testSuite = new boolean[]{false, true}; + for (int i = 0; i < testSuite.length; i++) { + BlockWithLocations locs = getBlockWithLocations(1, testSuite[i]); + BlockWithLocationsProto locsProto = PBHelper.convert(locs); + BlockWithLocations locs2 = PBHelper.convert(locsProto); + compare(locs, locs2); + } } @Test public void testConvertBlocksWithLocations() { - BlockWithLocations[] list = new BlockWithLocations[] { - getBlockWithLocations(1), getBlockWithLocations(2) }; - BlocksWithLocations locs = new BlocksWithLocations(list); - BlocksWithLocationsProto locsProto = PBHelper.convert(locs); - BlocksWithLocations locs2 = PBHelper.convert(locsProto); - BlockWithLocations[] blocks = locs.getBlocks(); - BlockWithLocations[] blocks2 = locs2.getBlocks(); - assertEquals(blocks.length, blocks2.length); - for (int i = 0; i < blocks.length; i++) { - compare(blocks[i], blocks2[i]); + boolean[] testSuite = new boolean[]{false, true}; + for (int i = 0; i < testSuite.length; i++) { + BlockWithLocations[] list = new BlockWithLocations[]{ + getBlockWithLocations(1, testSuite[i]), + getBlockWithLocations(2, testSuite[i])}; + BlocksWithLocations locs = new BlocksWithLocations(list); + BlocksWithLocationsProto locsProto = PBHelper.convert(locs); + BlocksWithLocations locs2 = PBHelper.convert(locsProto); + BlockWithLocations[] blocks = locs.getBlocks(); + BlockWithLocations[] blocks2 = locs2.getBlocks(); + assertEquals(blocks.length, blocks2.length); + for (int j = 0; j < blocks.length; j++) { + compare(blocks[j], blocks2[j]); + } } } @@ -489,16 +518,16 @@ private LocatedBlock createLocatedBlockNoStorageMedia() { @Test public void testConvertLocatedBlock() { LocatedBlock lb = createLocatedBlock(); - LocatedBlockProto lbProto = PBHelperClient.convert(lb); - LocatedBlock lb2 = PBHelperClient.convert(lbProto); + LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb); + LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto); compare(lb,lb2); } @Test public void testConvertLocatedBlockNoStorageMedia() { LocatedBlock lb = createLocatedBlockNoStorageMedia(); - LocatedBlockProto lbProto = PBHelperClient.convert(lb); - LocatedBlock lb2 = PBHelperClient.convert(lbProto); + LocatedBlockProto lbProto = PBHelperClient.convertLocatedBlock(lb); + LocatedBlock lb2 = PBHelperClient.convertLocatedBlockProto(lbProto); compare(lb,lb2); } @@ -508,8 +537,8 @@ public void testConvertLocatedBlockList() { for (int i=0;i<3;i++) { lbl.add(createLocatedBlock()); } - List lbpl = PBHelperClient.convertLocatedBlock2(lbl); - List lbl2 = PBHelperClient.convertLocatedBlock(lbpl); + List lbpl = PBHelperClient.convertLocatedBlocks2(lbl); + List lbl2 = PBHelperClient.convertLocatedBlocks(lbpl); assertEquals(lbl.size(), lbl2.size()); for (int i=0;i blkRecoveryInfosList = new ArrayList(); + blkRecoveryInfosList.add(blkECRecoveryInfo0); + blkRecoveryInfosList.add(blkECRecoveryInfo1); + BlockECRecoveryCommand blkECRecoveryCmd = new BlockECRecoveryCommand( + DatanodeProtocol.DNA_ERASURE_CODING_RECOVERY, blkRecoveryInfosList); + BlockECRecoveryCommandProto blkECRecoveryCmdProto = PBHelper + .convert(blkECRecoveryCmd); + blkECRecoveryCmd = PBHelper.convert(blkECRecoveryCmdProto); + Iterator iterator = blkECRecoveryCmd.getECTasks() + .iterator(); + assertBlockECRecoveryInfoEquals(blkECRecoveryInfo0, iterator.next()); + assertBlockECRecoveryInfoEquals(blkECRecoveryInfo1, iterator.next()); + } + + private void assertBlockECRecoveryInfoEquals( + BlockECRecoveryInfo blkECRecoveryInfo1, + BlockECRecoveryInfo blkECRecoveryInfo2) { + assertEquals(blkECRecoveryInfo1.getExtendedBlock(), + blkECRecoveryInfo2.getExtendedBlock()); + + DatanodeInfo[] sourceDnInfos1 = blkECRecoveryInfo1.getSourceDnInfos(); + DatanodeInfo[] sourceDnInfos2 = blkECRecoveryInfo2.getSourceDnInfos(); + assertDnInfosEqual(sourceDnInfos1, sourceDnInfos2); + + DatanodeInfo[] targetDnInfos1 = blkECRecoveryInfo1.getTargetDnInfos(); + DatanodeInfo[] targetDnInfos2 = blkECRecoveryInfo2.getTargetDnInfos(); + assertDnInfosEqual(targetDnInfos1, targetDnInfos2); + + String[] targetStorageIDs1 = blkECRecoveryInfo1.getTargetStorageIDs(); + String[] targetStorageIDs2 = blkECRecoveryInfo2.getTargetStorageIDs(); + assertEquals(targetStorageIDs1.length, targetStorageIDs2.length); + for (int i = 0; i < targetStorageIDs1.length; i++) { + assertEquals(targetStorageIDs1[i], targetStorageIDs2[i]); + } + + short[] liveBlockIndices1 = blkECRecoveryInfo1.getLiveBlockIndices(); + short[] liveBlockIndices2 = blkECRecoveryInfo2.getLiveBlockIndices(); + for (int i = 0; i < liveBlockIndices1.length; i++) { + assertEquals(liveBlockIndices1[i], liveBlockIndices2[i]); + } + + ErasureCodingPolicy ecPolicy1 = blkECRecoveryInfo1.getErasureCodingPolicy(); + ErasureCodingPolicy ecPolicy2 = blkECRecoveryInfo2.getErasureCodingPolicy(); + // Compare ECPolicies same as default ECPolicy as we used system default + // ECPolicy used in this test + compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy1); + compareECPolicies(ErasureCodingPolicyManager.getSystemDefaultPolicy(), ecPolicy2); + } + + private void compareECPolicies(ErasureCodingPolicy ecPolicy1, ErasureCodingPolicy ecPolicy2) { + assertEquals(ecPolicy1.getName(), ecPolicy2.getName()); + assertEquals(ecPolicy1.getNumDataUnits(), ecPolicy2.getNumDataUnits()); + assertEquals(ecPolicy1.getNumParityUnits(), ecPolicy2.getNumParityUnits()); + } + + private void assertDnInfosEqual(DatanodeInfo[] dnInfos1, + DatanodeInfo[] dnInfos2) { + assertEquals(dnInfos1.length, dnInfos2.length); + for (int i = 0; i < dnInfos1.length; i++) { + compare(dnInfos1[i], dnInfos2[i]); + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java index fad9f7b0d193b..2d7cabaadd757 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/balancer/TestBalancer.java @@ -66,6 +66,7 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.NameNodeProxies; +import org.apache.hadoop.hdfs.StripedFileTestUtil; import org.apache.hadoop.hdfs.protocol.Block; import org.apache.hadoop.hdfs.protocol.ClientProtocol; import org.apache.hadoop.hdfs.protocol.DatanodeInfo; @@ -73,6 +74,7 @@ import org.apache.hadoop.hdfs.protocol.HdfsConstants; 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.balancer.Balancer.Cli; import org.apache.hadoop.hdfs.server.balancer.Balancer.Result; import org.apache.hadoop.hdfs.server.balancer.BalancerParameters; @@ -147,6 +149,23 @@ static void initConfWithRamDisk(Configuration conf, conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L); } + int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + int groupSize = dataBlocks + parityBlocks; + private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock; + + static void initConfWithStripe(Configuration conf) { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); + conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); + SimulatedFSDataset.setFactory(conf); + conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); + conf.setLong(DFSConfigKeys.DFS_BALANCER_MOVEDWINWIDTH_KEY, 2000L); + conf.setLong(DFSConfigKeys.DFS_BALANCER_GETBLOCKS_MIN_BLOCK_SIZE_KEY, 1L); + } + /* create a file with a length of fileLen */ static void createFile(MiniDFSCluster cluster, Path filePath, long fileLen, short replicationFactor, int nnIndex) @@ -935,9 +954,9 @@ public void testBalancer1() throws Exception { void testBalancer1Internal(Configuration conf) throws Exception { initConf(conf); testUnevenDistribution(conf, - new long[] {50*CAPACITY/100, 10*CAPACITY/100}, + new long[]{50 * CAPACITY / 100, 10 * CAPACITY / 100}, new long[]{CAPACITY, CAPACITY}, - new String[] {RACK0, RACK1}); + new String[]{RACK0, RACK1}); } @Test(expected=HadoopIllegalArgumentException.class) @@ -951,7 +970,7 @@ public void testBalancerWithZeroThreadsForMove() throws Exception { public void testBalancerWithNonZeroThreadsForMove() throws Exception { Configuration conf = new HdfsConfiguration(); conf.setInt(DFSConfigKeys.DFS_DATANODE_BALANCE_MAX_NUM_CONCURRENT_MOVES_KEY, 8); - testBalancer1Internal (conf); + testBalancer1Internal(conf); } @Test(timeout=100000) @@ -961,8 +980,8 @@ public void testBalancer2() throws Exception { void testBalancer2Internal(Configuration conf) throws Exception { initConf(conf); - testBalancerDefaultConstructor(conf, new long[] { CAPACITY, CAPACITY }, - new String[] { RACK0, RACK1 }, CAPACITY, RACK2); + testBalancerDefaultConstructor(conf, new long[]{CAPACITY, CAPACITY}, + new String[]{RACK0, RACK1}, CAPACITY, RACK2); } private void testBalancerDefaultConstructor(Configuration conf, @@ -1679,7 +1698,74 @@ public void testMinBlockSizeAndSourceNodes() throws Exception { cluster.shutdown(); } } - + public void integrationTestWithStripedFile(Configuration conf) throws Exception { + initConfWithStripe(conf); + doTestBalancerWithStripedFile(conf); + } + + @Test(timeout = 100000) + public void testBalancerWithStripedFile() throws Exception { + Configuration conf = new Configuration(); + initConfWithStripe(conf); + doTestBalancerWithStripedFile(conf); + } + + private void doTestBalancerWithStripedFile(Configuration conf) throws Exception { + int numOfDatanodes = dataBlocks + parityBlocks + 2; + int numOfRacks = dataBlocks; + long capacity = 20 * DEFAULT_STRIPE_BLOCK_SIZE; + long[] capacities = new long[numOfDatanodes]; + for (int i = 0; i < capacities.length; i++) { + capacities[i] = capacity; + } + String[] racks = new String[numOfDatanodes]; + for (int i = 0; i < numOfDatanodes; i++) { + racks[i] = "/rack" + (i % numOfRacks); + } + cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(numOfDatanodes) + .racks(racks) + .simulatedCapacities(capacities) + .build(); + + try { + cluster.waitActive(); + client = NameNodeProxies.createProxy(conf, cluster.getFileSystem(0).getUri(), + ClientProtocol.class).getProxy(); + client.setErasureCodingPolicy("/", null); + + long totalCapacity = sum(capacities); + + // fill up the cluster with 30% data. It'll be 45% full plus parity. + long fileLen = totalCapacity * 3 / 10; + long totalUsedSpace = fileLen * (dataBlocks + parityBlocks) / dataBlocks; + FileSystem fs = cluster.getFileSystem(0); + DFSTestUtil.createFile(fs, filePath, fileLen, (short) 3, r.nextLong()); + + // verify locations of striped blocks + LocatedBlocks locatedBlocks = client.getBlockLocations(fileName, 0, fileLen); + StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); + + // add one datanode + String newRack = "/rack" + (++numOfRacks); + cluster.startDataNodes(conf, 1, true, null, + new String[]{newRack}, null, new long[]{capacity}); + totalCapacity += capacity; + cluster.triggerHeartbeats(); + + // run balancer and validate results + BalancerParameters p = BalancerParameters.DEFAULT; + Collection namenodes = DFSUtil.getNsServiceRpcUris(conf); + runBalancer(conf, totalUsedSpace, totalCapacity, p, 0); + + // verify locations of striped blocks + locatedBlocks = client.getBlockLocations(fileName, 0, fileLen); + StripedFileTestUtil.verifyLocatedStripedBlocks(locatedBlocks, groupSize); + } finally { + cluster.shutdown(); + } + } + /** * @param args */ 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 148135bae9727..64d80bdbe269c 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 static int computeInvalidationWork(BlockManager bm) { */ public static int computeAllPendingWork(BlockManager bm) { int work = computeInvalidationWork(bm); - work += bm.computeReplicationWork(Integer.MAX_VALUE); + work += bm.computeBlockRecoveryWork(Integer.MAX_VALUE); return work; } @@ -306,4 +306,12 @@ public static void recheckDecommissionState(DatanodeManager dm) throws ExecutionException, InterruptedException { dm.getDecomManager().runMonitor(); } + + /** + * add block to the replicateBlocks queue of the Datanode + */ + public static void addBlockToBeReplicated(DatanodeDescriptor node, + Block block, DatanodeStorageInfo[] targets) { + node.addBlockToBeReplicated(block, targets); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java index ceef9f2964598..d6213ff23914b 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfo.java @@ -64,24 +64,12 @@ public void testAddStorage() throws Exception { final DatanodeStorageInfo storage = DFSTestUtil.createDatanodeStorageInfo("storageID", "127.0.0.1"); - boolean added = blockInfo.addStorage(storage); + boolean added = blockInfo.addStorage(storage, blockInfo); Assert.assertTrue(added); Assert.assertEquals(storage, blockInfo.getStorageInfo(0)); } - @Test - public void testCopyConstructor() { - BlockInfoContiguous old = new BlockInfoContiguous((short) 3); - try { - BlockInfoContiguous copy = new BlockInfoContiguous(old); - assertEquals(old.getBlockCollectionId(), copy.getBlockCollectionId()); - assertEquals(old.getCapacity(), copy.getCapacity()); - } catch (Exception e) { - Assert.fail("Copy constructor throws exception: " + e); - } - } - @Test public void testReplaceStorage() throws Exception { 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 new file mode 100644 index 0000000000000..4650dce6ed483 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockInfoStriped.java @@ -0,0 +1,220 @@ +/** + * 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.DFSTestUtil; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeStorageInfo.AddBlockResult; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.junit.Assert; +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.StripedFileTestUtil.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_BLOCKS; +import static org.junit.Assert.assertArrayEquals; +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.fail; + +/** + * Test {@link BlockInfoStriped} + */ +public class TestBlockInfoStriped { + private static final int TOTAL_NUM_BLOCKS = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + private static final long BASE_ID = -1600; + private static final Block baseBlock = new Block(BASE_ID); + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + private final BlockInfoStriped info = new BlockInfoStriped(baseBlock, + testECPolicy); + + private Block[] createReportedBlocks(int num) { + Block[] blocks = new Block[num]; + for (int i = 0; i < num; i++) { + blocks[i] = new Block(BASE_ID + i); + } + return blocks; + } + + /** + * Test adding storage and reported block + */ + @Test + public void testAddStorage() { + // first add NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS storages, i.e., a complete + // group of blocks/storages + DatanodeStorageInfo[] storageInfos = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS); + Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS); + int i = 0; + for (; i < storageInfos.length; i += 2) { + info.addStorage(storageInfos[i], blocks[i]); + Assert.assertEquals(i/2 + 1, info.numNodes()); + } + i /= 2; + for (int j = 1; j < storageInfos.length; j += 2) { + Assert.assertTrue(info.addStorage(storageInfos[j], blocks[j])); + Assert.assertEquals(i + (j+1)/2, info.numNodes()); + } + + // check + byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity()); + Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length); + i = 0; + for (DatanodeStorageInfo storage : storageInfos) { + int index = info.findStorageInfo(storage); + Assert.assertEquals(i++, index); + Assert.assertEquals(index, indices[index]); + } + + // the same block is reported from the same storage twice + i = 0; + for (DatanodeStorageInfo storage : storageInfos) { + Assert.assertTrue(info.addStorage(storage, blocks[i++])); + } + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity()); + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.numNodes()); + Assert.assertEquals(TOTAL_NUM_BLOCKS, indices.length); + i = 0; + for (DatanodeStorageInfo storage : storageInfos) { + int index = info.findStorageInfo(storage); + Assert.assertEquals(i++, index); + Assert.assertEquals(index, indices[index]); + } + + // the same block is reported from another storage + DatanodeStorageInfo[] storageInfos2 = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS * 2); + // only add the second half of info2 + for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) { + info.addStorage(storageInfos2[i], blocks[i % TOTAL_NUM_BLOCKS]); + Assert.assertEquals(i + 1, info.getCapacity()); + Assert.assertEquals(i + 1, info.numNodes()); + indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(i + 1, indices.length); + } + for (i = TOTAL_NUM_BLOCKS; i < storageInfos2.length; i++) { + int index = info.findStorageInfo(storageInfos2[i]); + Assert.assertEquals(i++, index); + Assert.assertEquals(index - TOTAL_NUM_BLOCKS, indices[index]); + } + } + + @Test + public void testRemoveStorage() { + // first add TOTAL_NUM_BLOCKS into the BlockInfoStriped + DatanodeStorageInfo[] storages = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS); + Block[] blocks = createReportedBlocks(TOTAL_NUM_BLOCKS); + for (int i = 0; i < storages.length; i++) { + info.addStorage(storages[i], blocks[i]); + } + + // remove two storages + info.removeStorage(storages[0]); + info.removeStorage(storages[2]); + + // check + Assert.assertEquals(TOTAL_NUM_BLOCKS, info.getCapacity()); + Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes()); + byte[] indices = (byte[]) Whitebox.getInternalState(info, "indices"); + for (int i = 0; i < storages.length; i++) { + int index = info.findStorageInfo(storages[i]); + if (i != 0 && i != 2) { + Assert.assertEquals(i, index); + Assert.assertEquals(index, indices[index]); + } else { + Assert.assertEquals(-1, index); + Assert.assertEquals(-1, indices[i]); + } + } + + // the same block is reported from another storage + DatanodeStorageInfo[] storages2 = DFSTestUtil.createDatanodeStorageInfos( + TOTAL_NUM_BLOCKS * 2); + for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) { + info.addStorage(storages2[i], blocks[i % TOTAL_NUM_BLOCKS]); + } + // now we should have 8 storages + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.numNodes()); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity()); + indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length); + int j = TOTAL_NUM_BLOCKS; + for (int i = TOTAL_NUM_BLOCKS; i < storages2.length; i++) { + int index = info.findStorageInfo(storages2[i]); + if (i == TOTAL_NUM_BLOCKS || i == TOTAL_NUM_BLOCKS + 2) { + Assert.assertEquals(i - TOTAL_NUM_BLOCKS, index); + } else { + Assert.assertEquals(j++, index); + } + } + + // remove the storages from storages2 + for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) { + info.removeStorage(storages2[i + TOTAL_NUM_BLOCKS]); + } + // now we should have 3 storages + Assert.assertEquals(TOTAL_NUM_BLOCKS - 2, info.numNodes()); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, info.getCapacity()); + indices = (byte[]) Whitebox.getInternalState(info, "indices"); + Assert.assertEquals(TOTAL_NUM_BLOCKS * 2 - 2, indices.length); + for (int i = 0; i < TOTAL_NUM_BLOCKS; i++) { + if (i == 0 || i == 2) { + int index = info.findStorageInfo(storages2[i + TOTAL_NUM_BLOCKS]); + Assert.assertEquals(-1, index); + } else { + int index = info.findStorageInfo(storages[i]); + Assert.assertEquals(i, index); + } + } + for (int i = TOTAL_NUM_BLOCKS; i < TOTAL_NUM_BLOCKS * 2 - 2; i++) { + Assert.assertEquals(-1, indices[i]); + Assert.assertNull(info.getDatanode(i)); + } + } + + @Test + public void testWrite() { + long blkID = 1; + long numBytes = 1; + long generationStamp = 1; + ByteBuffer byteBuffer = ByteBuffer.allocate(Long.SIZE / Byte.SIZE * 3); + byteBuffer.putLong(blkID).putLong(numBytes).putLong(generationStamp); + + ByteArrayOutputStream byteStream = new ByteArrayOutputStream(); + DataOutput out = new DataOutputStream(byteStream); + BlockInfoStriped blk = new BlockInfoStriped(new Block(blkID, numBytes, + generationStamp), testECPolicy); + + try { + blk.write(out); + } catch(Exception ex) { + fail("testWrite error:" + ex.getMessage()); + } + assertEquals(byteBuffer.array().length, byteStream.toByteArray().length); + assertArrayEquals(byteBuffer.array(), byteStream.toByteArray()); + } +} 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 d40b1c7e2b86e..094794b36d140 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 @@ -388,7 +388,7 @@ private void fulfillPipeline(BlockInfo blockInfo, for (int i = 1; i < pipeline.length; i++) { DatanodeStorageInfo storage = pipeline[i]; bm.addBlock(storage, blockInfo, null); - blockInfo.addStorage(storage); + blockInfo.addStorage(storage, blockInfo); } } @@ -398,7 +398,7 @@ private BlockInfo blockOnNodes(long blkId, List nodes) { for (DatanodeDescriptor dn : nodes) { for (DatanodeStorageInfo storage : dn.getStorageInfos()) { - blockInfo.addStorage(storage); + blockInfo.addStorage(storage, blockInfo); } } return blockInfo; @@ -462,8 +462,8 @@ private DatanodeStorageInfo[] scheduleSingleReplication(BlockInfo block) { 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); @@ -517,35 +517,38 @@ public void testHighestPriReplSrcChosenDespiteMaxReplLimit() throws Exception { 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)); - - 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, + new ArrayList(), + UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)[0]); + + 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(), - UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED)); + new ArrayList(), + 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.", - bm.chooseSourceDatanode( - aBlock, + 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(), - UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY)); + new ArrayList(), + UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY).length); } @Test @@ -566,30 +569,28 @@ public void testFavorDecomUntilHardLimit() throws Exception { 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(), - UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)); + new NumberReplicas(), new LinkedList(), + 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.", - bm.chooseSourceDatanode( - aBlock, + 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(), - UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED)); + new NumberReplicas(), new LinkedList(), + UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED).length); } - - @Test public void testSafeModeIBR() throws Exception { DatanodeDescriptor node = spy(nodes.get(0)); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java index 50d548a8b4e2a..ce7aee30e0fd8 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFS.java @@ -24,7 +24,6 @@ import java.io.IOException; import java.net.InetSocketAddress; import java.net.Socket; -import java.util.EnumSet; import java.util.List; import java.util.Random; @@ -69,28 +68,32 @@ public class TestBlockTokenWithDFS { - private static final int BLOCK_SIZE = 1024; - private static final int FILE_SIZE = 2 * BLOCK_SIZE; + protected static int BLOCK_SIZE = 1024; + protected static int FILE_SIZE = 2 * BLOCK_SIZE; private static final String FILE_TO_READ = "/fileToRead.dat"; private static final String FILE_TO_WRITE = "/fileToWrite.dat"; private static final String FILE_TO_APPEND = "/fileToAppend.dat"; - private final byte[] rawData = new byte[FILE_SIZE]; { GenericTestUtils.setLogLevel(DFSClient.LOG, Level.ALL); + } + + public static byte[] generateBytes(int fileSize){ Random r = new Random(); + byte[] rawData = new byte[fileSize]; r.nextBytes(rawData); + return rawData; } - private void createFile(FileSystem fs, Path filename) throws IOException { + private void createFile(FileSystem fs, Path filename, byte[] expected) throws IOException { FSDataOutputStream out = fs.create(filename); - out.write(rawData); + out.write(expected); out.close(); } // read a file using blockSeekTo() - private boolean checkFile1(FSDataInputStream in) { - byte[] toRead = new byte[FILE_SIZE]; + private boolean checkFile1(FSDataInputStream in, byte[] expected) { + byte[] toRead = new byte[expected.length]; int totalRead = 0; int nRead = 0; try { @@ -101,27 +104,27 @@ private boolean checkFile1(FSDataInputStream in) { return false; } assertEquals("Cannot read file.", toRead.length, totalRead); - return checkFile(toRead); + return checkFile(toRead, expected); } // read a file using fetchBlockByteRange() - private boolean checkFile2(FSDataInputStream in) { - byte[] toRead = new byte[FILE_SIZE]; + private boolean checkFile2(FSDataInputStream in, byte[] expected) { + byte[] toRead = new byte[expected.length]; try { assertEquals("Cannot read file", toRead.length, in.read(0, toRead, 0, toRead.length)); } catch (IOException e) { return false; } - return checkFile(toRead); + return checkFile(toRead, expected); } - private boolean checkFile(byte[] fileToCheck) { - if (fileToCheck.length != rawData.length) { + private boolean checkFile(byte[] fileToCheck, byte[] expected) { + if (fileToCheck.length != expected.length) { return false; } for (int i = 0; i < fileToCheck.length; i++) { - if (fileToCheck[i] != rawData[i]) { + if (fileToCheck[i] != expected[i]) { return false; } } @@ -137,7 +140,7 @@ private static FSDataOutputStream writeFile(FileSystem fileSys, Path name, } // try reading a block using a BlockReader directly - private static void tryRead(final Configuration conf, LocatedBlock lblock, + protected void tryRead(final Configuration conf, LocatedBlock lblock, boolean shouldSucceed) { InetSocketAddress targetAddr = null; IOException ioe = null; @@ -148,7 +151,7 @@ private static void tryRead(final Configuration conf, LocatedBlock lblock, targetAddr = NetUtils.createSocketAddr(nodes[0].getXferAddr()); blockReader = new BlockReaderFactory(new DfsClientConf(conf)). - setFileName(BlockReaderFactory.getFileName(targetAddr, + setFileName(BlockReaderFactory.getFileName(targetAddr, "test-blockpoolid", block.getBlockId())). setBlock(block). setBlockToken(lblock.getBlockToken()). @@ -205,7 +208,7 @@ public Peer newConnectedPeer(InetSocketAddress addr, } // get a conf for testing - private static Configuration getConf(int numDataNodes) { + protected Configuration getConf(int numDataNodes) { Configuration conf = new Configuration(); conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); @@ -241,16 +244,16 @@ public void testAppend() throws Exception { SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); Path fileToAppend = new Path(FILE_TO_APPEND); FileSystem fs = cluster.getFileSystem(); - + byte[] expected = generateBytes(FILE_SIZE); // write a one-byte file FSDataOutputStream stm = writeFile(fs, fileToAppend, (short) numDataNodes, BLOCK_SIZE); - stm.write(rawData, 0, 1); + stm.write(expected, 0, 1); stm.close(); // open the file again for append stm = fs.append(fileToAppend); - int mid = rawData.length - 1; - stm.write(rawData, 1, mid - 1); + int mid = expected.length - 1; + stm.write(expected, 1, mid - 1); stm.hflush(); /* @@ -267,11 +270,11 @@ public void testAppend() throws Exception { // remove a datanode to force re-establishing pipeline cluster.stopDataNode(0); // append the rest of the file - stm.write(rawData, mid, rawData.length - mid); + stm.write(expected, mid, expected.length - mid); stm.close(); // check if append is successful FSDataInputStream in5 = fs.open(fileToAppend); - assertTrue(checkFile1(in5)); + assertTrue(checkFile1(in5, expected)); } finally { if (cluster != null) { cluster.shutdown(); @@ -303,11 +306,12 @@ public void testWrite() throws Exception { Path fileToWrite = new Path(FILE_TO_WRITE); FileSystem fs = cluster.getFileSystem(); + byte[] expected = generateBytes(FILE_SIZE); FSDataOutputStream stm = writeFile(fs, fileToWrite, (short) numDataNodes, BLOCK_SIZE); // write a partial block - int mid = rawData.length - 1; - stm.write(rawData, 0, mid); + int mid = expected.length - 1; + stm.write(expected, 0, mid); stm.hflush(); /* @@ -324,11 +328,11 @@ public void testWrite() throws Exception { // remove a datanode to force re-establishing pipeline cluster.stopDataNode(0); // write the rest of the file - stm.write(rawData, mid, rawData.length - mid); + stm.write(expected, mid, expected.length - mid); stm.close(); // check if write is successful FSDataInputStream in4 = fs.open(fileToWrite); - assertTrue(checkFile1(in4)); + assertTrue(checkFile1(in4, expected)); } finally { if (cluster != null) { cluster.shutdown(); @@ -346,125 +350,137 @@ public void testRead() throws Exception { cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDataNodes).build(); cluster.waitActive(); assertEquals(numDataNodes, cluster.getDataNodes().size()); + doTestRead(conf, cluster, false); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } - final NameNode nn = cluster.getNameNode(); - final NamenodeProtocols nnProto = nn.getRpcServer(); - final BlockManager bm = nn.getNamesystem().getBlockManager(); - final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); + protected void doTestRead(Configuration conf, MiniDFSCluster cluster, + boolean isStriped) throws Exception { + final int numDataNodes = cluster.getDataNodes().size(); + final NameNode nn = cluster.getNameNode(); + final NamenodeProtocols nnProto = nn.getRpcServer(); + final BlockManager bm = nn.getNamesystem().getBlockManager(); + final BlockTokenSecretManager sm = bm.getBlockTokenSecretManager(); - // set a short token lifetime (1 second) initially - SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); + // set a short token lifetime (1 second) initially + SecurityTestUtil.setBlockTokenLifetime(sm, 1000L); - Path fileToRead = new Path(FILE_TO_READ); - FileSystem fs = cluster.getFileSystem(); - createFile(fs, fileToRead); + Path fileToRead = new Path(FILE_TO_READ); + FileSystem fs = cluster.getFileSystem(); + byte[] expected = generateBytes(FILE_SIZE); + createFile(fs, fileToRead, expected); /* * setup for testing expiration handling of cached tokens */ - // read using blockSeekTo(). Acquired tokens are cached in in1 - FSDataInputStream in1 = fs.open(fileToRead); - assertTrue(checkFile1(in1)); - // read using blockSeekTo(). Acquired tokens are cached in in2 - FSDataInputStream in2 = fs.open(fileToRead); - assertTrue(checkFile1(in2)); - // read using fetchBlockByteRange(). Acquired tokens are cached in in3 - FSDataInputStream in3 = fs.open(fileToRead); - assertTrue(checkFile2(in3)); + // read using blockSeekTo(). Acquired tokens are cached in in1 + FSDataInputStream in1 = fs.open(fileToRead); + assertTrue(checkFile1(in1,expected)); + // read using blockSeekTo(). Acquired tokens are cached in in2 + FSDataInputStream in2 = fs.open(fileToRead); + assertTrue(checkFile1(in2,expected)); + // read using fetchBlockByteRange(). Acquired tokens are cached in in3 + FSDataInputStream in3 = fs.open(fileToRead); + assertTrue(checkFile2(in3,expected)); /* * testing READ interface on DN using a BlockReader */ - DFSClient client = null; - try { - client = new DFSClient(new InetSocketAddress("localhost", + DFSClient client = null; + try { + client = new DFSClient(new InetSocketAddress("localhost", cluster.getNameNodePort()), conf); - } finally { - if (client != null) client.close(); - } - List locatedBlocks = nnProto.getBlockLocations( - FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks(); - LocatedBlock lblock = locatedBlocks.get(0); // first block - Token myToken = lblock.getBlockToken(); - // verify token is not expired - assertFalse(SecurityTestUtil.isBlockTokenExpired(myToken)); - // read with valid token, should succeed - tryRead(conf, lblock, true); + } finally { + if (client != null) client.close(); + } + List locatedBlocks = nnProto.getBlockLocations( + FILE_TO_READ, 0, FILE_SIZE).getLocatedBlocks(); + LocatedBlock lblock = locatedBlocks.get(0); // first block + // verify token is not expired + assertFalse(isBlockTokenExpired(lblock)); + // read with valid token, should succeed + tryRead(conf, lblock, true); /* * wait till myToken and all cached tokens in in1, in2 and in3 expire */ - while (!SecurityTestUtil.isBlockTokenExpired(myToken)) { - try { - Thread.sleep(10); - } catch (InterruptedException ignored) { - } + while (!isBlockTokenExpired(lblock)) { + try { + Thread.sleep(10); + } catch (InterruptedException ignored) { } + } /* * continue testing READ interface on DN using a BlockReader */ - // verify token is expired - assertTrue(SecurityTestUtil.isBlockTokenExpired(myToken)); - // read should fail - tryRead(conf, lblock, false); - // use a valid new token - lblock.setBlockToken(sm.generateToken(lblock.getBlock(), - EnumSet.of(BlockTokenIdentifier.AccessMode.READ))); - // read should succeed - tryRead(conf, lblock, true); - // use a token with wrong blockID - ExtendedBlock wrongBlock = new ExtendedBlock(lblock.getBlock() - .getBlockPoolId(), lblock.getBlock().getBlockId() + 1); - lblock.setBlockToken(sm.generateToken(wrongBlock, - EnumSet.of(BlockTokenIdentifier.AccessMode.READ))); - // read should fail - tryRead(conf, lblock, false); - // use a token with wrong access modes - lblock.setBlockToken(sm.generateToken(lblock.getBlock(), - EnumSet.of(BlockTokenIdentifier.AccessMode.WRITE, - BlockTokenIdentifier.AccessMode.COPY, - BlockTokenIdentifier.AccessMode.REPLACE))); - // read should fail - tryRead(conf, lblock, false); - - // set a long token lifetime for future tokens - SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L); + // verify token is expired + assertTrue(isBlockTokenExpired(lblock)); + // read should fail + tryRead(conf, lblock, false); + // use a valid new token + bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ); + // read should succeed + tryRead(conf, lblock, true); + // use a token with wrong blockID + long rightId = lblock.getBlock().getBlockId(); + long wrongId = rightId + 1; + lblock.getBlock().setBlockId(wrongId); + bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.READ); + lblock.getBlock().setBlockId(rightId); + // read should fail + tryRead(conf, lblock, false); + // use a token with wrong access modes + bm.setBlockToken(lblock, BlockTokenIdentifier.AccessMode.WRITE); + // read should fail + tryRead(conf, lblock, false); + + // set a long token lifetime for future tokens + SecurityTestUtil.setBlockTokenLifetime(sm, 600 * 1000L); /* * testing that when cached tokens are expired, DFSClient will re-fetch * tokens transparently for READ. */ - // confirm all tokens cached in in1 are expired by now - List lblocks = DFSTestUtil.getAllBlocks(in1); - for (LocatedBlock blk : lblocks) { - assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() is able to re-fetch token transparently - in1.seek(0); - assertTrue(checkFile1(in1)); - - // confirm all tokens cached in in2 are expired by now - List lblocks2 = DFSTestUtil.getAllBlocks(in2); - for (LocatedBlock blk : lblocks2) { - assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() is able to re-fetch token transparently (testing - // via another interface method) + // confirm all tokens cached in in1 are expired by now + List lblocks = DFSTestUtil.getAllBlocks(in1); + for (LocatedBlock blk : lblocks) { + assertTrue(isBlockTokenExpired(blk)); + } + // verify blockSeekTo() is able to re-fetch token transparently + in1.seek(0); + assertTrue(checkFile1(in1, expected)); + + // confirm all tokens cached in in2 are expired by now + List lblocks2 = DFSTestUtil.getAllBlocks(in2); + for (LocatedBlock blk : lblocks2) { + assertTrue(isBlockTokenExpired(blk)); + } + // verify blockSeekTo() is able to re-fetch token transparently (testing + // via another interface method) + if (isStriped) { + // striped block doesn't support seekToNewSource + in2.seek(0); + } else { assertTrue(in2.seekToNewSource(0)); - assertTrue(checkFile1(in2)); + } + assertTrue(checkFile1(in2,expected)); - // confirm all tokens cached in in3 are expired by now - List lblocks3 = DFSTestUtil.getAllBlocks(in3); - for (LocatedBlock blk : lblocks3) { - assertTrue(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify fetchBlockByteRange() is able to re-fetch token transparently - assertTrue(checkFile2(in3)); + // confirm all tokens cached in in3 are expired by now + List lblocks3 = DFSTestUtil.getAllBlocks(in3); + for (LocatedBlock blk : lblocks3) { + assertTrue(isBlockTokenExpired(blk)); + } + // verify fetchBlockByteRange() is able to re-fetch token transparently + assertTrue(checkFile2(in3,expected)); /* * testing that after datanodes are restarted on the same ports, cached @@ -473,37 +489,42 @@ public void testRead() throws Exception { * new tokens can be fetched from namenode). */ - // restart datanodes on the same ports that they currently use - assertTrue(cluster.restartDataNodes(true)); - cluster.waitActive(); - assertEquals(numDataNodes, cluster.getDataNodes().size()); - cluster.shutdownNameNode(0); + // restart datanodes on the same ports that they currently use + assertTrue(cluster.restartDataNodes(true)); + cluster.waitActive(); + assertEquals(numDataNodes, cluster.getDataNodes().size()); + cluster.shutdownNameNode(0); - // confirm tokens cached in in1 are still valid - lblocks = DFSTestUtil.getAllBlocks(in1); - for (LocatedBlock blk : lblocks) { - assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() still works (forced to use cached tokens) - in1.seek(0); - assertTrue(checkFile1(in1)); - - // confirm tokens cached in in2 are still valid - lblocks2 = DFSTestUtil.getAllBlocks(in2); - for (LocatedBlock blk : lblocks2) { - assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify blockSeekTo() still works (forced to use cached tokens) + // confirm tokens cached in in1 are still valid + lblocks = DFSTestUtil.getAllBlocks(in1); + for (LocatedBlock blk : lblocks) { + assertFalse(isBlockTokenExpired(blk)); + } + // verify blockSeekTo() still works (forced to use cached tokens) + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + + // confirm tokens cached in in2 are still valid + lblocks2 = DFSTestUtil.getAllBlocks(in2); + for (LocatedBlock blk : lblocks2) { + assertFalse(isBlockTokenExpired(blk)); + } + + // verify blockSeekTo() still works (forced to use cached tokens) + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); + } + assertTrue(checkFile1(in2,expected)); - // confirm tokens cached in in3 are still valid - lblocks3 = DFSTestUtil.getAllBlocks(in3); - for (LocatedBlock blk : lblocks3) { - assertFalse(SecurityTestUtil.isBlockTokenExpired(blk.getBlockToken())); - } - // verify fetchBlockByteRange() still works (forced to use cached tokens) - assertTrue(checkFile2(in3)); + // confirm tokens cached in in3 are still valid + lblocks3 = DFSTestUtil.getAllBlocks(in3); + for (LocatedBlock blk : lblocks3) { + assertFalse(isBlockTokenExpired(blk)); + } + // verify fetchBlockByteRange() still works (forced to use cached tokens) + assertTrue(checkFile2(in3,expected)); /* * testing that when namenode is restarted, cached tokens should still @@ -512,18 +533,23 @@ public void testRead() throws Exception { * setup for this test depends on the previous test. */ - // restart the namenode and then shut it down for test - cluster.restartNameNode(0); - cluster.shutdownNameNode(0); + // restart the namenode and then shut it down for test + cluster.restartNameNode(0); + cluster.shutdownNameNode(0); - // verify blockSeekTo() still works (forced to use cached tokens) - in1.seek(0); - assertTrue(checkFile1(in1)); - // verify again blockSeekTo() still works (forced to use cached tokens) + // verify blockSeekTo() still works (forced to use cached tokens) + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + // verify again blockSeekTo() still works (forced to use cached tokens) + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); - // verify fetchBlockByteRange() still works (forced to use cached tokens) - assertTrue(checkFile2(in3)); + } + assertTrue(checkFile1(in2,expected)); + + // verify fetchBlockByteRange() still works (forced to use cached tokens) + assertTrue(checkFile2(in3,expected)); /* * testing that after both namenode and datanodes got restarted (namenode @@ -532,58 +558,60 @@ public void testRead() throws Exception { * setup of this test depends on the previous test. */ - // restore the cluster and restart the datanodes for test - cluster.restartNameNode(0); - assertTrue(cluster.restartDataNodes(true)); - cluster.waitActive(); - assertEquals(numDataNodes, cluster.getDataNodes().size()); - - // shutdown namenode so that DFSClient can't get new tokens from namenode - cluster.shutdownNameNode(0); - - // verify blockSeekTo() fails (cached tokens become invalid) - in1.seek(0); - assertFalse(checkFile1(in1)); - // verify fetchBlockByteRange() fails (cached tokens become invalid) - assertFalse(checkFile2(in3)); - - // restart the namenode to allow DFSClient to re-fetch tokens - cluster.restartNameNode(0); - // verify blockSeekTo() works again (by transparently re-fetching - // tokens from namenode) - in1.seek(0); - assertTrue(checkFile1(in1)); + // restore the cluster and restart the datanodes for test + cluster.restartNameNode(0); + assertTrue(cluster.restartDataNodes(true)); + cluster.waitActive(); + assertEquals(numDataNodes, cluster.getDataNodes().size()); + + // shutdown namenode so that DFSClient can't get new tokens from namenode + cluster.shutdownNameNode(0); + + // verify blockSeekTo() fails (cached tokens become invalid) + in1.seek(0); + assertFalse(checkFile1(in1,expected)); + // verify fetchBlockByteRange() fails (cached tokens become invalid) + assertFalse(checkFile2(in3,expected)); + + // restart the namenode to allow DFSClient to re-fetch tokens + cluster.restartNameNode(0); + // verify blockSeekTo() works again (by transparently re-fetching + // tokens from namenode) + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); - // verify fetchBlockByteRange() works again (by transparently - // re-fetching tokens from namenode) - assertTrue(checkFile2(in3)); + } + assertTrue(checkFile1(in2,expected)); + // verify fetchBlockByteRange() works again (by transparently + // re-fetching tokens from namenode) + assertTrue(checkFile2(in3,expected)); /* * testing that when datanodes are restarted on different ports, DFSClient * is able to re-fetch tokens transparently to connect to them */ - // restart datanodes on newly assigned ports - assertTrue(cluster.restartDataNodes(false)); - cluster.waitActive(); - assertEquals(numDataNodes, cluster.getDataNodes().size()); - // verify blockSeekTo() is able to re-fetch token transparently - in1.seek(0); - assertTrue(checkFile1(in1)); - // verify blockSeekTo() is able to re-fetch token transparently + // restart datanodes on newly assigned ports + assertTrue(cluster.restartDataNodes(false)); + cluster.waitActive(); + assertEquals(numDataNodes, cluster.getDataNodes().size()); + // verify blockSeekTo() is able to re-fetch token transparently + in1.seek(0); + assertTrue(checkFile1(in1,expected)); + // verify blockSeekTo() is able to re-fetch token transparently + if (isStriped) { + in2.seek(0); + } else { in2.seekToNewSource(0); - assertTrue(checkFile1(in2)); - // verify fetchBlockByteRange() is able to re-fetch token transparently - assertTrue(checkFile2(in3)); - - } finally { - if (cluster != null) { - cluster.shutdown(); - } } - } + assertTrue(checkFile1(in2,expected)); + // verify fetchBlockByteRange() is able to re-fetch token transparently + assertTrue(checkFile2(in3,expected)); + } /** * Integration testing of access token, involving NN, DN, and Balancer */ @@ -593,4 +621,8 @@ public void testEnd2End() throws Exception { conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); new TestBalancer().integrationTest(conf); } + + protected boolean isBlockTokenExpired(LocatedBlock lb) throws IOException { + return SecurityTestUtil.isBlockTokenExpired(lb.getBlockToken()); + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java new file mode 100644 index 0000000000000..ae33ffe3401b9 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestBlockTokenWithDFSStriped.java @@ -0,0 +1,115 @@ +/** + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.hadoop.hdfs.server.blockmanagement; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.hdfs.DFSConfigKeys; +import org.apache.hadoop.hdfs.MiniDFSCluster; +import org.apache.hadoop.hdfs.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.protocol.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.balancer.TestBalancer; +import org.apache.hadoop.hdfs.util.StripedBlockUtil; +import org.junit.Test; + +import java.io.IOException; + +public class TestBlockTokenWithDFSStriped extends TestBlockTokenWithDFS { + + private final static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + private final static int numDNs = dataBlocks + parityBlocks + 2; + private static MiniDFSCluster cluster; + private static Configuration conf; + + { + BLOCK_SIZE = cellSize * stripesPerBlock; + FILE_SIZE = BLOCK_SIZE * dataBlocks * 3; + } + + private Configuration getConf() { + Configuration conf = super.getConf(numDNs); + conf.setInt("io.bytes.per.checksum", cellSize); + return conf; + } + + @Test + @Override + public void testRead() throws Exception { + conf = getConf(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.getFileSystem().getClient() + .setErasureCodingPolicy("/", null); + try { + cluster.waitActive(); + doTestRead(conf, cluster, true); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + /** + * tested at {@link org.apache.hadoop.hdfs.TestDFSStripedOutputStreamWithFailure#testBlockTokenExpired()} + */ + @Test + @Override + public void testWrite(){ + } + + @Test + @Override + public void testAppend() throws Exception { + //TODO: support Append for striped file + } + + @Test + @Override + public void testEnd2End() throws Exception { + Configuration conf = new Configuration(); + conf.setBoolean(DFSConfigKeys.DFS_BLOCK_ACCESS_TOKEN_ENABLE_KEY, true); + new TestBalancer().integrationTestWithStripedFile(conf); + } + + @Override + protected void tryRead(final Configuration conf, LocatedBlock lblock, + boolean shouldSucceed) { + LocatedStripedBlock lsb = (LocatedStripedBlock) lblock; + LocatedBlock[] internalBlocks = StripedBlockUtil.parseStripedBlockGroup + (lsb, cellSize, dataBlocks, parityBlocks); + for (LocatedBlock internalBlock : internalBlocks) { + super.tryRead(conf, internalBlock, shouldSucceed); + } + } + + @Override + protected boolean isBlockTokenExpired(LocatedBlock lb) throws IOException { + LocatedStripedBlock lsb = (LocatedStripedBlock) lb; + LocatedBlock[] internalBlocks = StripedBlockUtil.parseStripedBlockGroup + (lsb, cellSize, dataBlocks, parityBlocks); + for (LocatedBlock internalBlock : internalBlocks) { + if(super.isBlockTokenExpired(internalBlock)){ + return true; + } + } + return false; + } +} 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 cea686596e5c3..b11b48aed7950 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 @@ -185,9 +185,12 @@ public void testRemovingStorageDoesNotProduceZombies() throws Exception { String datanodeUuid; // Find the first storage which this block is in. try { + BlockInfo storedBlock = + cluster.getNamesystem().getBlockManager(). + getStoredBlock(block.getLocalBlock()); Iterator storageInfoIter = cluster.getNamesystem().getBlockManager(). - getStorages(block.getLocalBlock()).iterator(); + blocksMap.getStorages(storedBlock).iterator(); assertTrue(storageInfoIter.hasNext()); DatanodeStorageInfo info = storageInfoIter.next(); storageIdToRemove = info.getStorageID(); 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 1c3f075d5f426..c33667d5e00f5 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 void testNodeCount() throws Exception { DatanodeDescriptor nonExcessDN = null; for(DatanodeStorageInfo storage : bm.blocksMap.getStorages(block.getLocalBlock())) { final DatanodeDescriptor dn = storage.getDatanodeDescriptor(); - Collection blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid()); + Collection blocks = bm.excessReplicateMap.get(dn.getDatanodeUuid()); if (blocks == null || !blocks.contains(block.getLocalBlock()) ) { nonExcessDN = dn; break; 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 2d7bb440d0cd8..83b3aa0f6a178 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.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.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 void testChooseReplicaToDelete() throws Exception { // All replicas for deletion should be scheduled on lastDN. // And should not actually be deleted, because lastDN does not heartbeat. namesystem.readLock(); - Collection dnBlocks = + Collection dnBlocks = namesystem.getBlockManager().excessReplicateMap.get(lastDNid); assertEquals("Replicas on node " + lastDNid + " should have been deleted", SMALL_FILE_LENGTH / SMALL_BLOCK_SIZE, dnBlocks.size()); 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 90bc1b0ffdc21..a653d45f26643 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 @@ -1275,8 +1275,17 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() BlockManager bm = new BlockManager(mockNS, new HdfsConfiguration()); UnderReplicatedBlocks underReplicatedBlocks = bm.neededReplications; - BlockInfo block1 = genBlockInfo(ThreadLocalRandom.current().nextLong()); - BlockInfo block2 = genBlockInfo(ThreadLocalRandom.current().nextLong()); + long blkID1 = ThreadLocalRandom.current().nextLong(); + if (blkID1 < 0) { + blkID1 *= -1; + } + long blkID2 = ThreadLocalRandom.current().nextLong(); + if (blkID2 < 0) { + blkID2 *= -1; + } + + BlockInfo block1 = genBlockInfo(blkID1); + BlockInfo block2 = genBlockInfo(blkID2); // Adding QUEUE_UNDER_REPLICATED block underReplicatedBlocks.add(block1, 0, 1, 1); @@ -1291,7 +1300,7 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() chosenBlocks = underReplicatedBlocks.chooseUnderReplicatedBlocks(1); assertTheChosenBlocks(chosenBlocks, 1, 0, 0, 0, 0); - final BlockInfo info = new BlockInfoContiguous(block1, (short) 1); + final BlockInfoContiguous info = new BlockInfoContiguous(block1, (short) 1); final BlockCollection mbc = mock(BlockCollection.class); when(mbc.getId()).thenReturn(1000L); when(mbc.getLastBlock()).thenReturn(info); @@ -1315,9 +1324,10 @@ public void testAddStoredBlockDoesNotCauseSkippedReplication() when(storage.removeBlock(any(BlockInfo.class))).thenReturn(true); when(storage.addBlock(any(BlockInfo.class))).thenReturn (DatanodeStorageInfo.AddBlockResult.ADDED); - info.addStorage(storage); + info.addStorage(storage, info); - when(mbc.getLastBlock()).thenReturn(info); + BlockInfo lastBlk = mbc.getLastBlock(); + when(mbc.getLastBlock()).thenReturn(lastBlk, info); bm.convertLastBlockToUnderConstruction(mbc, 0L); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java new file mode 100644 index 0000000000000..00b4e3bba37ec --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/blockmanagement/TestSequentialBlockGroupId.java @@ -0,0 +1,222 @@ +/** + * 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 static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.BLOCK_GROUP_INDEX_MASK; +import static org.apache.hadoop.hdfs.server.common.HdfsServerConstants.MAX_BLOCKS_IN_GROUP; +import static org.hamcrest.CoreMatchers.is; +import static org.hamcrest.CoreMatchers.not; +import static org.junit.Assert.assertThat; +import static org.mockito.Mockito.doAnswer; +import static org.mockito.Mockito.spy; + +import java.io.IOException; +import java.util.List; + +import org.apache.commons.logging.Log; +import org.apache.commons.logging.LogFactory; +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FileSystem; +import org.apache.hadoop.fs.Path; +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.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.LocatedBlock; +import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; +import org.apache.hadoop.test.GenericTestUtils; +import org.junit.After; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; +import org.mockito.internal.util.reflection.Whitebox; +import org.mockito.stubbing.Answer; + +/** + * Tests the sequential blockGroup ID generation mechanism and blockGroup ID + * collision handling. + */ +public class TestSequentialBlockGroupId { + private static final Log LOG = LogFactory + .getLog("TestSequentialBlockGroupId"); + + private final short REPLICATION = 1; + private final long SEED = 0; + private final int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + + private final int stripesPerBlock = 2; + private final int blockSize = cellSize * stripesPerBlock; + private final int numDNs = dataBlocks + parityBlocks + 2; + private final int blockGrpCount = 4; + private final int fileLen = blockSize * dataBlocks * blockGrpCount; + + private MiniDFSCluster cluster; + private FileSystem fs; + private SequentialBlockGroupIdGenerator blockGrpIdGenerator; + private Path ecDir = new Path("/ecDir"); + + @Before + public void setup() throws Exception { + Configuration conf = new HdfsConfiguration(); + conf.setInt(DFSConfigKeys.DFS_REPLICATION_KEY, 1); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + + fs = cluster.getFileSystem(); + blockGrpIdGenerator = cluster.getNamesystem().getBlockIdManager() + .getBlockGroupIdGenerator(); + fs.mkdirs(ecDir); + cluster.getFileSystem().getClient() + .setErasureCodingPolicy("/ecDir", null); + } + + @After + public void teardown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Test that blockGroup IDs are generating unique value. + */ + @Test(timeout = 60000) + public void testBlockGroupIdGeneration() throws IOException { + long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue(); + + // Create a file that is 4 blocks long. + Path path = new Path(ecDir, "testBlockGrpIdGeneration.dat"); + DFSTestUtil.createFile(fs, path, cellSize, fileLen, blockSize, REPLICATION, + SEED); + List blocks = DFSTestUtil.getAllBlocks(fs, path); + assertThat("Wrong BlockGrps", blocks.size(), is(blockGrpCount)); + + // initialising the block group generator for verifying the block id + blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); + // Ensure that the block IDs are generating unique value. + for (int i = 0; i < blocks.size(); ++i) { + blockGrpIdGenerator + .skipTo((blockGrpIdGenerator.getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + + MAX_BLOCKS_IN_GROUP); + long nextBlockExpectedId = blockGrpIdGenerator.getCurrentValue(); + long nextBlockGrpId = blocks.get(i).getBlock().getBlockId(); + LOG.info("BlockGrp" + i + " id is " + nextBlockGrpId); + assertThat("BlockGrpId mismatches!", nextBlockGrpId, + is(nextBlockExpectedId)); + } + } + + /** + * Test that collisions in the blockGroup ID space are handled gracefully. + */ + @Test(timeout = 60000) + public void testTriggerBlockGroupIdCollision() throws IOException { + long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue(); + + // Create a file with a few blocks to rev up the global block ID + // counter. + Path path1 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file1.dat"); + DFSTestUtil.createFile(fs, path1, cellSize, fileLen, blockSize, + REPLICATION, SEED); + List blocks1 = DFSTestUtil.getAllBlocks(fs, path1); + assertThat("Wrong BlockGrps", blocks1.size(), is(blockGrpCount)); + + // Rewind the block ID counter in the name system object. This will result + // in block ID collisions when we try to allocate new blocks. + blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); + + // Trigger collisions by creating a new file. + Path path2 = new Path(ecDir, "testBlockGrpIdCollisionDetection_file2.dat"); + DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize, + REPLICATION, SEED); + List blocks2 = DFSTestUtil.getAllBlocks(fs, path2); + assertThat("Wrong BlockGrps", blocks2.size(), is(blockGrpCount)); + + // Make sure that file1 and file2 block IDs are different + for (LocatedBlock locBlock1 : blocks1) { + long blockId1 = locBlock1.getBlock().getBlockId(); + for (LocatedBlock locBlock2 : blocks2) { + long blockId2 = locBlock2.getBlock().getBlockId(); + assertThat("BlockGrpId mismatches!", blockId1, is(not(blockId2))); + } + } + } + + /** + * Test that collisions in the blockGroup ID when the id is occupied by legacy + * block. + */ + @Test(timeout = 60000) + public void testTriggerBlockGroupIdCollisionWithLegacyBlockId() + throws Exception { + long blockGroupIdInitialValue = blockGrpIdGenerator.getCurrentValue(); + blockGrpIdGenerator + .skipTo((blockGrpIdGenerator.getCurrentValue() & ~BLOCK_GROUP_INDEX_MASK) + + MAX_BLOCKS_IN_GROUP); + final long curBlockGroupIdValue = blockGrpIdGenerator.getCurrentValue(); + + // Creates contiguous block with negative blockId so that it would trigger + // collision during blockGroup Id generation + FSNamesystem fsn = cluster.getNamesystem(); + // Replace SequentialBlockIdGenerator with a spy + SequentialBlockIdGenerator blockIdGenerator = spy(fsn.getBlockIdManager() + .getBlockIdGenerator()); + Whitebox.setInternalState(fsn.getBlockIdManager(), "blockIdGenerator", + blockIdGenerator); + SequentialBlockIdGenerator spySequentialBlockIdGenerator = new SequentialBlockIdGenerator( + null) { + @Override + public long nextValue() { + return curBlockGroupIdValue; + } + }; + final Answer delegator = new GenericTestUtils.DelegateAnswer( + spySequentialBlockIdGenerator); + doAnswer(delegator).when(blockIdGenerator).nextValue(); + + Path path1 = new Path("/testCollisionWithLegacyBlock_file1.dat"); + DFSTestUtil.createFile(fs, path1, 1024, REPLICATION, SEED); + + List contiguousBlocks = DFSTestUtil.getAllBlocks(fs, path1); + assertThat(contiguousBlocks.size(), is(1)); + Assert.assertEquals("Unexpected BlockId!", curBlockGroupIdValue, + contiguousBlocks.get(0).getBlock().getBlockId()); + + // Reset back to the initial value to trigger collision + blockGrpIdGenerator.setCurrentValue(blockGroupIdInitialValue); + // Trigger collisions by creating a new file. + Path path2 = new Path(ecDir, "testCollisionWithLegacyBlock_file2.dat"); + DFSTestUtil.createFile(fs, path2, cellSize, fileLen, blockSize, + REPLICATION, SEED); + List blocks2 = DFSTestUtil.getAllBlocks(fs, path2); + assertThat("Wrong BlockGrps", blocks2.size(), is(blockGrpCount)); + + // Make sure that file1 and file2 block IDs are different + for (LocatedBlock locBlock1 : contiguousBlocks) { + long blockId1 = locBlock1.getBlock().getBlockId(); + for (LocatedBlock locBlock2 : blocks2) { + long blockId2 = locBlock2.getBlock().getBlockId(); + assertThat("BlockGrpId mismatches!", blockId1, is(not(blockId2))); + } + } + } +} 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 de36e077aea81..7cd2e19427d4a 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 @@ -19,6 +19,8 @@ package org.apache.hadoop.hdfs.server.blockmanagement; import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; import org.junit.Test; import static org.junit.Assert.assertEquals; @@ -28,10 +30,19 @@ public class TestUnderReplicatedBlockQueues { + private final ErasureCodingPolicy ecPolicy = + ErasureCodingPolicyManager.getSystemDefaultPolicy(); + private BlockInfo genBlockInfo(long id) { return new BlockInfoContiguous(new Block(id), (short) 3); } + private BlockInfo genStripedBlockInfo(long id, long numBytes) { + BlockInfoStriped sblk = new BlockInfoStriped(new Block(id), ecPolicy); + sblk.setNumBytes(numBytes); + return sblk; + } + /** * Test that adding blocks with different replication counts puts them * into different queues @@ -85,6 +96,54 @@ public void testBlockPriorities() throws Throwable { assertEquals(2, queues.getCorruptReplOneBlockSize()); } + @Test + public void testStripedBlockPriorities() throws Throwable { + int dataBlkNum = ecPolicy.getNumDataUnits(); + int parityBlkNUm = ecPolicy.getNumParityUnits(); + doTestStripedBlockPriorities(1, parityBlkNUm); + doTestStripedBlockPriorities(dataBlkNum, parityBlkNUm); + } + + private void doTestStripedBlockPriorities(int dataBlkNum, int parityBlkNum) + throws Throwable { + int groupSize = dataBlkNum + parityBlkNum; + long numBytes = ecPolicy.getCellSize() * dataBlkNum; + UnderReplicatedBlocks queues = new UnderReplicatedBlocks(); + + // add a striped block which been left NUM_DATA_BLOCKS internal blocks + BlockInfo block1 = genStripedBlockInfo(-100, numBytes); + assertAdded(queues, block1, dataBlkNum, 0, groupSize); + assertEquals(1, queues.getUnderReplicatedBlockCount()); + assertEquals(1, queues.size()); + assertInLevel(queues, block1, UnderReplicatedBlocks.QUEUE_HIGHEST_PRIORITY); + + // add a striped block which been left NUM_DATA_BLOCKS+1 internal blocks + BlockInfo block2 = genStripedBlockInfo(-200, numBytes); + assertAdded(queues, block2, dataBlkNum + 1, 0, groupSize); + assertEquals(2, queues.getUnderReplicatedBlockCount()); + assertEquals(2, queues.size()); + assertInLevel(queues, block2, + UnderReplicatedBlocks.QUEUE_VERY_UNDER_REPLICATED); + + // add a striped block which been left NUM_DATA_BLOCKS+2 internal blocks + BlockInfo block3 = genStripedBlockInfo(-300, numBytes); + assertAdded(queues, block3, dataBlkNum + 2, 0, groupSize); + assertEquals(3, queues.getUnderReplicatedBlockCount()); + assertEquals(3, queues.size()); + assertInLevel(queues, block3, + UnderReplicatedBlocks.QUEUE_UNDER_REPLICATED); + + // add a corrupted block + BlockInfo block_corrupt = genStripedBlockInfo(-400, numBytes); + assertEquals(0, queues.getCorruptBlockSize()); + assertAdded(queues, block_corrupt, dataBlkNum - 1, 0, groupSize); + assertEquals(4, queues.size()); + assertEquals(3, queues.getUnderReplicatedBlockCount()); + assertEquals(1, queues.getCorruptBlockSize()); + assertInLevel(queues, block_corrupt, + UnderReplicatedBlocks.QUEUE_WITH_CORRUPT_BLOCKS); + } + private void assertAdded(UnderReplicatedBlocks queues, BlockInfo block, int curReplicas, diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java index acbd8a86228ae..4316854d4ba55 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/datanode/SimulatedFSDataset.java @@ -99,7 +99,7 @@ public static void setFactory(Configuration conf) { public static byte simulatedByte(Block b, long offsetInBlk) { byte firstByte = (byte) (b.getBlockId() & BYTE_MASK); - return (byte) ((firstByte + offsetInBlk) & BYTE_MASK); + return (byte) ((firstByte + offsetInBlk % 29) & BYTE_MASK); } public static final String CONFIG_PROPERTY_CAPACITY = 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 989e216409a4b..d8c651f5fe963 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 @@ -227,15 +227,6 @@ private static Block getDummyBlock() { 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. @@ -251,8 +242,9 @@ public void testNnLearnsNewStorages() // 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); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java index d3d814c10d296..7cf56562a5972 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/mover/TestMover.java @@ -27,6 +27,7 @@ import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.Path; import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.hdfs.DFSConfigKeys; import org.apache.hadoop.hdfs.DistributedFileSystem; import org.apache.hadoop.hdfs.DFSTestUtil; @@ -34,7 +35,12 @@ import org.apache.hadoop.hdfs.HdfsConfiguration; import org.apache.hadoop.hdfs.MiniDFSCluster; import org.apache.hadoop.hdfs.MiniDFSNNTopology; +import org.apache.hadoop.hdfs.NameNodeProxies; +import org.apache.hadoop.hdfs.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.ClientProtocol; +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.server.balancer.Dispatcher.DBlock; import org.apache.hadoop.hdfs.server.balancer.ExitStatus; import org.apache.hadoop.hdfs.server.balancer.NameNodeConnector; @@ -99,7 +105,7 @@ public void testScheduleSameBlock() throws IOException { final LocatedBlock lb = dfs.getClient().getLocatedBlocks(file, 0).get(0); final List locations = MLocation.toLocations(lb); final MLocation ml = locations.get(0); - final DBlock db = mover.newDBlock(lb.getBlock().getLocalBlock(), locations); + final DBlock db = mover.newDBlock(lb, locations, null); final List storageTypes = new ArrayList( Arrays.asList(StorageType.DEFAULT, StorageType.DEFAULT)); @@ -409,4 +415,119 @@ public void testMoverFailedRetry() throws Exception { cluster.shutdown(); } } + + int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final static int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final static int stripesPerBlock = 4; + static int DEFAULT_STRIPE_BLOCK_SIZE = cellSize * stripesPerBlock; + + static void initConfWithStripe(Configuration conf) { + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, DEFAULT_STRIPE_BLOCK_SIZE); + conf.setLong(DFSConfigKeys.DFS_HEARTBEAT_INTERVAL_KEY, 1L); + conf.setLong(DFSConfigKeys.DFS_NAMENODE_REPLICATION_INTERVAL_KEY, 1L); + conf.setBoolean(DFSConfigKeys.DFS_NAMENODE_REPLICATION_CONSIDERLOAD_KEY, false); + } + + @Test(timeout = 300000) + public void testMoverWithStripedFile() throws Exception { + final Configuration conf = new HdfsConfiguration(); + initConfWithStripe(conf); + + // start 10 datanodes + int numOfDatanodes =10; + int storagesPerDatanode=2; + long capacity = 10 * DEFAULT_STRIPE_BLOCK_SIZE; + long[][] capacities = new long[numOfDatanodes][storagesPerDatanode]; + for (int i = 0; i < numOfDatanodes; i++) { + for(int j=0;j infos = Arrays.asList(bg.getLocations()); + + // let a internal block be over replicated with 2 redundant blocks. + // Therefor number of internal blocks is over GROUP_SIZE. (5 data blocks + + // 3 parity blocks + 2 redundant blocks > GROUP_SIZE) + blk.setBlockId(groupId + 2); + List dataNodeList = cluster.getDataNodes(); + for (int i = 0; i < numDNs; i++) { + if (!infos.contains(dataNodeList.get(i).getDatanodeId())) { + cluster.injectBlocks(i, Arrays.asList(blk), bpid); + System.out.println("XXX: inject block into datanode " + i); + } + } + + // update blocksMap + cluster.triggerBlockReports(); + // add to invalidates + cluster.triggerHeartbeats(); + // datanode delete block + cluster.triggerHeartbeats(); + // update blocksMap + cluster.triggerBlockReports(); + + // verify that all internal blocks exists + lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); + } + + @Test + public void testProcessOverReplicatedAndCorruptStripedBlock() + throws Exception { + long fileLen = DATA_BLK_NUM * BLOCK_SIZE; + DFSTestUtil.createStripedFile(cluster, filePath, null, 1, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + long gs = bg.getBlock().getGenerationStamp(); + String bpid = bg.getBlock().getBlockPoolId(); + long groupId = bg.getBlock().getBlockId(); + Block blk = new Block(groupId, BLOCK_SIZE, gs); + BlockInfoStriped blockInfo = new BlockInfoStriped(blk, + ErasureCodingPolicyManager.getSystemDefaultPolicy()); + for (int i = 0; i < GROUP_SIZE; i++) { + blk.setBlockId(groupId + i); + cluster.injectBlocks(i, Arrays.asList(blk), bpid); + } + cluster.triggerBlockReports(); + + // let a internal block be corrupt + BlockManager bm = cluster.getNamesystem().getBlockManager(); + List infos = Arrays.asList(bg.getLocations()); + List storages = Arrays.asList(bg.getStorageIDs()); + cluster.getNamesystem().writeLock(); + try { + bm.findAndMarkBlockAsCorrupt(lbs.getLastLocatedBlock().getBlock(), + infos.get(0), storages.get(0), "TEST"); + } finally { + cluster.getNamesystem().writeUnlock(); + } + assertEquals(1, bm.countNodes(blockInfo).corruptReplicas()); + + // let a internal block be over replicated with 2 redundant block. + blk.setBlockId(groupId + 2); + cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid); + cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid); + + // update blocksMap + cluster.triggerBlockReports(); + // add to invalidates + cluster.triggerHeartbeats(); + // datanode delete block + cluster.triggerHeartbeats(); + // update blocksMap + cluster.triggerBlockReports(); + + // verify that all internal blocks exists + lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE); + } + + @Test + public void testProcessOverReplicatedAndMissingStripedBlock() + throws Exception { + long fileLen = CELLSIZE * DATA_BLK_NUM; + DFSTestUtil.createStripedFile(cluster, filePath, null, 1, + NUM_STRIPE_PER_BLOCK, false); + LocatedBlocks lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + LocatedStripedBlock bg = (LocatedStripedBlock) (lbs.get(0)); + long gs = bg.getBlock().getGenerationStamp(); + String bpid = bg.getBlock().getBlockPoolId(); + long groupId = bg.getBlock().getBlockId(); + Block blk = new Block(groupId, BLOCK_SIZE, gs); + // only inject GROUP_SIZE - 1 blocks, so there is one block missing + for (int i = 0; i < GROUP_SIZE - 1; i++) { + blk.setBlockId(groupId + i); + cluster.injectBlocks(i, Arrays.asList(blk), bpid); + } + cluster.triggerBlockReports(); + + // let a internal block be over replicated with 2 redundant blocks. + // Therefor number of internal blocks is over GROUP_SIZE. (5 data blocks + + // 3 parity blocks + 2 redundant blocks > GROUP_SIZE) + blk.setBlockId(groupId + 2); + cluster.injectBlocks(numDNs - 3, Arrays.asList(blk), bpid); + cluster.injectBlocks(numDNs - 2, Arrays.asList(blk), bpid); + + // update blocksMap + cluster.triggerBlockReports(); + Thread.sleep(2000); + // add to invalidates + cluster.triggerHeartbeats(); + // datanode delete block + cluster.triggerHeartbeats(); + // update blocksMap + cluster.triggerBlockReports(); + + // Since one block is missing, when over-replicated blocks got deleted, + // we are left GROUP_SIZE - 1 blocks. + lbs = cluster.getNameNodeRpc().getBlockLocations( + filePath.toString(), 0, fileLen); + StripedFileTestUtil.verifyLocatedStripedBlocks(lbs, GROUP_SIZE - 1); + } + +} 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 0000000000000..836bff1b4a7c4 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestAddStripedBlocks.java @@ -0,0 +1,430 @@ +/** + * 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.DFSStripedOutputStream; +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.StripedFileTestUtil; +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.BlockManager; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockManagerTestUtil; +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; +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; +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.apache.hadoop.hdfs.DFSConfigKeys.DFS_BYTES_PER_CHECKSUM_DEFAULT; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS; +import static org.junit.Assert.assertEquals; + +public class TestAddStripedBlocks { + private final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS + + StripedFileTestUtil.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.getClient().setErasureCodingPolicy("/", null); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + /** + * Make sure the IDs of striped blocks do not conflict + */ + @Test + public void testAllocateBlockId() throws Exception { + Path testPath = new Path("/testfile"); + // create a file while allocates a new block + DFSTestUtil.writeFile(dfs, testPath, "hello, world!"); + LocatedBlocks lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0); + final long firstId = lb.get(0).getBlock().getBlockId(); + // delete the file + dfs.delete(testPath, true); + + // allocate a new block, and make sure the new block's id does not conflict + // with the previous one + DFSTestUtil.writeFile(dfs, testPath, "hello again"); + lb = dfs.getClient().getLocatedBlocks(testPath.toString(), 0); + final long secondId = lb.get(0).getBlock().getBlockId(); + Assert.assertEquals(firstId + HdfsServerConstants.MAX_BLOCKS_IN_GROUP, secondId); + } + + private static void writeAndFlushStripedOutputStream( + DFSStripedOutputStream out, int chunkSize) throws IOException { + // FSOutputSummer.BUFFER_NUM_CHUNKS == 9 + byte[] toWrite = new byte[chunkSize * 9 + 1]; + out.write(toWrite); + DFSTestUtil.flushInternal(out); + } + + @Test (timeout=60000) + 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); + writeAndFlushStripedOutputStream( + (DFSStripedOutputStream) out.getWrappedStream(), + DFS_BYTES_PER_CHECKSUM_DEFAULT); + + 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(), true); + + // restart NameNode to check editlog + 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); + + // 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); + } finally { + IOUtils.cleanup(null, out); + } + } + + private void checkStripedBlockUC(BlockInfoStriped block, + boolean checkReplica) { + assertEquals(0, block.numNodes()); + Assert.assertFalse(block.isComplete()); + Assert.assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, block.getDataBlockNum()); + Assert.assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, + block.getParityBlockNum()); + Assert.assertEquals(0, + block.getBlockId() & HdfsServerConstants.BLOCK_GROUP_INDEX_MASK); + + Assert.assertEquals(HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, + block.getBlockUCState()); + if (checkReplica) { + Assert.assertEquals(GROUP_SIZE, + block.getUnderConstructionFeature().getNumExpectedLocations()); + DatanodeStorageInfo[] storages = block.getUnderConstructionFeature() + .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; + } + + @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); + writeAndFlushStripedOutputStream( + (DFSStripedOutputStream) out.getWrappedStream(), + DFS_BYTES_PER_CHECKSUM_DEFAULT); + + FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + BlockInfoStriped lastBlk = (BlockInfoStriped) fileNode.getLastBlock(); + DatanodeInfo[] expectedDNs = DatanodeStorageInfo.toDatanodeInfos( + lastBlk.getUnderConstructionFeature().getExpectedStorageLocations()); + int[] indices = lastBlk.getUnderConstructionFeature().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 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(); + + DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature() + .getExpectedStorageLocations(); + int[] indices = lastBlock.getUnderConstructionFeature().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 BlockInfoUCStriped is correct + int i = 0; + for (DataNode dn : cluster.getDataNodes()) { + final Block block = new Block(lastBlock.getBlockId() + i++, + 0, lastBlock.getGenerationStamp()); + 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 = lastBlock.getUnderConstructionFeature().getExpectedStorageLocations(); + indices = lastBlock.getUnderConstructionFeature().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 blocks = new ArrayList<>(); + ReplicaBeingWritten replica = new ReplicaBeingWritten(block, null, null, + null); + blocks.add(replica); + BlockListAsLongs bll = BlockListAsLongs.encode(blocks); + StorageBlockReport[] reports = {new StorageBlockReport(storage, + bll)}; + cluster.getNameNodeRpc().blockReport(dn.getDNRegistrationForBP(bpId), + bpId, reports, null); + } + + DatanodeStorageInfo[] locs = lastBlock.getUnderConstructionFeature() + .getExpectedStorageLocations(); + int[] indices = lastBlock.getUnderConstructionFeature().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]); + } + } + + @Test + public void testCheckStripedReplicaCorrupt() throws Exception { + final int numBlocks = 4; + final int numStripes = 4; + final Path filePath = new Path("/corrupt"); + final FSNamesystem ns = cluster.getNameNode().getNamesystem(); + final BlockManager bm = ns.getBlockManager(); + DFSTestUtil.createStripedFile(cluster, filePath, null, + numBlocks, numStripes, false); + + INodeFile fileNode = ns.getFSDirectory().getINode(filePath.toString()). + asFile(); + Assert.assertTrue(fileNode.isStriped()); + BlockInfo stored = fileNode.getBlocks()[0]; + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(0, ns.getCorruptReplicaBlocks()); + + // Now send a block report with correct size + DatanodeStorage storage = new DatanodeStorage(UUID.randomUUID().toString()); + final Block reported = new Block(stored); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE); + StorageReceivedDeletedBlocks[] reports = DFSTestUtil + .makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(0).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(0, ns.getCorruptReplicaBlocks()); + + // Now send a block report with wrong size + reported.setBlockId(stored.getBlockId() + 1); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE - 1); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(1).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + + // Now send a parity block report with correct size + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(2).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + + // Now send a parity block report with wrong size + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 1); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(3).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + // the total number of corrupted block info is still 1 + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + // 2 internal blocks corrupted + Assert.assertEquals(2, bm.getCorruptReplicas(stored).size()); + + // Now change the size of stored block, and test verifying the last + // block size + stored.setNumBytes(stored.getNumBytes() + 10); + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS + 2); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(4).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); + + // Now send a parity block report with correct size based on adjusted + // size of stored block + /** Now stored block has {@link numStripes} full stripes + a cell + 10 */ + stored.setNumBytes(stored.getNumBytes() + BLOCK_STRIPED_CELL_SIZE); + reported.setBlockId(stored.getBlockId()); + reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(0).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); + + reported.setBlockId(stored.getBlockId() + 1); + reported.setNumBytes(numStripes * BLOCK_STRIPED_CELL_SIZE + 10); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(5).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); + + reported.setBlockId(stored.getBlockId() + NUM_DATA_BLOCKS); + reported.setNumBytes((numStripes + 1) * BLOCK_STRIPED_CELL_SIZE); + reports = DFSTestUtil.makeReportForReceivedBlock(reported, + ReceivedDeletedBlockInfo.BlockStatus.RECEIVED_BLOCK, storage); + ns.processIncrementalBlockReport( + cluster.getDataNodes().get(2).getDatanodeId(), reports[0]); + BlockManagerTestUtil.updateState(ns.getBlockManager()); + Assert.assertEquals(1, ns.getCorruptReplicaBlocks()); + Assert.assertEquals(3, bm.getCorruptReplicas(stored).size()); + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java index c5262d457b9fa..ec3d924264a08 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestDeadDatanode.java @@ -162,7 +162,8 @@ public void testDeadNodeAsBlockTarget() throws Exception { // choose the targets, but local node should not get selected as this is not // part of the cluster anymore DatanodeStorageInfo[] results = bm.chooseTarget4NewBlock("/hello", 3, - clientNode, new HashSet(), 256 * 1024 * 1024L, null, (byte) 7); + clientNode, new HashSet(), 256 * 1024 * 1024L, null, (byte) 7, + false); for (DatanodeStorageInfo datanodeStorageInfo : results) { assertFalse("Dead node should not be choosen", datanodeStorageInfo .getDatanodeDescriptor().equals(clientNode)); 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 df07a62d021b4..2bb3d5f89ca0e 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 @@ -21,6 +21,7 @@ import static org.junit.Assert.assertEquals; import static org.junit.Assert.assertTrue; import static org.junit.Assert.fail; +import static org.junit.Assert.assertFalse; import static org.mockito.Mockito.doNothing; import static org.mockito.Mockito.spy; @@ -38,14 +39,23 @@ 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.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; +import org.apache.hadoop.hdfs.DistributedFileSystem; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoContiguous; 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.GenericTestUtils; import org.apache.hadoop.test.PathUtils; @@ -65,6 +75,9 @@ public class TestFSEditLogLoader { private static final File TEST_DIR = PathUtils.getTestDir(TestFSEditLogLoader.class); private static final int NUM_DATA_NODES = 0; + + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); @Test public void testDisplayRecentEditLogOpCodes() throws IOException { @@ -417,4 +430,260 @@ public void testFSEditLogOpCodes() throws IOException { 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 = StripedFileTestUtil.NUM_DATA_BLOCKS; + short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS; + + //set the storage policy of the directory + fs.mkdir(new Path(testDir), new FsPermission("755")); + fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null); + + // 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), testECPolicy); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.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.isStriped()); + + BlockInfo[] blks = inodeLoaded.getBlocks(); + assertEquals(1, blks.length); + assertEquals(blkId, blks[0].getBlockId()); + assertEquals(blkNumBytes, blks[0].getNumBytes()); + assertEquals(timestamp, blks[0].getGenerationStamp()); + assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum()); + assertEquals(parityNum, ((BlockInfoStriped)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 = StripedFileTestUtil.NUM_DATA_BLOCKS; + short parityNum = StripedFileTestUtil.NUM_PARITY_BLOCKS; + + //set the storage policy of the directory + fs.mkdir(new Path(testDir), new FsPermission("755")); + fs.getClient().getNamenode().setErasureCodingPolicy(testDir, null); + + //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), testECPolicy); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.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.isStriped()); + + BlockInfo[] blks = inodeLoaded.getBlocks(); + assertEquals(1, blks.length); + assertTrue(blks[0].isStriped()); + assertEquals(blkId, blks[0].getBlockId()); + assertEquals(newBlkNumBytes, blks[0].getNumBytes()); + assertEquals(newTimestamp, blks[0].getGenerationStamp()); + assertEquals(blockNum, ((BlockInfoStriped)blks[0]).getDataBlockNum()); + assertEquals(parityNum, ((BlockInfoStriped)blks[0]).getParityBlockNum()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForAddBlock() 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 = "/test_block_manager"; + String testFile = "testfile_addblock"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_addblock"; + String clientMachine = "testMachine_addblock"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + //check whether the hasNonEcBlockUsingStripedID is set + //after loading a addblock-editlog + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + fns.getEditLog().logAddBlock(testFilePath, file); + file.toCompleteFile(System.currentTimeMillis()); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForUpdateBlocks() + 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 = "/test_block_manager"; + String testFile = "testfile_002"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser2"; + String clientMachine = "testMachine1"; + long blkId = 100; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + file.toCompleteFile(System.currentTimeMillis()); + + long newBlkNumBytes = 1024*8; + long newTimestamp = 1426222918+3600; + file.toUnderConstruction(clientName, clientMachine); + file.getLastBlock().setBlockId(-100); + file.getLastBlock().setNumBytes(newBlkNumBytes); + file.getLastBlock().setGenerationStamp(newTimestamp); + fns.getEditLog().logUpdateBlocks(testFilePath, file, true); + file.toCompleteFile(System.currentTimeMillis()); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + } 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 df20fd62d499f..e45d08d6c5860 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 @@ -22,11 +22,25 @@ 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.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +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.StartupOption; import org.junit.Assert; +import org.apache.hadoop.fs.permission.PermissionStatus; +import org.apache.hadoop.fs.permission.FsPermission; import org.apache.hadoop.conf.Configuration; import org.apache.hadoop.fs.FSDataOutputStream; import org.apache.hadoop.fs.FileSystem; @@ -34,15 +48,15 @@ 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; +import org.apache.hadoop.hdfs.StripedFileTestUtil; 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.BlockInfo; 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.server.namenode.NNStorage.NameNodeDirType; import org.apache.hadoop.hdfs.util.MD5FileUtils; @@ -50,10 +64,15 @@ import org.apache.hadoop.test.PathUtils; import org.junit.Test; +import static org.junit.Assert.assertArrayEquals; + public class TestFSImage { private static final String HADOOP_2_7_ZER0_BLOCK_SIZE_TGZ = "image-with-zero-block-size.tar.gz"; + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + @Test public void testPersist() throws IOException { Configuration conf = new Configuration(); @@ -120,6 +139,123 @@ private void testPersistHelper(Configuration conf) throws IOException { } } + private void testSaveAndLoadStripedINodeFile(FSNamesystem fsn, Configuration conf, + boolean isUC) throws IOException{ + // contruct a INode with StripedBlock for saving and loading + fsn.setErasureCodingPolicy("/", null, false); + 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; + INodeFile file = new INodeFile(id, name, permissionStatus, mtime, atime, + blks, replication, preferredBlockSize, (byte) 0, true); + ByteArrayOutputStream bs = new ByteArrayOutputStream(); + + //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), + testECPolicy); + file.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()); + // TODO for striped blocks, we currently save and load them as contiguous + // blocks to/from legacy fsimage + assertEquals(3, fileByLoaded.getBlocks().length); + assertEquals(preferredBlockSize, fileByLoaded.getPreferredBlockSize()); + + 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 testSaveAndLoadStripedINodeFile() throws IOException{ + Configuration conf = new Configuration(); + MiniDFSCluster cluster = null; + try { + cluster = new MiniDFSCluster.Builder(conf).build(); + cluster.waitActive(); + testSaveAndLoadStripedINodeFile(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 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(); + testSaveAndLoadStripedINodeFile(cluster.getNamesystem(), conf, true); + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + /** * On checkpointing , stale fsimage checkpoint file should be deleted. */ @@ -260,8 +396,7 @@ public void testZeroBlockSize() throws Exception { .format(false) .manageDataDfsDirs(false) .manageNameDfsDirs(false) - .waitSafeMode(false) - .startupOption(StartupOption.UPGRADE) + .waitSafeMode(false).startupOption(StartupOption.UPGRADE) .build(); try { FileSystem fs = cluster.getFileSystem(); @@ -274,4 +409,207 @@ public void testZeroBlockSize() throws Exception { FileUtil.fullyDelete(dfsDir); } } + + /** + * Ensure that FSImage supports BlockGroup. + */ + @Test + public void testSupportBlockGroup() throws IOException { + final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS + + StripedFileTestUtil.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.getClient().getNamenode().setErasureCodingPolicy("/", null); + 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(); + assertTrue(inode.isStriped()); + BlockInfo[] blks = inode.getBlocks(); + assertEquals(1, blks.length); + assertTrue(blks[0].isStriped()); + assertEquals(StripedFileTestUtil.NUM_DATA_BLOCKS, ((BlockInfoStriped)blks[0]).getDataBlockNum()); + assertEquals(StripedFileTestUtil.NUM_PARITY_BLOCKS, ((BlockInfoStriped)blks[0]).getParityBlockNum()); + } finally { + cluster.shutdown(); + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForLoadFile() 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 = "/test_block_manager"; + String testFile = "testfile_loadfile"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_loadfile"; + String clientMachine = "testMachine_loadfile"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + file.toCompleteFile(System.currentTimeMillis()); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + //after nonEcBlockUsingStripedID is deleted + //the hasNonEcBlockUsingStripedID is set to false + fs = cluster.getFileSystem(); + fs.delete(p,false); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertFalse(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForLoadUCFile() + 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 = "/test_block_manager"; + String testFile = "testfile_loaducfile"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_loaducfile"; + String clientMachine = "testMachine_loaducfile"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + fs.mkdir(new Path(testDir), new FsPermission("755")); + Path p = new Path(testFilePath); + + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } + + @Test + public void testHasNonEcBlockUsingStripedIDForLoadSnapshot() + 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 = "/test_block_manager"; + String testFile = "testfile_loadSnapshot"; + String testFilePath = testDir + "/" + testFile; + String clientName = "testUser_loadSnapshot"; + String clientMachine = "testMachine_loadSnapshot"; + long blkId = -1; + long blkNumBytes = 1024; + long timestamp = 1426222918; + + Path d = new Path(testDir); + fs.mkdir(d, new FsPermission("755")); + fs.allowSnapshot(d); + + Path p = new Path(testFilePath); + DFSTestUtil.createFile(fs, p, 0, (short) 1, 1); + BlockInfoContiguous cBlk = new BlockInfoContiguous( + new Block(blkId, blkNumBytes, timestamp), (short)3); + INodeFile file = (INodeFile)fns.getFSDirectory().getINode(testFilePath); + file.toUnderConstruction(clientName, clientMachine); + file.addBlock(cBlk); + file.toCompleteFile(System.currentTimeMillis()); + + fs.createSnapshot(d,"testHasNonEcBlockUsingStripeID"); + fs.truncate(p,0); + fns.enterSafeMode(false); + fns.saveNamespace(0, 0); + cluster.restartNameNodes(); + cluster.waitActive(); + fns = cluster.getNamesystem(); + assertTrue(fns.getBlockManager().hasNonEcBlockUsingStripedID()); + + cluster.shutdown(); + cluster = null; + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } } 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 34677ef217b89..0423d41f368d1 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 @@ -56,6 +56,7 @@ 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.common.HdfsServerConstants; import org.apache.hadoop.hdfs.server.common.HdfsServerConstants.StartupOption; import org.apache.hadoop.security.UserGroupInformation; @@ -1033,7 +1034,8 @@ public void testTruncateRecovery() throws IOException { 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(); 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 a84ddd045c8bb..6df88fd0976b0 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 @@ -86,6 +86,8 @@ import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeDescriptor; import org.apache.hadoop.hdfs.server.blockmanagement.DatanodeManager; import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.Result; +import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ReplicationResult; +import org.apache.hadoop.hdfs.server.namenode.NamenodeFsck.ErasureCodingResult; import org.apache.hadoop.hdfs.server.protocol.NamenodeProtocols; import org.apache.hadoop.hdfs.tools.DFSck; import org.apache.hadoop.io.IOUtils; @@ -1071,13 +1073,14 @@ public void testFsckMissingReplicas() throws IOException { final HdfsFileStatus file = namenode.getRpcServer().getFileInfo(pathString); assertNotNull(file); - Result res = new Result(conf); - fsck.check(pathString, file, res); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); + fsck.check(pathString, file, replRes, ecRes); // Also print the output from the fsck, for ex post facto sanity checks System.out.println(result.toString()); - assertEquals(res.missingReplicas, + assertEquals(replRes.missingReplicas, (NUM_BLOCKS*REPL_FACTOR) - (NUM_BLOCKS*NUM_REPLICAS)); - assertEquals(res.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR); + assertEquals(replRes.numExpectedReplicas, NUM_BLOCKS*REPL_FACTOR); } finally { if(dfs != null) { dfs.close(); @@ -1148,10 +1151,11 @@ public void testFsckMisPlacedReplicas() throws IOException { final HdfsFileStatus file = namenode.getRpcServer().getFileInfo(pathString); assertNotNull(file); - Result res = new Result(conf); - fsck.check(pathString, file, res); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); + fsck.check(pathString, file, replRes, ecRes); // check misReplicatedBlock number. - assertEquals(res.numMisReplicatedBlocks, NUM_BLOCKS); + assertEquals(replRes.numMisReplicatedBlocks, NUM_BLOCKS); } finally { if(dfs != null) { dfs.close(); @@ -1211,15 +1215,16 @@ public void testFsckFileNotFound() throws Exception { HdfsFileStatus file = new HdfsFileStatus(length, isDir, blockReplication, blockSize, modTime, accessTime, perms, owner, group, symlink, - path, fileId, numChildren, null, storagePolicy); - Result res = new Result(conf); + path, fileId, numChildren, null, storagePolicy, null); + Result replRes = new ReplicationResult(conf); + Result ecRes = new ErasureCodingResult(conf); try { - fsck.check(pathString, file, res); + fsck.check(pathString, file, replRes, ecRes); } catch (Exception e) { fail("Unexpected exception " + e.getMessage()); } - assertTrue(res.toString().contains("HEALTHY")); + assertTrue(replRes.isHealthy()); } /** Test fsck with symlinks in the filesystem */ @@ -1657,4 +1662,60 @@ public void testFsckWithDecommissionedReplicas() throws Exception { } } } + + @Test + public void testECFsck() throws Exception { + MiniDFSCluster cluster = null; + FileSystem fs = null; + try { + Configuration conf = new HdfsConfiguration(); + final long precision = 1L; + conf.setLong(DFSConfigKeys.DFS_NAMENODE_ACCESSTIME_PRECISION_KEY, precision); + conf.setLong(DFSConfigKeys.DFS_BLOCKREPORT_INTERVAL_MSEC_KEY, 10000L); + int totalSize = ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumDataUnits() + + ErasureCodingPolicyManager.getSystemDefaultPolicy().getNumParityUnits(); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(totalSize).build(); + fs = cluster.getFileSystem(); + + // create a contiguous file + Path replDirPath = new Path("/replicated"); + Path replFilePath = new Path(replDirPath, "replfile"); + final short factor = 3; + DFSTestUtil.createFile(fs, replFilePath, 1024, factor, 0); + DFSTestUtil.waitReplication(fs, replFilePath, factor); + + // create a large striped file + Path ecDirPath = new Path("/striped"); + Path largeFilePath = new Path(ecDirPath, "largeFile"); + DFSTestUtil.createStripedFile(cluster, largeFilePath, ecDirPath, 1, 2, true); + + // create a small striped file + Path smallFilePath = new Path(ecDirPath, "smallFile"); + DFSTestUtil.writeFile(fs, smallFilePath, "hello world!"); + + long replTime = fs.getFileStatus(replFilePath).getAccessTime(); + long ecTime = fs.getFileStatus(largeFilePath).getAccessTime(); + Thread.sleep(precision); + setupAuditLogs(); + String outStr = runFsck(conf, 0, true, "/"); + verifyAuditLogs(); + assertEquals(replTime, fs.getFileStatus(replFilePath).getAccessTime()); + assertEquals(ecTime, fs.getFileStatus(largeFilePath).getAccessTime()); + System.out.println(outStr); + assertTrue(outStr.contains(NamenodeFsck.HEALTHY_STATUS)); + if (fs != null) {try{fs.close();} catch(Exception e){}} + cluster.shutdown(); + + // restart the cluster; bring up namenode but not the data nodes + cluster = new MiniDFSCluster.Builder(conf) + .numDataNodes(0).format(false).build(); + outStr = runFsck(conf, 1, true, "/"); + // expect the result is corrupt + assertTrue(outStr.contains(NamenodeFsck.CORRUPT_STATUS)); + System.out.println(outStr); + } finally { + if (fs != null) {try{fs.close();} catch(Exception e){}} + if (cluster != null) { cluster.shutdown(); } + } + } } diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java index 9f5850a341140..c33e668b7ec88 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestINodeFile.java @@ -96,7 +96,7 @@ INodeFile createINodeFile(short replication, long preferredBlockSize) { private static INodeFile createINodeFile(byte storagePolicyID) { return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, - null, (short)3, 1024L, storagePolicyID); + null, (short)3, 1024L, storagePolicyID, false); } @Test diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java new file mode 100644 index 0000000000000..26f9b8ee4f765 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestQuotaWithStripedBlocks.java @@ -0,0 +1,125 @@ +/** + * 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.fs.StorageType; +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.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.apache.hadoop.hdfs.protocol.ExtendedBlock; +import org.apache.hadoop.hdfs.protocol.HdfsConstants; +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; + +/** + * Make sure we correctly update the quota usage with the striped blocks. + */ +public class TestQuotaWithStripedBlocks { + private static final int BLOCK_SIZE = 1024 * 1024; + private static final long DISK_QUOTA = BLOCK_SIZE * 10; + private static final ErasureCodingPolicy ecPolicy = + ErasureCodingPolicyManager.getSystemDefaultPolicy(); + private static final int NUM_DATA_BLOCKS = ecPolicy.getNumDataUnits(); + private static final int NUM_PARITY_BLOCKS = ecPolicy.getNumParityUnits(); + private static final int GROUP_SIZE = NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS; + private static final Path ecDir = new Path("/ec"); + + private MiniDFSCluster cluster; + private FSDirectory dir; + private DistributedFileSystem dfs; + + @Before + public void setUp() throws IOException { + final Configuration conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, BLOCK_SIZE); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE).build(); + cluster.waitActive(); + + dir = cluster.getNamesystem().getFSDirectory(); + dfs = cluster.getFileSystem(); + + dfs.mkdirs(ecDir); + dfs.getClient().setErasureCodingPolicy(ecDir.toString(), ecPolicy); + dfs.setQuota(ecDir, Long.MAX_VALUE - 1, DISK_QUOTA); + dfs.setQuotaByStorageType(ecDir, StorageType.DISK, DISK_QUOTA); + dfs.setStoragePolicy(ecDir, HdfsConstants.HOT_STORAGE_POLICY_NAME); + } + + @After + public void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test + public void testUpdatingQuotaCount() throws Exception { + final Path file = new Path(ecDir, "file"); + FSDataOutputStream out = null; + + try { + out = dfs.create(file, (short) 1); + + INodeFile fileNode = dir.getINode4Write(file.toString()).asFile(); + ExtendedBlock previous = null; + // Create striped blocks which have a cell in each block. + Block newBlock = DFSTestUtil.addStripedBlockToFile(cluster.getDataNodes(), + dfs, cluster.getNamesystem(), file.toString(), fileNode, + dfs.getClient().getClientName(), previous, 1); + previous = new ExtendedBlock(cluster.getNamesystem().getBlockPoolId(), + newBlock); + + final INodeDirectory dirNode = dir.getINode4Write(ecDir.toString()) + .asDirectory(); + final long spaceUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getStorageSpace(); + final long diskUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK); + // When we add a new block we update the quota using the full block size. + Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, spaceUsed); + Assert.assertEquals(BLOCK_SIZE * GROUP_SIZE, diskUsed); + + dfs.getClient().getNamenode().complete(file.toString(), + dfs.getClient().getClientName(), previous, fileNode.getId()); + + final long actualSpaceUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getStorageSpace(); + final long actualDiskUsed = dirNode.getDirectoryWithQuotaFeature() + .getSpaceConsumed().getTypeSpaces().get(StorageType.DISK); + // In this case the file's real size is cell size * block group size. + Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, + actualSpaceUsed); + Assert.assertEquals(StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE * GROUP_SIZE, + actualDiskUsed); + } finally { + IOUtils.cleanup(null, out); + } + } +} 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 0000000000000..6774aedba3329 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestRecoverStripedBlocks.java @@ -0,0 +1,169 @@ +/** + * 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.Path; +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.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.DatanodeStorageInfo; +import org.apache.hadoop.hdfs.server.datanode.DataNode; +import org.apache.hadoop.hdfs.server.protocol.BlockECRecoveryCommand.BlockECRecoveryInfo; +import org.junit.Test; +import java.util.List; + +import static org.apache.hadoop.hdfs.StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_DATA_BLOCKS; +import static org.apache.hadoop.hdfs.StripedFileTestUtil.NUM_PARITY_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 = + (short) (NUM_DATA_BLOCKS + NUM_PARITY_BLOCKS); + private MiniDFSCluster cluster; + private final Path dirPath = new Path("/dir"); + private Path filePath = new Path(dirPath, "file"); + private int maxReplicationStreams = + DFSConfigKeys.DFS_NAMENODE_REPLICATION_MAX_STREAMS_DEFAULT; + + private void initConf(Configuration conf) { + // 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); + } + + @Test + public void testMissingStripedBlock() throws Exception { + doTestMissingStripedBlock(1, 0); + } + + @Test + public void testMissingStripedBlockWithBusyNode1() throws Exception { + doTestMissingStripedBlock(2, 1); + } + + @Test + public void testMissingStripedBlockWithBusyNode2() throws Exception { + doTestMissingStripedBlock(3, 1); + } + + /** + * Start GROUP_SIZE + 1 datanodes. + * Inject striped blocks to first GROUP_SIZE datanodes. + * Then make numOfBusy datanodes busy, make numOfMissed datanodes missed. + * Then trigger BlockManager to compute recovery works. (so all recovery work + * will be scheduled to the last datanode) + * Finally, verify the recovery work of the last datanode. + */ + private void doTestMissingStripedBlock(int numOfMissed, int numOfBusy) + throws Exception { + Configuration conf = new HdfsConfiguration(); + initConf(conf); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE + 1) + .build(); + + try { + cluster.waitActive(); + final int numBlocks = 4; + DFSTestUtil.createStripedFile(cluster, filePath, + dirPath, numBlocks, 1, true); + // all blocks will be located at first GROUP_SIZE DNs, the last DN is + // empty because of the util function createStripedFile + + // make sure the file is complete in NN + final INodeFile fileNode = cluster.getNamesystem().getFSDirectory() + .getINode4Write(filePath.toString()).asFile(); + assertFalse(fileNode.isUnderConstruction()); + assertTrue(fileNode.isStriped()); + BlockInfo[] blocks = fileNode.getBlocks(); + assertEquals(numBlocks, blocks.length); + for (BlockInfo blk : blocks) { + assertTrue(blk.isStriped()); + assertTrue(blk.isComplete()); + assertEquals(BLOCK_STRIPED_CELL_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); + + // make numOfBusy nodes busy + int i = 0; + for (; i < numOfBusy; i++) { + DatanodeDescriptor busyNode = storageInfos[i].getDatanodeDescriptor(); + for (int j = 0; j < maxReplicationStreams + 1; j++) { + BlockManagerTestUtil.addBlockToBeReplicated(busyNode, new Block(j), + new DatanodeStorageInfo[]{storageInfos[0]}); + } + } + + // make numOfMissed internal blocks missed + for (; i < numOfBusy + numOfMissed; i++) { + DatanodeDescriptor missedNode = storageInfos[i].getDatanodeDescriptor(); + assertEquals(numBlocks, missedNode.numBlocks()); + bm.getDatanodeManager().removeDatanode(missedNode); + } + + 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 recovery = + last.getErasureCodeCommand(numBlocks); + for (BlockECRecoveryInfo info : recovery) { + assertEquals(1, info.getTargetDnInfos().length); + assertEquals(last, info.getTargetDnInfos()[0]); + assertEquals(info.getSourceDnInfos().length, + info.getLiveBlockIndices().length); + if (GROUP_SIZE - numOfMissed == NUM_DATA_BLOCKS) { + // It's a QUEUE_HIGHEST_PRIORITY block, so the busy DNs will be chosen + // to make sure we have NUM_DATA_BLOCKS DNs to do recovery work. + assertEquals(NUM_DATA_BLOCKS, info.getSourceDnInfos().length); + } else { + // The block has no highest priority, so we don't use the busy DNs as + // sources + assertEquals(GROUP_SIZE - numOfMissed - numOfBusy, + info.getSourceDnInfos().length); + } + } + } finally { + cluster.shutdown(); + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java new file mode 100644 index 0000000000000..9f7ea3e7bc02f --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/TestStripedINodeFile.java @@ -0,0 +1,285 @@ +/** + * 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 static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; +import static org.junit.Assert.assertFalse; + +import java.io.IOException; + +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.permission.FsPermission; +import org.apache.hadoop.fs.permission.PermissionStatus; +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.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.Block; +import org.apache.hadoop.hdfs.protocol.BlockStoragePolicy; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +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.BlockStoragePolicySuite; +import org.apache.hadoop.hdfs.server.common.HdfsServerConstants; + +import org.junit.Test; + +/** + * This class tests INodeFile with striped feature. + */ +public class TestStripedINodeFile { + public static final Log LOG = LogFactory.getLog(TestINodeFile.class); + + private static final PermissionStatus perm = new PermissionStatus( + "userName", null, FsPermission.getDefault()); + + private final BlockStoragePolicySuite defaultSuite = + BlockStoragePolicySuite.createDefaultSuite(); + private final BlockStoragePolicy defaultPolicy = + defaultSuite.getDefaultPolicy(); + + private static final ErasureCodingPolicy testECPolicy + = ErasureCodingPolicyManager.getSystemDefaultPolicy(); + + private static INodeFile createStripedINodeFile() { + return new INodeFile(HdfsConstants.GRANDFATHER_INODE_ID, null, perm, 0L, 0L, + null, (short)0, 1024L, HdfsConstants.COLD_STORAGE_POLICY_ID, true); + } + + @Test + public void testBlockStripedFeature() + throws IOException, InterruptedException{ + INodeFile inf = createStripedINodeFile(); + assertTrue(inf.isStriped()); + } + + @Test + public void testBlockStripedTotalBlockCount() { + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, testECPolicy); + assertEquals(9, blockInfoStriped.getTotalBlockNum()); + } + + @Test + public void testBlockStripedLength() + throws IOException, InterruptedException { + INodeFile inf = createStripedINodeFile(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, testECPolicy); + inf.addBlock(blockInfoStriped); + assertEquals(1, inf.getBlocks().length); + } + + @Test + public void testBlockStripedConsumedSpace() + throws IOException, InterruptedException { + INodeFile inf = createStripedINodeFile(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, testECPolicy); + blockInfoStriped.setNumBytes(1); + inf.addBlock(blockInfoStriped); + // 0. Calculate the total bytes per stripes + // 1. Calculate the number of stripes in this block group. + // 2. Calculate the last remaining length which does not make a stripe. + // 3. Total consumed space is the total of + // a. The total of the full cells of data blocks and parity blocks. + // b. The remaining of data block which does not make a stripe. + // c. The last parity block cells. These size should be same + // to the first cell in this stripe. + // So the total consumed space is the sum of + // a. * ( - 1) * = 0 + // b. % = 1 + // c. * = 1 * 3 + assertEquals(4, inf.storagespaceConsumedStriped().getStorageSpace()); + assertEquals(4, inf.storagespaceConsumed(defaultPolicy).getStorageSpace()); + } + + @Test + public void testMultipleBlockStripedConsumedSpace() + throws IOException, InterruptedException { + INodeFile inf = createStripedINodeFile(); + Block blk1 = new Block(1); + BlockInfoStriped blockInfoStriped1 + = new BlockInfoStriped(blk1, testECPolicy); + blockInfoStriped1.setNumBytes(1); + Block blk2 = new Block(2); + BlockInfoStriped blockInfoStriped2 + = new BlockInfoStriped(blk2, testECPolicy); + blockInfoStriped2.setNumBytes(1); + inf.addBlock(blockInfoStriped1); + inf.addBlock(blockInfoStriped2); + // This is the double size of one block in above case. + assertEquals(4 * 2, inf.storagespaceConsumedStriped().getStorageSpace()); + assertEquals(4 * 2, inf.storagespaceConsumed(defaultPolicy).getStorageSpace()); + } + + @Test + public void testBlockStripedFileSize() + throws IOException, InterruptedException { + INodeFile inf = createStripedINodeFile(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, testECPolicy); + blockInfoStriped.setNumBytes(100); + inf.addBlock(blockInfoStriped); + // Compute file size should return actual data + // size which is retained by this file. + assertEquals(100, inf.computeFileSize()); + assertEquals(100, inf.computeFileSize(false, false)); + } + + @Test + public void testBlockStripedUCFileSize() + throws IOException, InterruptedException { + INodeFile inf = createStripedINodeFile(); + Block blk = new Block(1); + BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy); + bInfoUCStriped.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); + bInfoUCStriped.setNumBytes(100); + inf.addBlock(bInfoUCStriped); + assertEquals(100, inf.computeFileSize()); + assertEquals(0, inf.computeFileSize(false, false)); + } + + @Test + public void testBlockStripedComputeQuotaUsage() + throws IOException, InterruptedException { + INodeFile inf = createStripedINodeFile(); + Block blk = new Block(1); + BlockInfoStriped blockInfoStriped + = new BlockInfoStriped(blk, testECPolicy); + blockInfoStriped.setNumBytes(100); + inf.addBlock(blockInfoStriped); + + QuotaCounts counts = + inf.computeQuotaUsageWithStriped(defaultPolicy, + new QuotaCounts.Builder().build()); + assertEquals(1, counts.getNameSpace()); + // The total consumed space is the sum of + // a. * ( - 1) * = 0 + // b. % = 100 + // c. * = 100 * 3 + assertEquals(400, counts.getStorageSpace()); + } + + @Test + public void testBlockStripedUCComputeQuotaUsage() + throws IOException, InterruptedException { + INodeFile inf = createStripedINodeFile(); + Block blk = new Block(1); + BlockInfoStriped bInfoUCStriped = new BlockInfoStriped(blk, testECPolicy); + bInfoUCStriped.convertToBlockUnderConstruction( + HdfsServerConstants.BlockUCState.UNDER_CONSTRUCTION, null); + bInfoUCStriped.setNumBytes(100); + inf.addBlock(bInfoUCStriped); + + QuotaCounts counts + = inf.computeQuotaUsageWithStriped(defaultPolicy, + new QuotaCounts.Builder().build()); + assertEquals(1024, inf.getPreferredBlockSize()); + assertEquals(1, counts.getNameSpace()); + // Consumed space in the case of BlockInfoStripedUC can be calculated + // by using preferred block size. This is 1024 and total block num + // is 9(= 3 + 6). Consumed storage space should be 1024 * 9 = 9216. + assertEquals(9216, counts.getStorageSpace()); + } + + /** + * Test the behavior of striped and contiguous block deletions. + */ + @Test(timeout = 60000) + public void testDeleteOp() throws Exception { + MiniDFSCluster cluster = null; + try { + final int len = 1024; + final Path parentDir = new Path("/parentDir"); + final Path ecDir = new Path(parentDir, "ecDir"); + final Path ecFile = new Path(ecDir, "ecFile"); + final Path contiguousFile = new Path(parentDir, "someFile"); + final DistributedFileSystem dfs; + final Configuration conf = new Configuration(); + final short GROUP_SIZE = (short) (StripedFileTestUtil.NUM_DATA_BLOCKS + + StripedFileTestUtil.NUM_PARITY_BLOCKS); + conf.setInt(DFSConfigKeys.DFS_NAMENODE_MAX_XATTRS_PER_INODE_KEY, 2); + + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(GROUP_SIZE) + .build(); + cluster.waitActive(); + + FSNamesystem fsn = cluster.getNamesystem(); + dfs = cluster.getFileSystem(); + dfs.mkdirs(ecDir); + + // set erasure coding policy + dfs.setErasureCodingPolicy(ecDir, null); + DFSTestUtil.createFile(dfs, ecFile, len, (short) 1, 0xFEED); + DFSTestUtil.createFile(dfs, contiguousFile, len, (short) 1, 0xFEED); + final FSDirectory fsd = fsn.getFSDirectory(); + + // Case-1: Verify the behavior of striped blocks + // Get blocks of striped file + INode inodeStriped = fsd.getINode("/parentDir/ecDir/ecFile"); + assertTrue("Failed to get INodeFile for /parentDir/ecDir/ecFile", + inodeStriped instanceof INodeFile); + INodeFile inodeStripedFile = (INodeFile) inodeStriped; + BlockInfo[] stripedBlks = inodeStripedFile.getBlocks(); + for (BlockInfo blockInfo : stripedBlks) { + assertFalse("Mistakenly marked the block as deleted!", + blockInfo.isDeleted()); + } + + // delete directory with erasure coding policy + dfs.delete(ecDir, true); + for (BlockInfo blockInfo : stripedBlks) { + assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted()); + } + + // Case-2: Verify the behavior of contiguous blocks + // Get blocks of contiguous file + INode inode = fsd.getINode("/parentDir/someFile"); + assertTrue("Failed to get INodeFile for /parentDir/someFile", + inode instanceof INodeFile); + INodeFile inodeFile = (INodeFile) inode; + BlockInfo[] contiguousBlks = inodeFile.getBlocks(); + for (BlockInfo blockInfo : contiguousBlks) { + assertFalse("Mistakenly marked the block as deleted!", + blockInfo.isDeleted()); + } + + // delete parent directory + dfs.delete(parentDir, true); + for (BlockInfo blockInfo : contiguousBlks) { + assertTrue("Didn't mark the block as deleted!", blockInfo.isDeleted()); + } + } finally { + if (cluster != null) { + cluster.shutdown(); + } + } + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java index 4af9c75f776f6..f3221b19c02d2 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/ha/TestRetryCacheWithHA.java @@ -73,6 +73,7 @@ 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.DatanodeManager; import org.apache.hadoop.hdfs.server.namenode.FSNamesystem; import org.apache.hadoop.hdfs.server.namenode.INodeFile; import org.apache.hadoop.hdfs.server.namenode.snapshot.SnapshotTestHelper; @@ -736,7 +737,13 @@ void invoke() throws Exception { DatanodeInfo[] newNodes = new DatanodeInfo[2]; newNodes[0] = nodes[0]; newNodes[1] = nodes[1]; - String[] storageIDs = {"s0", "s1"}; + final DatanodeManager dm = cluster.getNamesystem(0).getBlockManager() + .getDatanodeManager(); + final String storageID1 = dm.getDatanode(newNodes[0]).getStorageInfos()[0] + .getStorageID(); + final String storageID2 = dm.getDatanode(newNodes[1]).getStorageInfos()[0] + .getStorageID(); + String[] storageIDs = {storageID1, storageID2}; client.getNamenode().updatePipeline(client.getClientName(), oldBlock, newBlock, newNodes, storageIDs); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java index 29d227294e58c..8b19b6d937166 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/server/namenode/snapshot/SnapshotTestHelper.java @@ -175,7 +175,7 @@ public static void checkSnapshotCreation(DistributedFileSystem hdfs, * localName (className@hashCode) parent permission group user * * Specific information for different types of INode: - * {@link INodeDirectory}:childrenSize + * {@link INodeDirectory}:childrenSize * {@link INodeFile}: fileSize, block list. Check {@link BlockInfo#toString()} * and {@link BlockUnderConstructionFeature#toString()} for detailed information. * diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java new file mode 100644 index 0000000000000..6d4d79734e9a6 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/tools/offlineImageViewer/TestOfflineImageViewerWithStripedBlocks.java @@ -0,0 +1,162 @@ +/** + * 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.tools.offlineImageViewer; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertTrue; + +import java.io.File; +import java.io.IOException; + +import org.apache.hadoop.conf.Configuration; +import org.apache.hadoop.fs.FSDataOutputStream; +import org.apache.hadoop.fs.Path; +import org.apache.hadoop.fs.UnresolvedLinkException; +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.StripedFileTestUtil; +import org.apache.hadoop.hdfs.protocol.HdfsConstants.SafeModeAction; +import org.apache.hadoop.hdfs.protocol.SnapshotAccessControlException; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfo; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockInfoStriped; +import org.apache.hadoop.hdfs.server.namenode.FSDirectory; +import org.apache.hadoop.hdfs.server.namenode.FSImageTestUtil; +import org.apache.hadoop.hdfs.server.namenode.INodeFile; +import org.junit.AfterClass; +import org.junit.BeforeClass; +import org.junit.Test; + +public class TestOfflineImageViewerWithStripedBlocks { + private static int dataBlocks = StripedFileTestUtil.NUM_DATA_BLOCKS; + private static int parityBlocks = StripedFileTestUtil.NUM_PARITY_BLOCKS; + + private static MiniDFSCluster cluster; + private static DistributedFileSystem fs; + private static final int cellSize = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private static final int stripesPerBlock = 3; + private static final int blockSize = cellSize * stripesPerBlock; + + @BeforeClass + public static void setup() throws IOException { + int numDNs = dataBlocks + parityBlocks + 2; + Configuration conf = new Configuration(); + conf.setLong(DFSConfigKeys.DFS_BLOCK_SIZE_KEY, blockSize); + cluster = new MiniDFSCluster.Builder(conf).numDataNodes(numDNs).build(); + cluster.waitActive(); + cluster.getFileSystem().getClient().setErasureCodingPolicy("/", null); + fs = cluster.getFileSystem(); + Path eczone = new Path("/eczone"); + fs.mkdirs(eczone); + } + + @AfterClass + public static void tearDown() { + if (cluster != null) { + cluster.shutdown(); + } + } + + @Test(timeout = 60000) + public void testFileEqualToOneStripe() throws Exception { + int numBytes = cellSize; + testFileSize(numBytes); + } + + @Test(timeout = 60000) + public void testFileLessThanOneStripe() throws Exception { + int numBytes = cellSize - 100; + testFileSize(numBytes); + } + + @Test(timeout = 60000) + public void testFileHavingMultipleBlocks() throws Exception { + int numBytes = blockSize * 3; + testFileSize(numBytes); + } + + @Test(timeout = 60000) + public void testFileLargerThanABlockGroup1() throws IOException { + testFileSize(blockSize * dataBlocks + cellSize + 123); + } + + @Test(timeout = 60000) + public void testFileLargerThanABlockGroup2() throws IOException { + testFileSize(blockSize * dataBlocks * 3 + cellSize * dataBlocks + cellSize + + 123); + } + + @Test(timeout = 60000) + public void testFileFullBlockGroup() throws IOException { + testFileSize(blockSize * dataBlocks); + } + + @Test(timeout = 60000) + public void testFileMoreThanOneStripe() throws Exception { + int numBytes = blockSize + blockSize / 2; + testFileSize(numBytes); + } + + private void testFileSize(int numBytes) throws IOException, + UnresolvedLinkException, SnapshotAccessControlException { + fs.setSafeMode(SafeModeAction.SAFEMODE_LEAVE); + File orgFsimage = null; + Path file = new Path("/eczone/striped"); + FSDataOutputStream out = fs.create(file, true); + byte[] bytes = DFSTestUtil.generateSequentialBytes(0, numBytes); + out.write(bytes); + out.close(); + + // Write results to the fsimage file + fs.setSafeMode(SafeModeAction.SAFEMODE_ENTER, false); + fs.saveNamespace(); + + // Determine location of fsimage file + orgFsimage = FSImageTestUtil.findLatestImageFile(FSImageTestUtil + .getFSImage(cluster.getNameNode()).getStorage().getStorageDir(0)); + if (orgFsimage == null) { + throw new RuntimeException("Didn't generate or can't find fsimage"); + } + FSImageLoader loader = FSImageLoader.load(orgFsimage.getAbsolutePath()); + String fileStatus = loader.getFileStatus("/eczone/striped"); + long expectedFileSize = bytes.length; + + // Verify space consumed present in BlockInfoStriped + FSDirectory fsdir = cluster.getNamesystem().getFSDirectory(); + INodeFile fileNode = fsdir.getINode4Write(file.toString()).asFile(); + assertTrue("Invalid block size", fileNode.getBlocks().length > 0); + long actualFileSize = 0; + for (BlockInfo blockInfo : fileNode.getBlocks()) { + assertTrue("Didn't find block striped information", + blockInfo instanceof BlockInfoStriped); + actualFileSize += blockInfo.getNumBytes(); + } + + assertEquals("Wrongly computed file size contains striped blocks", + expectedFileSize, actualFileSize); + + // Verify space consumed present in filestatus + String EXPECTED_FILE_SIZE = "\"length\":" + + String.valueOf(expectedFileSize); + assertTrue( + "Wrongly computed file size contains striped blocks, file status:" + + fileStatus + ". Expected file size is : " + EXPECTED_FILE_SIZE, + fileStatus.contains(EXPECTED_FILE_SIZE)); + } +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java new file mode 100644 index 0000000000000..95b0135720f91 --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/util/TestStripedBlockUtil.java @@ -0,0 +1,279 @@ +/** + * 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.util; + +import com.google.common.base.Preconditions; +import org.apache.hadoop.fs.StorageType; +import org.apache.hadoop.hdfs.DFSTestUtil; +import org.apache.hadoop.hdfs.StripedFileTestUtil; +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.LocatedStripedBlock; +import org.apache.hadoop.hdfs.server.blockmanagement.BlockIdManager; +import static org.apache.hadoop.hdfs.util.StripedBlockUtil.*; + +import org.apache.hadoop.hdfs.server.namenode.ErasureCodingPolicyManager; +import org.apache.hadoop.hdfs.protocol.ErasureCodingPolicy; +import org.junit.Before; +import org.junit.Test; + +import java.util.Random; + +import static org.junit.Assert.assertEquals; +import static org.junit.Assert.assertFalse; + +/** + * Need to cover the following combinations: + * 1. Block group size: + * 1.1 One byte + * 1.2 Smaller than cell + * 1.3 One full cell + * 1.4 x full cells, where x is smaller than number of data blocks + * 1.5 x full cells plus a partial cell + * 1.6 One full stripe + * 1.7 One full stripe plus a partial cell + * 1.8 One full stripe plus x full cells + * 1.9 One full stripe plus x full cells plus a partial cell + * 1.10 y full stripes, but smaller than full block group size + * 1.11 Full block group size + * + * 2. Byte range start + * 2.1 Zero + * 2.2 Within first cell + * 2.3 End of first cell + * 2.4 Start of a middle* cell in the first stripe (* neither first or last) + * 2.5 End of middle cell in the first stripe + * 2.6 Within a middle cell in the first stripe + * 2.7 Start of the last cell in the first stripe + * 2.8 Within the last cell in the first stripe + * 2.9 End of the last cell in the first stripe + * 2.10 Start of a middle stripe + * 2.11 Within a middle stripe + * 2.12 End of a middle stripe + * 2.13 Start of the last stripe + * 2.14 Within the last stripe + * 2.15 End of the last stripe (last byte) + * + * 3. Byte range length: same settings as block group size + * + * We should test in total 11 x 15 x 11 = 1815 combinations + * TODO: test parity block logic + */ +public class TestStripedBlockUtil { + private final short DATA_BLK_NUM = StripedFileTestUtil.NUM_DATA_BLOCKS; + private final short PARITY_BLK_NUM = StripedFileTestUtil.NUM_PARITY_BLOCKS; + private final short BLK_GROUP_WIDTH = (short) (DATA_BLK_NUM + PARITY_BLK_NUM); + private final int CELLSIZE = StripedFileTestUtil.BLOCK_STRIPED_CELL_SIZE; + private final int FULL_STRIPE_SIZE = DATA_BLK_NUM * CELLSIZE; + /** number of full stripes in a full block group */ + private final int BLK_GROUP_STRIPE_NUM = 16; + private final ErasureCodingPolicy ECPOLICY = ErasureCodingPolicyManager. + getSystemDefaultPolicy(); + private final Random random = new Random(); + + private int[] blockGroupSizes; + private int[] byteRangeStartOffsets; + private int[] byteRangeSizes; + + @Before + public void setup(){ + blockGroupSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE, + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE, + BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE}; + byteRangeStartOffsets = new int[] {0, getDelta(CELLSIZE), CELLSIZE - 1}; + byteRangeSizes = new int[]{1, getDelta(CELLSIZE), CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE, + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE, FULL_STRIPE_SIZE + getDelta(CELLSIZE), + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE, + FULL_STRIPE_SIZE + getDelta(DATA_BLK_NUM) * CELLSIZE + getDelta(CELLSIZE), + getDelta(BLK_GROUP_STRIPE_NUM) * FULL_STRIPE_SIZE, + BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE}; + } + + private int getDelta(int size) { + return 1 + random.nextInt(size - 2); + } + private byte hashIntToByte(int i) { + int BYTE_MASK = 0xff; + return (byte) (((i + 13) * 29) & BYTE_MASK); + } + + private LocatedStripedBlock createDummyLocatedBlock(int bgSize) { + final long blockGroupID = -1048576; + DatanodeInfo[] locs = new DatanodeInfo[BLK_GROUP_WIDTH]; + String[] storageIDs = new String[BLK_GROUP_WIDTH]; + StorageType[] storageTypes = new StorageType[BLK_GROUP_WIDTH]; + int[] indices = new int[BLK_GROUP_WIDTH]; + for (int i = 0; i < BLK_GROUP_WIDTH; i++) { + indices[i] = (i + 2) % DATA_BLK_NUM; + // Location port always equal to logical index of a block, + // for easier verification + locs[i] = DFSTestUtil.getLocalDatanodeInfo(indices[i]); + storageIDs[i] = locs[i].getDatanodeUuid(); + storageTypes[i] = StorageType.DISK; + } + return new LocatedStripedBlock(new ExtendedBlock("pool", blockGroupID, + bgSize, 1001), locs, storageIDs, storageTypes, indices, 0, false, + null); + } + + private byte[][] createInternalBlkBuffers(int bgSize) { + byte[][] bufs = new byte[DATA_BLK_NUM + PARITY_BLK_NUM][]; + int[] pos = new int[DATA_BLK_NUM + PARITY_BLK_NUM]; + for (int i = 0; i < DATA_BLK_NUM + PARITY_BLK_NUM; i++) { + int bufSize = (int) getInternalBlockLength( + bgSize, CELLSIZE, DATA_BLK_NUM, i); + bufs[i] = new byte[bufSize]; + pos[i] = 0; + } + int done = 0; + while (done < bgSize) { + Preconditions.checkState(done % CELLSIZE == 0); + StripingCell cell = new StripingCell(ECPOLICY, CELLSIZE, done / CELLSIZE, 0); + int idxInStripe = cell.idxInStripe; + int size = Math.min(CELLSIZE, bgSize - done); + for (int i = 0; i < size; i++) { + bufs[idxInStripe][pos[idxInStripe] + i] = hashIntToByte(done + i); + } + done += size; + pos[idxInStripe] += size; + } + + return bufs; + } + + @Test + public void testParseDummyStripedBlock() { + LocatedStripedBlock lsb = createDummyLocatedBlock( + BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE); + LocatedBlock[] blocks = parseStripedBlockGroup( + lsb, CELLSIZE, DATA_BLK_NUM, PARITY_BLK_NUM); + assertEquals(DATA_BLK_NUM + PARITY_BLK_NUM, blocks.length); + for (int i = 0; i < DATA_BLK_NUM; i++) { + assertFalse(blocks[i].isStriped()); + assertEquals(i, + BlockIdManager.getBlockIndex(blocks[i].getBlock().getLocalBlock())); + assertEquals(0, blocks[i].getStartOffset()); + assertEquals(1, blocks[i].getLocations().length); + assertEquals(i, blocks[i].getLocations()[0].getIpcPort()); + assertEquals(i, blocks[i].getLocations()[0].getXferPort()); + } + } + + private void verifyInternalBlocks (int numBytesInGroup, int[] expected) { + for (int i = 1; i < BLK_GROUP_WIDTH; i++) { + assertEquals(expected[i], + getInternalBlockLength(numBytesInGroup, CELLSIZE, DATA_BLK_NUM, i)); + } + } + + @Test + public void testGetInternalBlockLength () { + // A small delta that is smaller than a cell + final int delta = 10; + + // Block group is smaller than a cell + verifyInternalBlocks(CELLSIZE - delta, + new int[] {CELLSIZE - delta, 0, 0, 0, 0, 0, + CELLSIZE - delta, CELLSIZE - delta, CELLSIZE - delta}); + + // Block group is exactly as large as a cell + verifyInternalBlocks(CELLSIZE, + new int[] {CELLSIZE, 0, 0, 0, 0, 0, + CELLSIZE, CELLSIZE, CELLSIZE}); + + // Block group is a little larger than a cell + verifyInternalBlocks(CELLSIZE + delta, + new int[] {CELLSIZE, delta, 0, 0, 0, 0, + CELLSIZE, CELLSIZE, CELLSIZE}); + + // Block group contains multiple stripes and ends at stripe boundary + verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE, + new int[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE}); + + // Block group contains multiple stripes and ends at cell boundary + // (not ending at stripe boundary) + verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE + CELLSIZE, + new int[] {3 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 3 * CELLSIZE, 3 * CELLSIZE, 3 * CELLSIZE}); + + // Block group contains multiple stripes and doesn't end at cell boundary + verifyInternalBlocks(2 * DATA_BLK_NUM * CELLSIZE - delta, + new int[] {2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE - delta, + 2 * CELLSIZE, 2 * CELLSIZE, 2 * CELLSIZE}); + } + + /** + * Test dividing a byte range into aligned stripes and verify the aligned + * ranges can be translated back to the byte range. + */ + @Test + public void testDivideByteRangeIntoStripes() { + byte[] assembled = new byte[BLK_GROUP_STRIPE_NUM * FULL_STRIPE_SIZE]; + for (int bgSize : blockGroupSizes) { + LocatedStripedBlock blockGroup = createDummyLocatedBlock(bgSize); + byte[][] internalBlkBufs = createInternalBlkBuffers(bgSize); + for (int brStart : byteRangeStartOffsets) { + for (int brSize : byteRangeSizes) { + if (brStart + brSize > bgSize) { + continue; + } + AlignedStripe[] stripes = divideByteRangeIntoStripes(ECPOLICY, + CELLSIZE, blockGroup, brStart, brStart + brSize - 1, assembled, 0); + + for (AlignedStripe stripe : stripes) { + for (int i = 0; i < DATA_BLK_NUM; i++) { + StripingChunk chunk = stripe.chunks[i]; + if (chunk == null || chunk.state != StripingChunk.REQUESTED) { + continue; + } + int done = 0; + for (int j = 0; j < chunk.byteArray.getLengths().length; j++) { + System.arraycopy(internalBlkBufs[i], + (int) stripe.getOffsetInBlock() + done, assembled, + chunk.byteArray.getOffsets()[j], + chunk.byteArray.getLengths()[j]); + done += chunk.byteArray.getLengths()[j]; + } + } + } + for (int i = 0; i < brSize; i++) { + if (hashIntToByte(brStart + i) != assembled[i]) { + System.out.println("Oops"); + } + assertEquals("Byte at " + (brStart + i) + " should be the same", + hashIntToByte(brStart + i), assembled[i]); + } + } + } + } + } + +} diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java index 391f1903f96c1..8947c5b49a16d 100644 --- a/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/java/org/apache/hadoop/hdfs/web/TestJsonUtil.java @@ -65,7 +65,7 @@ public void testHdfsFileStatus() throws IOException { final HdfsFileStatus status = new HdfsFileStatus(1001L, false, 3, 1L << 26, now, now + 10, new FsPermission((short) 0644), "user", "group", DFSUtil.string2Bytes("bar"), DFSUtil.string2Bytes("foo"), - HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0); + HdfsConstants.GRANDFATHER_INODE_ID, 0, null, (byte) 0, null); final FileStatus fstatus = toFileStatus(status, parent); System.out.println("status = " + status); System.out.println("fstatus = " + fstatus); diff --git a/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml new file mode 100644 index 0000000000000..e72d862264e0c --- /dev/null +++ b/hadoop-hdfs-project/hadoop-hdfs/src/test/resources/testErasureCodingConf.xml @@ -0,0 +1,377 @@ + + + + + + + + test + + + + + + + help: help for erasure coding command + + -help + + + + + + SubstringComparator + Usage: hdfs erasurecode [generic options] + + + + + + help: setPolicy command + + -fs NAMENODE -help setPolicy + + + + + + RegexpComparator + ^[ \t]*Set a specified erasure coding policy to a directory( )* + + + RegexpComparator + ^-setPolicy \[-p <policyName>\] <path>(.)* + + + + + + help: getPolicy command + + -fs NAMENODE -help getPolicy + + + + + + SubstringComparator + Get erasure coding policy information about at specified path + + + RegexpComparator + ^-getPolicy <path>(.)* + + + + + + help: listPolicies command + + -fs NAMENODE -help listPolicies + + + + + + SubstringComparator + Get the list of erasure coding policies supported + + + RegexpComparator + ^-listPolicies (.)* + + + + + + + setPolicy : set erasure coding policy on a directory to encode files + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir + + + -fs NAMENODE -rmdir /ecdir + + + + SubstringComparator + EC policy set successfully at NAMENODE/ecdir + + + + + + setPolicy : set a policy twice + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy /ecdir + -fs NAMENODE -setPolicy /ecdir + + + -fs NAMENODE -rmdir /ecdir + + + + SubstringComparator + Directory /ecdir already has an erasure coding policy + + + + + + setPolicy : default policy + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy /ecdir + -fs NAMENODE -getPolicy /ecdir + + + -fs NAMENODE -rmdir /ecdir + + + + SubstringComparator + ErasureCodingPolicy=[Name=RS-6-3-64k + + + + + + getPolicy : get EC policy information at specified path, which doesn't have an EC policy + + -fs NAMENODE -mkdir /noec + -fs NAMENODE -getPolicy /noec + + + -fs NAMENODE -rmdir /noec + + + + SubstringComparator + Path NAMENODE/noec is not erasure coded + + + + + + getPolicy : get EC policy information at specified path, which doesn't have an EC policy + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir + -fs NAMENODE -getPolicy /ecdir + + + -fs NAMENODE -rmdir /ecdir + + + + SubstringComparator + ErasureCodingPolicy=[Name=RS-6-3-64k + + + + + + getPolicy : get EC policy information at specified path, which doesn't have an EC policy + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -p RS-6-3-64k /ecdir + -fs NAMENODE -touchz /ecdir/ecfile + -fs NAMENODE -getPolicy /ecdir/ecfile + + + -fs NAMENODE -rm /ecdir/ecfile + -fs NAMENODE -rmdir /ecdir + + + + SubstringComparator + ErasureCodingPolicy=[Name=RS-6-3-64k + + + + + + listPolicies : get the list of ECPolicies supported + + -fs NAMENODE -listPolicies + + + + + + SubstringComparator + RS-6-3 + + + + + + + setPolicy : illegal parameters - path is missing + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy + + + -fs NAMENODE -rmdir /ecdir + + + + RegexpComparator + ^-setPolicy: <path> is missing(.)* + + + + + + setPolicy : illegal parameters - policy name is missing + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -p + + + -fs NAMENODE -rmdir /ecdir + + + + RegexpComparator + ^-setPolicy: option -p requires 1 argument(.)* + + + + + + setPolicy : illegal parameters - too many arguments + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy /ecdir1 /ecdir2 + + + -fs NAMENODE -rmdir /ecdir + + + + SubstringComparator + -setPolicy: Too many arguments + + + + + + setPolicy : illegal parameters - invalidpolicy + + -fs NAMENODE -mkdir /ecdir + -fs NAMENODE -setPolicy -p invalidpolicy /ecdir + + + -fs NAMENODE -rmdir /ecdir + + + + SubstringComparator + Policy 'invalidpolicy' does not match any of the supported policies. Please select any one of [RS-6-3-64k] + + + + + + setPolicy : illegal parameters - no such file + + -fs NAMENODE -setPolicy /ecdir + + + + + + RegexpComparator + ^setPolicy: `/ecdir': No such file or directory(.)* + + + + + + getPolicy : illegal parameters - path is missing + + -fs NAMENODE -getPolicy + + + + + + RegexpComparator + ^-getPolicy: <path> is missing(.)* + + + + + + getPolicy : illegal parameters - too many arguments + + -fs NAMENODE -getPolicy /ecdir /ecdir + + + -fs NAMENODE -rm /ecdir + + + + SubstringComparator + -getPolicy: Too many arguments + + + + + + getPolicy : illegal parameters - no such file + + -fs NAMENODE -getPolicy /ecdir + + + + + + RegexpComparator + ^getPolicy: `/ecdir': No such file or directory(.)* + + + + + + listPolicies : illegal parameters - too many parameters + + -fs NAMENODE -listPolicies /ecdir + + + + + + SubstringComparator + -listPolicies: Too many parameters + + + + + + diff --git a/hadoop-project/src/site/site.xml b/hadoop-project/src/site/site.xml index 272ea7bbf3085..5c92d73a94af0 100644 --- a/hadoop-project/src/site/site.xml +++ b/hadoop-project/src/site/site.xml @@ -98,6 +98,7 @@ +