Revert "HDDS-629. Make ApplyTransaction calls in ContainerStateMachine idempotent. Contributed by Shashikant Banerjee."

This reverts commit 0473b68000.
This commit is contained in:
Jitendra Pandey 2018-10-13 12:14:39 -07:00
parent 0473b68000
commit 22f37af935
12 changed files with 328 additions and 208 deletions

View File

@ -113,7 +113,6 @@ public static Versioning getVersioning(boolean versioning) {
public static final String DELETING_KEY_PREFIX = "#deleting#";
public static final String DELETED_KEY_PREFIX = "#deleted#";
public static final String DELETE_TRANSACTION_KEY_PREFIX = "#delTX#";
public static final String BLOCK_COMMIT_SEQUENCE_ID_PREFIX = "#BCSID";
/**
* OM LevelDB prefixes.

View File

@ -42,8 +42,7 @@ public final class MetadataKeyFilters {
new MetadataKeyFilters.KeyPrefixFilter()
.addFilter(OzoneConsts.DELETING_KEY_PREFIX, true)
.addFilter(OzoneConsts.DELETED_KEY_PREFIX, true)
.addFilter(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX, true)
.addFilter(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX, true);
.addFilter(OzoneConsts.DELETE_TRANSACTION_KEY_PREFIX, true);
private MetadataKeyFilters() {
}

View File

@ -0,0 +1,205 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.helpers;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
/**
* Container Report iterates the closed containers and sends a container report
* to SCM.
*/
public class ContainerReport {
private static final int UNKNOWN = -1;
private final String finalhash;
private long size;
private long keyCount;
private long bytesUsed;
private long readCount;
private long writeCount;
private long readBytes;
private long writeBytes;
private long containerID;
public long getContainerID() {
return containerID;
}
public void setContainerID(long containerID) {
this.containerID = containerID;
}
/**
* Constructs the ContainerReport.
*
* @param containerID - Container ID.
* @param finalhash - Final Hash.
*/
public ContainerReport(long containerID, String finalhash) {
this.containerID = containerID;
this.finalhash = finalhash;
this.size = UNKNOWN;
this.keyCount = UNKNOWN;
this.bytesUsed = 0L;
this.readCount = 0L;
this.readBytes = 0L;
this.writeCount = 0L;
this.writeBytes = 0L;
}
/**
* Gets a containerReport from protobuf class.
*
* @param info - ContainerInfo.
* @return - ContainerReport.
*/
public static ContainerReport getFromProtoBuf(ContainerInfo info) {
Preconditions.checkNotNull(info);
ContainerReport report = new ContainerReport(info.getContainerID(),
info.getFinalhash());
if (info.hasSize()) {
report.setSize(info.getSize());
}
if (info.hasKeyCount()) {
report.setKeyCount(info.getKeyCount());
}
if (info.hasUsed()) {
report.setBytesUsed(info.getUsed());
}
if (info.hasReadCount()) {
report.setReadCount(info.getReadCount());
}
if (info.hasReadBytes()) {
report.setReadBytes(info.getReadBytes());
}
if (info.hasWriteCount()) {
report.setWriteCount(info.getWriteCount());
}
if (info.hasWriteBytes()) {
report.setWriteBytes(info.getWriteBytes());
}
report.setContainerID(info.getContainerID());
return report;
}
/**
* Returns the final signature for this container.
*
* @return - hash
*/
public String getFinalhash() {
return finalhash;
}
/**
* Returns a positive number it is a valid number, -1 if not known.
*
* @return size or -1
*/
public long getSize() {
return size;
}
/**
* Sets the size of the container on disk.
*
* @param size - int
*/
public void setSize(long size) {
this.size = size;
}
/**
* Gets number of keys in the container if known.
*
* @return - Number of keys or -1 for not known.
*/
public long getKeyCount() {
return keyCount;
}
/**
* Sets the key count.
*
* @param keyCount - Key Count
*/
public void setKeyCount(long keyCount) {
this.keyCount = keyCount;
}
public long getReadCount() {
return readCount;
}
public void setReadCount(long readCount) {
this.readCount = readCount;
}
public long getWriteCount() {
return writeCount;
}
public void setWriteCount(long writeCount) {
this.writeCount = writeCount;
}
public long getReadBytes() {
return readBytes;
}
public void setReadBytes(long readBytes) {
this.readBytes = readBytes;
}
public long getWriteBytes() {
return writeBytes;
}
public void setWriteBytes(long writeBytes) {
this.writeBytes = writeBytes;
}
public long getBytesUsed() {
return bytesUsed;
}
public void setBytesUsed(long bytesUsed) {
this.bytesUsed = bytesUsed;
}
/**
* Gets a containerInfo protobuf message from ContainerReports.
*
* @return ContainerInfo
*/
public ContainerInfo getProtoBufMessage() {
return ContainerInfo.newBuilder()
.setKeyCount(this.getKeyCount())
.setSize(this.getSize())
.setUsed(this.getBytesUsed())
.setReadCount(this.getReadCount())
.setReadBytes(this.getReadBytes())
.setWriteCount(this.getWriteCount())
.setWriteBytes(this.getWriteBytes())
.setFinalhash(this.getFinalhash())
.setContainerID(this.getContainerID())
.build();
}
}

View File

@ -0,0 +1,117 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
*
* http://www.apache.org/licenses/LICENSE-2.0
*
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS,
* WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
* See the License for the specific language governing permissions and
* limitations under the License.
*/
package org.apache.hadoop.ozone.container.common.helpers;
import com.google.common.base.Preconditions;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.ContainerInfo;
import static java.lang.Math.max;
/**
* KeyValueContainer Report iterates the closed containers and sends a
* container report to SCM.
*/
public class KeyValueContainerReport extends ContainerReport{
private long deleteTransactionId;
/**
* Constructs the KeyValueContainerReport.
*
* @param containerID - Container ID.
* @param finalhash - Final Hash.
*/
public KeyValueContainerReport(long containerID, String finalhash) {
super(containerID, finalhash);
this.deleteTransactionId = 0;
}
/**
* Sets the deleteTransactionId if it is greater than existing.
* @param transactionId - deleteTransactionId
*/
public void updateDeleteTransactionId(long transactionId) {
this.deleteTransactionId = max(transactionId, deleteTransactionId);
}
/**
* Gets the deleteTransactionId.
* @return - deleteTransactionId.
*/
public long getDeleteTransactionId() {
return this.deleteTransactionId;
}
/**
* Gets a containerReport from protobuf class.
*
* @param info - ContainerInfo.
* @return - ContainerReport.
*/
public static KeyValueContainerReport getFromProtoBuf(ContainerInfo info) {
Preconditions.checkNotNull(info);
KeyValueContainerReport report = new KeyValueContainerReport(
info.getContainerID(), info.getFinalhash());
if (info.hasSize()) {
report.setSize(info.getSize());
}
if (info.hasKeyCount()) {
report.setKeyCount(info.getKeyCount());
}
if (info.hasUsed()) {
report.setBytesUsed(info.getUsed());
}
if (info.hasReadCount()) {
report.setReadCount(info.getReadCount());
}
if (info.hasReadBytes()) {
report.setReadBytes(info.getReadBytes());
}
if (info.hasWriteCount()) {
report.setWriteCount(info.getWriteCount());
}
if (info.hasWriteBytes()) {
report.setWriteBytes(info.getWriteBytes());
}
if (info.hasDeleteTransactionId()) {
report.updateDeleteTransactionId(info.getDeleteTransactionId());
}
report.setContainerID(info.getContainerID());
return report;
}
/**
* Gets a containerInfo protobuf message from ContainerReports.
*
* @return ContainerInfo
*/
@Override
public ContainerInfo getProtoBufMessage() {
return ContainerInfo.newBuilder()
.setKeyCount(this.getKeyCount())
.setSize(this.getSize())
.setUsed(this.getBytesUsed())
.setReadCount(this.getReadCount())
.setReadBytes(this.getReadBytes())
.setWriteCount(this.getWriteCount())
.setWriteBytes(this.getWriteBytes())
.setFinalhash(this.getFinalhash())
.setContainerID(this.getContainerID())
.setDeleteTransactionId(this.getDeleteTransactionId())
.build();
}
}

View File

@ -132,9 +132,4 @@ void exportContainerData(OutputStream stream,
*/
StorageContainerDatanodeProtocolProtos.ContainerInfo getContainerReport()
throws StorageContainerException;
/**
* updates the blockCommitSequenceId.
*/
void updateBlockCommitSequenceId(long blockCommitSequenceId);
}

View File

@ -508,12 +508,6 @@ public File getContainerFile() {
.getContainerID() + OzoneConsts.CONTAINER_EXTENSION);
}
@Override
public void updateBlockCommitSequenceId(long blockCommitSequenceId) {
containerData.updateBlockCommitSequenceId(blockCommitSequenceId);
}
/**
* Returns KeyValueContainerReport for the KeyValueContainer.
*/
@ -530,8 +524,7 @@ public void updateBlockCommitSequenceId(long blockCommitSequenceId) {
.setKeyCount(containerData.getKeyCount())
.setUsed(containerData.getBytesUsed())
.setState(getHddsState())
.setDeleteTransactionId(containerData.getDeleteTransactionId())
.setBlockCommitSequenceId(containerData.getBlockCommitSequenceId());
.setDeleteTransactionId(containerData.getDeleteTransactionId());
return ciBuilder.build();
}

View File

@ -72,8 +72,6 @@ public class KeyValueContainerData extends ContainerData {
private long deleteTransactionId;
private long blockCommitSequenceId;
static {
// Initialize YAML fields
KV_YAML_FIELDS = Lists.newArrayList();
@ -153,20 +151,6 @@ public String getContainerPath() {
return new File(metadataPath).getParent();
}
/**
* Returns the blockCommitSequenceId.
*/
public long getBlockCommitSequenceId() {
return blockCommitSequenceId;
}
/**
* updates the blockCommitSequenceId.
*/
public void updateBlockCommitSequenceId(long id) {
this.blockCommitSequenceId = id;
}
/**
* Get chunks path.
* @return - Path where chunks are stored

View File

@ -25,16 +25,12 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
import org.apache.hadoop.hdfs.DFSUtil;
import org.apache.hadoop.ozone.OzoneConsts;
import org.apache.hadoop.ozone.container.common.helpers.BlockData;
import org.apache.hadoop.ozone.container.keyvalue.KeyValueContainerData;
import org.apache.hadoop.ozone.container.keyvalue.helpers.BlockUtils;
import org.apache.hadoop.ozone.container.common.interfaces.Container;
import org.apache.hadoop.ozone.container.keyvalue.interfaces.BlockManager;
import org.apache.hadoop.ozone.container.common.utils.ContainerCache;
import org.apache.hadoop.utils.BatchOperation;
import org.apache.hadoop.utils.MetadataKeyFilters;
import org.apache.hadoop.utils.MetadataStore;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@ -87,16 +83,8 @@ public long putBlock(Container container, BlockData data) throws IOException {
// This is a post condition that acts as a hint to the user.
// Should never fail.
Preconditions.checkNotNull(db, "DB cannot be null here");
long blockCommitSequenceId = data.getBlockCommitSequenceId();
// update the blockData as well as BlockCommitSequenceId here
BatchOperation batch = new BatchOperation();
batch.put(Longs.toByteArray(data.getLocalID()),
data.getProtoBufMessage().toByteArray());
batch.put(DFSUtil.string2Bytes(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX),
Longs.toByteArray(blockCommitSequenceId));
db.writeBatch(batch);
container.updateBlockCommitSequenceId(blockCommitSequenceId);
db.put(Longs.toByteArray(data.getLocalID()), data.getProtoBufMessage()
.toByteArray());
// Increment keycount here
container.getContainerData().incrKeyCount();
return data.getSize();
@ -220,9 +208,8 @@ public List<BlockData> listBlock(Container container, long startLocalID, int
MetadataStore db = BlockUtils.getDB(cData, config);
result = new ArrayList<>();
byte[] startKeyInBytes = Longs.toByteArray(startLocalID);
List<Map.Entry<byte[], byte[]>> range =
db.getSequentialRangeKVs(startKeyInBytes, count,
MetadataKeyFilters.getNormalKeyFilter());
List<Map.Entry<byte[], byte[]>> range = db.getSequentialRangeKVs(
startKeyInBytes, count, null);
for (Map.Entry<byte[], byte[]> entry : range) {
BlockData value = BlockUtils.getBlockData(entry.getValue());
BlockData data = new BlockData(value.getBlockID());

View File

@ -191,13 +191,6 @@ public void verifyContainerData(ContainerData containerData)
kvContainerData
.updateDeleteTransactionId(Longs.fromByteArray(delTxnId));
}
// sets the BlockCommitSequenceId.
byte[] bcsId = containerDB.get(
DFSUtil.string2Bytes(OzoneConsts.BLOCK_COMMIT_SEQUENCE_ID_PREFIX));
if (bcsId != null) {
kvContainerData
.updateBlockCommitSequenceId(Longs.fromByteArray(bcsId));
}
containerSet.addContainer(kvContainer);
} else {
throw new StorageContainerException("Container File is corrupted. " +

View File

@ -215,7 +215,6 @@ message ContainerInfo {
optional string finalhash = 9;
optional hadoop.hdds.LifeCycleState state = 10;
optional int64 deleteTransactionId = 11;
optional uint64 blockCommitSequenceId = 12;
}
/*

View File

@ -78,10 +78,6 @@ public long getOffset() {
return offset;
}
public long getBlockCommitSequenceId() {
return blockCommitSequenceId;
}
/**
* Builder of OmKeyLocationInfo.
*/

View File

@ -1,147 +0,0 @@
/**
* Licensed to the Apache Software Foundation (ASF) under one or more
* contributor license agreements. See the NOTICE file distributed with this
* work for additional information regarding copyright ownership. The ASF
* licenses this file to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance with the License.
* You may obtain a copy of the License at
* <p>
* http://www.apache.org/licenses/LICENSE-2.0
* <p>
* Unless required by applicable law or agreed to in writing, software
* distributed under the License is distributed on an "AS IS" BASIS, WITHOUT
* WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. See the
* License for the specific language governing permissions and limitations under
* the License.
*/
package org.apache.hadoop.ozone.client.rpc;
import org.apache.hadoop.hdds.client.ReplicationFactor;
import org.apache.hadoop.hdds.client.ReplicationType;
import org.apache.hadoop.hdds.conf.OzoneConfiguration;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.ozone.MiniOzoneCluster;
import org.apache.hadoop.ozone.client.ObjectStore;
import org.apache.hadoop.ozone.client.OzoneClient;
import org.apache.hadoop.ozone.client.OzoneClientFactory;
import org.apache.hadoop.ozone.client.io.OzoneOutputStream;
import org.apache.hadoop.ozone.om.helpers.OmKeyArgs;
import org.apache.hadoop.ozone.om.helpers.OmKeyInfo;
import org.apache.hadoop.ozone.om.helpers.OmKeyLocationInfo;
import org.apache.hadoop.test.GenericTestUtils;
import org.junit.AfterClass;
import org.junit.Assert;
import org.junit.BeforeClass;
import org.junit.Test;
import java.io.File;
import java.io.IOException;
import java.util.List;
import java.util.concurrent.TimeUnit;
import static org.apache.hadoop.hdds.HddsConfigKeys.
HDDS_COMMAND_STATUS_REPORT_INTERVAL;
import static org.apache.hadoop.hdds.HddsConfigKeys.
HDDS_CONTAINER_REPORT_INTERVAL;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
HDDS_SCM_WATCHER_TIMEOUT;
import static org.apache.hadoop.hdds.scm.ScmConfigKeys.
OZONE_SCM_STALENODE_INTERVAL;
/**
* Tests the validity BCSID of a container.
*/
public class TestBCSID {
private static MiniOzoneCluster cluster;
private static OzoneConfiguration conf;
private static OzoneClient client;
private static ObjectStore objectStore;
private static String volumeName;
private static String bucketName;
/**
* Create a MiniDFSCluster for testing.
*
* @throws IOException
*/
@BeforeClass
public static void init() throws Exception {
conf = new OzoneConfiguration();
String path = GenericTestUtils
.getTempPath(TestBCSID.class.getSimpleName());
File baseDir = new File(path);
baseDir.mkdirs();
conf.setTimeDuration(HDDS_CONTAINER_REPORT_INTERVAL, 200,
TimeUnit.MILLISECONDS);
conf.setTimeDuration(HDDS_COMMAND_STATUS_REPORT_INTERVAL, 200,
TimeUnit.MILLISECONDS);
conf.setTimeDuration(HDDS_SCM_WATCHER_TIMEOUT, 1000, TimeUnit.MILLISECONDS);
conf.setTimeDuration(OZONE_SCM_STALENODE_INTERVAL, 3, TimeUnit.SECONDS);
conf.setQuietMode(false);
cluster =
MiniOzoneCluster.newBuilder(conf).setNumDatanodes(1).setHbInterval(200)
.build();
cluster.waitForClusterToBeReady();
//the easiest way to create an open container is creating a key
client = OzoneClientFactory.getClient(conf);
objectStore = client.getObjectStore();
volumeName = "bcsid";
bucketName = volumeName;
objectStore.createVolume(volumeName);
objectStore.getVolume(volumeName).createBucket(bucketName);
}
/**
* Shutdown MiniDFSCluster.
*/
@AfterClass
public static void shutdown() {
if (cluster != null) {
cluster.shutdown();
}
}
@Test
public void testBCSID() throws Exception {
OzoneOutputStream key =
objectStore.getVolume(volumeName).getBucket(bucketName)
.createKey("ratis", 1024, ReplicationType.RATIS,
ReplicationFactor.ONE);
key.write("ratis".getBytes());
key.close();
// get the name of a valid container.
OmKeyArgs keyArgs = new OmKeyArgs.Builder().setVolumeName(volumeName).
setBucketName(bucketName).setType(HddsProtos.ReplicationType.RATIS)
.setFactor(HddsProtos.ReplicationFactor.ONE).setKeyName("ratis")
.build();
OmKeyInfo keyInfo = cluster.getOzoneManager().lookupKey(keyArgs);
List<OmKeyLocationInfo> keyLocationInfos =
keyInfo.getKeyLocationVersions().get(0).getBlocksLatestVersionOnly();
Assert.assertEquals(1, keyLocationInfos.size());
OmKeyLocationInfo omKeyLocationInfo = keyLocationInfos.get(0);
long blockCommitSequenceId =
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
.getContainer().getContainerSet()
.getContainer(omKeyLocationInfo.getContainerID())
.getContainerReport().getBlockCommitSequenceId();
Assert.assertTrue(blockCommitSequenceId > 0);
// make sure the persisted block Id in OM is same as that seen in the
// container report to be reported to SCM.
Assert.assertEquals(blockCommitSequenceId,
omKeyLocationInfo.getBlockCommitSequenceId());
// verify that on restarting the datanode, it reloads the BCSID correctly.
cluster.restartHddsDatanode(0);
Assert.assertEquals(blockCommitSequenceId,
cluster.getHddsDatanodes().get(0).getDatanodeStateMachine()
.getContainer().getContainerSet()
.getContainer(omKeyLocationInfo.getContainerID())
.getContainerReport().getBlockCommitSequenceId());
}
}