Skip to content
Merged
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -49,9 +49,11 @@
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.DeleteBlocksCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.DeleteContainerCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.FinalizeNewLayoutVersionCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ReconstructECContainersCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.RefreshVolumeUsageCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.ReplicateContainerCommandHandler;
import org.apache.hadoop.ozone.container.common.statemachine.commandhandler.SetNodeOperationalStateCommandHandler;
import org.apache.hadoop.ozone.container.ec.reconstruction.ECReconstructionSupervisor;
import org.apache.hadoop.ozone.container.keyvalue.TarContainerPacker;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.container.replication.ContainerReplicator;
Expand Down Expand Up @@ -96,6 +98,7 @@ public class DatanodeStateMachine implements Closeable {
private volatile Thread stateMachineThread = null;
private Thread cmdProcessThread = null;
private final ReplicationSupervisor supervisor;
private final ECReconstructionSupervisor ecReconstructionSupervisor;

private JvmPauseMonitor jvmPauseMonitor;
private CertificateClient dnCertClient;
Expand Down Expand Up @@ -178,6 +181,10 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
replicationSupervisorMetrics =
ReplicationSupervisorMetrics.create(supervisor);

ecReconstructionSupervisor =
new ECReconstructionSupervisor(container.getContainerSet(), context,
replicationConfig.getReplicationMaxStreams());


// When we add new handlers just adding a new handler here should do the
// trick.
Expand All @@ -187,6 +194,8 @@ public DatanodeStateMachine(DatanodeDetails datanodeDetails,
conf, dnConf.getBlockDeleteThreads(),
dnConf.getBlockDeleteQueueLimit()))
.addHandler(new ReplicateContainerCommandHandler(conf, supervisor))
.addHandler(new ReconstructECContainersCommandHandler(conf,
ecReconstructionSupervisor))
.addHandler(new DeleteContainerCommandHandler(
dnConf.getContainerDeleteThreads()))
.addHandler(new ClosePipelineCommandHandler())
Expand Down
Original file line number Diff line number Diff line change
@@ -0,0 +1,78 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <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.container.common.statemachine.commandhandler;

import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.protocol.proto.StorageContainerDatanodeProtocolProtos.SCMCommandProto.Type;
import org.apache.hadoop.ozone.container.common.statemachine.SCMConnectionManager;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;
import org.apache.hadoop.ozone.container.ec.reconstruction.ECReconstructionCommandInfo;
import org.apache.hadoop.ozone.container.ec.reconstruction.ECReconstructionCoordinatorTask;
import org.apache.hadoop.ozone.container.ec.reconstruction.ECReconstructionSupervisor;
import org.apache.hadoop.ozone.container.ozoneimpl.OzoneContainer;
import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand;
import org.apache.hadoop.ozone.protocol.commands.SCMCommand;

/**
* Command handler for reconstructing the lost EC containers.
*/
public class ReconstructECContainersCommandHandler implements CommandHandler {

private ECReconstructionSupervisor supervisor;
private ConfigurationSource conf;

public ReconstructECContainersCommandHandler(ConfigurationSource conf,
ECReconstructionSupervisor supervisor) {
this.conf = conf;
this.supervisor = supervisor;
}

@Override
public void handle(SCMCommand command, OzoneContainer container,
StateContext context, SCMConnectionManager connectionManager) {
ReconstructECContainersCommand ecContainersCommand =
(ReconstructECContainersCommand) command;
ECReconstructionCommandInfo reconstructionCommandInfo =
new ECReconstructionCommandInfo(ecContainersCommand.getContainerID(),
ecContainersCommand.getEcReplicationConfig(),
ecContainersCommand.getMissingContainerIndexes(),
ecContainersCommand.getSources(),
ecContainersCommand.getTargetDatanodes());
this.supervisor.addTask(
new ECReconstructionCoordinatorTask(reconstructionCommandInfo));
}

@Override
public Type getCommandType() {
return Type.reconstructECContainersCommand;
}

@Override
public int getInvocationCount() {
return 0;
}

@Override
public long getAverageRunTime() {
return 0;
}

public ConfigurationSource getConf() {
return conf;
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,82 @@
/*
* Licensed to the Apache Software Foundation (ASF) under one
* or more contributor license agreements. See the NOTICE file
* distributed with this work for additional information
* regarding copyright ownership. The ASF licenses this file
* to you under the Apache License, Version 2.0 (the
* "License"); you may not use this file except in compliance
* with the License. You may obtain a copy of the License at
* <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.container.ec.reconstruction;

import org.apache.hadoop.hdds.client.ECReplicationConfig;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand;
import org.apache.hadoop.ozone.protocol.commands.ReconstructECContainersCommand.DatanodeDetailsAndReplicaIndex;

import java.util.Arrays;
import java.util.List;

/**
* This class is to keep the required EC reconstruction info.
*/
public class ECReconstructionCommandInfo {
private long containerID;
private ECReplicationConfig ecReplicationConfig;
private byte[] missingContainerIndexes;
private List<ReconstructECContainersCommand.DatanodeDetailsAndReplicaIndex>
sources;
private List<DatanodeDetails> targetDatanodes;

public ECReconstructionCommandInfo(long containerID,
ECReplicationConfig ecReplicationConfig, byte[] missingContainerIndexes,
List<DatanodeDetailsAndReplicaIndex> sources,
List<DatanodeDetails> targetDatanodes) {
this.containerID = containerID;
this.ecReplicationConfig = ecReplicationConfig;
this.missingContainerIndexes =
Arrays.copyOf(missingContainerIndexes, missingContainerIndexes.length);
this.sources = sources;
this.targetDatanodes = targetDatanodes;
}

public long getContainerID() {
return containerID;
}

public byte[] getMissingContainerIndexes() {
return Arrays
.copyOf(missingContainerIndexes, missingContainerIndexes.length);
}

public ECReplicationConfig getEcReplicationConfig() {
return ecReplicationConfig;
}

public List<DatanodeDetailsAndReplicaIndex> getSources() {
return sources;
}

public List<DatanodeDetails> getTargetDatanodes() {
return targetDatanodes;
}

@Override
public String toString() {
return "ECReconstructionCommandInfo{"
+ "containerID=" + containerID
+ ", ecReplicationConfig=" + ecReplicationConfig
+ ", missingContainerIndexes=" + Arrays
.toString(missingContainerIndexes)
+ ", sources=" + sources
+ ", targetDatanodes=" + targetDatanodes + '}';
}
}
Original file line number Diff line number Diff line change
@@ -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
* <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.container.ec.reconstruction;

/**
* This is the actual EC reconstruction coordination task.
*/
public class ECReconstructionCoordinatorTask implements Runnable {
private ECReconstructionCommandInfo reconstructionCommandInfo;

public ECReconstructionCoordinatorTask(
ECReconstructionCommandInfo reconstructionCommandInfo) {
this.reconstructionCommandInfo = reconstructionCommandInfo;
}

@Override
public void run() {
// Implement the coordinator logic to handle a container group
// reconstruction.

// 1. Read container block meta info from the available min required good
// containers. ( Full block set should be available with 1st or parity
// indexes containers)
// 2. Find out the total number of blocks
// 3. Loop each block and use the ReconstructedInputStreams(HDDS-6665) and
// recover.
// 4. Write the recovered chunks to given targets/write locally to
// respective container. HDDS-6582
// 5. Close/finalize the recovered containers.
}

@Override
public String toString() {
return "ECReconstructionCoordinatorTask{" + "reconstructionCommandInfo="
+ reconstructionCommandInfo + '}';
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,72 @@
/*
* 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.container.ec.reconstruction;

import com.google.common.util.concurrent.ThreadFactoryBuilder;
import org.apache.hadoop.ozone.container.common.impl.ContainerSet;
import org.apache.hadoop.ozone.container.common.statemachine.StateContext;

import java.util.concurrent.ExecutorService;
import java.util.concurrent.LinkedBlockingQueue;
import java.util.concurrent.ThreadPoolExecutor;
import java.util.concurrent.TimeUnit;

/**
* This class is to handle all the EC reconstruction tasks to be scheduled as
* they arrive.
*/
public class ECReconstructionSupervisor {

private final ContainerSet containerSet;
private final StateContext context;
private final ExecutorService executor;

public ECReconstructionSupervisor(ContainerSet containerSet,
StateContext context, ExecutorService executor) {
this.containerSet = containerSet;
this.context = context;
this.executor = executor;
}

public ECReconstructionSupervisor(ContainerSet containerSet,
StateContext context, int poolSize) {
// TODO: ReplicationSupervisor and this class can be refactored to have a
// common interface.
this(containerSet, context,
new ThreadPoolExecutor(poolSize, poolSize, 60, TimeUnit.SECONDS,
new LinkedBlockingQueue<>(),
new ThreadFactoryBuilder().setDaemon(true)
.setNameFormat("ECContainerReconstructionThread-%d").build()));
}

public void stop() {
try {
executor.shutdown();
if (!executor.awaitTermination(3, TimeUnit.SECONDS)) {
executor.shutdownNow();
}
} catch (InterruptedException ie) {
// Ignore, we don't really care about the failure.
Thread.currentThread().interrupt();
}
}

public void addTask(ECReconstructionCoordinatorTask task) {
executor.execute(task);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,18 @@
/**
* 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.ec.reconstruction;
Original file line number Diff line number Diff line change
Expand Up @@ -135,7 +135,10 @@ public ECReplicationConfig getEcReplicationConfig() {
return ecReplicationConfig;
}

static class DatanodeDetailsAndReplicaIndex {
/**
* To store the datanode details with replica index.
*/
public static class DatanodeDetailsAndReplicaIndex {
private DatanodeDetails dnDetails;
private int replicaIndex;

Expand Down
Loading