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 @@ -589,23 +589,46 @@ public CompletableFuture<?> link(DataStream stream, LogEntryProto entry) {
if (stream == null) {
return JavaUtils.completeExceptionally(new IllegalStateException(
"DataStream is null"));
} else if (!(stream instanceof LocalStream)) {
return JavaUtils.completeExceptionally(new IllegalStateException(
"Unexpected DataStream " + stream.getClass()));
}
final DataChannel dataChannel = stream.getDataChannel();
if (dataChannel.isOpen()) {
return JavaUtils.completeExceptionally(new IllegalStateException(
"DataStream: " + stream + " is not closed properly"));
}

final ContainerCommandRequestProto request;
if (dataChannel instanceof KeyValueStreamDataChannel) {
request = ((KeyValueStreamDataChannel) dataChannel).getPutBlockRequest();
} else {
if (!(dataChannel instanceof KeyValueStreamDataChannel)) {
return JavaUtils.completeExceptionally(new IllegalStateException(
"Unexpected DataChannel " + dataChannel.getClass()));
}
return runCommandAsync(request, entry).whenComplete(
(res, e) -> LOG.debug("link {}, entry: {}, request: {}",
res.getResult(), entry, request));

final KeyValueStreamDataChannel kvStreamDataChannel =
(KeyValueStreamDataChannel) dataChannel;

final ContainerCommandRequestProto request =
kvStreamDataChannel.getPutBlockRequest();

return runCommandAsync(request, entry).whenComplete((response, e) -> {
if (e != null) {
LOG.warn("Failed to link logEntry {} for request {}",
TermIndex.valueOf(entry), request, e);
}
if (response != null) {
final ContainerProtos.Result result = response.getResult();
if (LOG.isDebugEnabled()) {
LOG.debug("{} to link logEntry {} for request {}, response: {}",
result, TermIndex.valueOf(entry), request, response);
}
if (result == ContainerProtos.Result.SUCCESS) {
kvStreamDataChannel.setLinked();
return;
}
}
// failed to link, cleanup
kvStreamDataChannel.cleanUp();
});
}

private ExecutorService getChunkExecutor(WriteChunkRequestProto req) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,11 +18,11 @@

package org.apache.hadoop.ozone.container.common.transport.server.ratis;

import org.apache.hadoop.ozone.container.keyvalue.impl.KeyValueStreamDataChannel;
import org.apache.ratis.statemachine.StateMachine;
import org.apache.ratis.util.JavaUtils;

import java.io.IOException;
import java.util.concurrent.CompletableFuture;
import java.util.concurrent.CompletionException;
import java.util.concurrent.Executor;

class LocalStream implements StateMachine.DataStream {
Expand All @@ -41,14 +41,13 @@ public StateMachine.DataChannel getDataChannel() {

@Override
public CompletableFuture<?> cleanUp() {
return CompletableFuture.supplyAsync(() -> {
try {
dataChannel.close();
return true;
} catch (IOException e) {
throw new CompletionException("Failed to close data channel", e);
}
});
if (!(dataChannel instanceof KeyValueStreamDataChannel)) {
return JavaUtils.completeExceptionally(new IllegalStateException(
"Unexpected DataChannel " + dataChannel.getClass()));
}
return CompletableFuture
.supplyAsync(((KeyValueStreamDataChannel) dataChannel)::cleanUp,
executor);
}

@Override
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -131,6 +131,12 @@ ReferenceCountedObject<ByteBuf> pollAll() {
refs.forEach(ReferenceCountedObject::release);
});
}

void cleanUpAll() {
while (!deque.isEmpty()) {
poll().release();
}
}
}

interface WriteMethod {
Expand Down Expand Up @@ -198,7 +204,18 @@ void assertOpen() throws IOException {
@Override
public void close() throws IOException {
if (closed.compareAndSet(false, true)) {
putBlockRequest.set(closeBuffers(buffers, super::writeFileChannel));
try {
putBlockRequest.set(closeBuffers(buffers, super::writeFileChannel));
} finally {
super.close();
}
}
}

@Override
protected void cleanupInternal() throws IOException {
buffers.cleanUpAll();
if (closed.compareAndSet(false, true)) {
super.close();
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -23,23 +23,32 @@
import org.apache.hadoop.ozone.container.common.helpers.ContainerMetrics;
import org.apache.hadoop.ozone.container.common.impl.ContainerData;
import org.apache.ratis.statemachine.StateMachine;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.File;
import java.io.FileNotFoundException;
import java.io.IOException;
import java.io.RandomAccessFile;
import java.nio.ByteBuffer;
import java.nio.channels.FileChannel;
import java.util.concurrent.atomic.AtomicBoolean;

import static org.apache.hadoop.ozone.container.common.utils.StorageVolumeUtil.onFailure;

/**
* For write state machine data.
*/
abstract class StreamDataChannelBase implements StateMachine.DataChannel {
abstract class StreamDataChannelBase
implements StateMachine.DataChannel {
static final Logger LOG = LoggerFactory.getLogger(
StreamDataChannelBase.class);

private final RandomAccessFile randomAccessFile;

private final File file;
private final AtomicBoolean linked = new AtomicBoolean();
private final AtomicBoolean cleaned = new AtomicBoolean();

private final ContainerData containerData;
private final ContainerMetrics metrics;
Expand Down Expand Up @@ -85,6 +94,29 @@ public final boolean isOpen() {
return getChannel().isOpen();
}

public void setLinked() {
linked.set(true);
}

/** @return true iff {@link StateMachine.DataChannel} is already linked. */
public boolean cleanUp() {
if (linked.get()) {
// already linked, nothing to do.
return true;
}
if (cleaned.compareAndSet(false, true)) {
// close and then delete the file.
try {
cleanupInternal();
} catch (IOException e) {
LOG.warn("Failed to close " + this, e);
}
}
return false;
}

protected abstract void cleanupInternal() throws IOException;

@Override
public void close() throws IOException {
try {
Expand Down