Skip to content
Merged
Show file tree
Hide file tree
Changes from 2 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
@@ -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.hdds.scm;

import java.io.IOException;
import java.nio.ByteBuffer;
import java.util.function.Function;

import org.apache.hadoop.hdds.scm.pipeline.Pipeline;

import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;

/**
* Interface to provide XceiverClient when needed.
*/
public interface XceiverClientFactory {

Function<ByteBuffer, ByteString> byteBufferToByteStringConversion();

XceiverClientSpi acquireClient(Pipeline pipeline) throws IOException;

void releaseClient(XceiverClientSpi xceiverClient, boolean invalidateClient);

XceiverClientSpi acquireClientForReadData(Pipeline pipeline)
throws IOException;

void releaseClientForReadData(XceiverClientSpi xceiverClient, boolean b);

}
Original file line number Diff line number Diff line change
Expand Up @@ -18,41 +18,40 @@

package org.apache.hadoop.hdds.scm;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;

import org.apache.hadoop.hdds.conf.Config;
import org.apache.hadoop.hdds.conf.ConfigGroup;
import org.apache.hadoop.hdds.conf.ConfigType;
import org.apache.hadoop.hdds.conf.ConfigurationSource;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.security.exception.SCMSecurityException;
import org.apache.hadoop.hdds.security.x509.certificate.utils.CertificateCodec;
import org.apache.hadoop.ozone.OzoneConfigKeys;
import org.apache.hadoop.ozone.OzoneSecurityUtil;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.Closeable;
import java.io.IOException;
import java.nio.ByteBuffer;
import java.security.cert.CertificateException;
import java.security.cert.X509Certificate;
import java.util.concurrent.Callable;
import java.util.concurrent.TimeUnit;
import java.util.function.Function;

import com.google.common.annotations.VisibleForTesting;
import com.google.common.base.Preconditions;
import com.google.common.cache.Cache;
import com.google.common.cache.CacheBuilder;
import com.google.common.cache.RemovalListener;
import com.google.common.cache.RemovalNotification;
import static java.util.concurrent.TimeUnit.MILLISECONDS;
import static org.apache.hadoop.hdds.conf.ConfigTag.OZONE;
import static org.apache.hadoop.hdds.conf.ConfigTag.PERFORMANCE;
import static org.apache.hadoop.hdds.scm.exceptions.SCMException.ResultCodes.NO_REPLICA_FOUND;
import org.apache.ratis.thirdparty.com.google.protobuf.ByteString;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

/**
* XceiverClientManager is responsible for the lifecycle of XceiverClient
Expand All @@ -66,7 +65,7 @@
* without reestablishing connection. But the connection will be closed if
* not being used for a period of time.
*/
public class XceiverClientManager implements Closeable {
public class XceiverClientManager implements Closeable, XceiverClientFactory {
private static final Logger LOG =
LoggerFactory.getLogger(XceiverClientManager.class);
//TODO : change this to SCM configuration class
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -18,32 +18,33 @@

package org.apache.hadoop.hdds.scm.storage;

import com.google.common.annotations.VisibleForTesting;
import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.function.Function;

import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetBlockResponseProto;
import org.apache.hadoop.hdds.protocol.proto.HddsProtos;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.ContainerNotFoundException;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;
import org.apache.hadoop.fs.Seekable;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.DatanodeBlockID;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.GetBlockResponseProto;

import com.google.common.annotations.VisibleForTesting;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;

import java.io.EOFException;
import java.io.IOException;
import java.io.InputStream;
import java.util.ArrayList;
import java.util.Arrays;
import java.util.List;
import java.util.function.Function;

/**
* An {@link InputStream} called from KeyInputStream to read a block from the
* container.
Expand All @@ -62,7 +63,7 @@ public class BlockInputStream extends InputStream implements Seekable {
private Pipeline pipeline;
private final Token<OzoneBlockTokenIdentifier> token;
private final boolean verifyChecksum;
private XceiverClientManager xceiverClientManager;
private XceiverClientFactory xceiverClientFactory;
private XceiverClientSpi xceiverClient;
private boolean initialized = false;

Expand Down Expand Up @@ -99,23 +100,24 @@ public class BlockInputStream extends InputStream implements Seekable {

public BlockInputStream(BlockID blockId, long blockLen, Pipeline pipeline,
Token<OzoneBlockTokenIdentifier> token, boolean verifyChecksum,
XceiverClientManager xceiverClientManager,
XceiverClientFactory xceiverClientFctry,
Function<BlockID, Pipeline> refreshPipelineFunction) {
this.blockID = blockId;
this.length = blockLen;
this.pipeline = pipeline;
this.token = token;
this.verifyChecksum = verifyChecksum;
this.xceiverClientManager = xceiverClientManager;
this.xceiverClientFactory = xceiverClientFctry;
this.refreshPipelineFunction = refreshPipelineFunction;
}

public BlockInputStream(BlockID blockId, long blockLen, Pipeline pipeline,
Token<OzoneBlockTokenIdentifier> token,
boolean verifyChecksum,
XceiverClientManager xceiverClientManager) {
XceiverClientManager xceiverClientFactory
) {
this(blockId, blockLen, pipeline, token, verifyChecksum,
xceiverClientManager, null);
xceiverClientFactory, null);
}
/**
* Initialize the BlockInputStream. Get the BlockData (list of chunks) from
Expand Down Expand Up @@ -181,7 +183,7 @@ protected List<ChunkInfo> getChunkInfos() throws IOException {
pipeline = Pipeline.newBuilder(pipeline)
.setType(HddsProtos.ReplicationType.STAND_ALONE).build();
}
xceiverClient = xceiverClientManager.acquireClientForReadData(pipeline);
xceiverClient = xceiverClientFactory.acquireClientForReadData(pipeline);
boolean success = false;
List<ChunkInfo> chunks;
try {
Expand All @@ -202,7 +204,7 @@ protected List<ChunkInfo> getChunkInfos() throws IOException {
success = true;
} finally {
if (!success) {
xceiverClientManager.releaseClientForReadData(xceiverClient, false);
xceiverClientFactory.releaseClientForReadData(xceiverClient, false);
}
}

Expand Down Expand Up @@ -378,9 +380,9 @@ public boolean seekToNewSource(long targetPos) throws IOException {

@Override
public synchronized void close() {
if (xceiverClientManager != null && xceiverClient != null) {
xceiverClientManager.releaseClient(xceiverClient, false);
xceiverClientManager = null;
if (xceiverClientFactory != null && xceiverClient != null) {
xceiverClientFactory.releaseClient(xceiverClient, false);
xceiverClientFactory = null;
xceiverClient = null;
}
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -37,7 +37,7 @@
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChunkInfo;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.KeyValue;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.XceiverClientReply;
import org.apache.hadoop.hdds.scm.XceiverClientSpi;
import org.apache.hadoop.hdds.scm.container.common.helpers.StorageContainerException;
Expand Down Expand Up @@ -80,7 +80,7 @@ public class BlockOutputStream extends OutputStream {
private AtomicReference<BlockID> blockID;

private final BlockData.Builder containerBlockData;
private XceiverClientManager xceiverClientManager;
private XceiverClientFactory xceiverClientFactory;
private XceiverClientSpi xceiverClient;
private final int bytesPerChecksum;
private int chunkIndex;
Expand Down Expand Up @@ -129,7 +129,7 @@ public class BlockOutputStream extends OutputStream {
* Creates a new BlockOutputStream.
*
* @param blockID block ID
* @param xceiverClientManager client manager that controls client
* @param xceiverClientFactory client manager that controls client
* @param pipeline pipeline where block will be written
* @param bufferPool pool of buffers
* @param streamBufferFlushSize flush size
Expand All @@ -139,7 +139,7 @@ public class BlockOutputStream extends OutputStream {
*/
@SuppressWarnings("parameternumber")
public BlockOutputStream(BlockID blockID,
XceiverClientManager xceiverClientManager, Pipeline pipeline,
XceiverClientFactory xceiverClientFactory, Pipeline pipeline,
int streamBufferSize, long streamBufferFlushSize,
boolean streamBufferFlushDelay, long streamBufferMaxSize,
BufferPool bufferPool, ChecksumType checksumType,
Expand All @@ -150,8 +150,8 @@ public BlockOutputStream(BlockID blockID,
this.containerBlockData =
BlockData.newBuilder().setBlockID(blockID.getDatanodeBlockIDProtobuf())
.addMetadata(keyValue);
this.xceiverClientManager = xceiverClientManager;
this.xceiverClient = xceiverClientManager.acquireClient(pipeline);
this.xceiverClientFactory = xceiverClientFactory;
this.xceiverClient = xceiverClientFactory.acquireClient(pipeline);
this.streamBufferSize = streamBufferSize;
this.streamBufferFlushSize = streamBufferFlushSize;
this.streamBufferMaxSize = streamBufferMaxSize;
Expand Down Expand Up @@ -477,7 +477,7 @@ ContainerCommandResponseProto> executePutBlock(boolean close,

@Override
public void flush() throws IOException {
if (xceiverClientManager != null && xceiverClient != null
if (xceiverClientFactory != null && xceiverClient != null
&& bufferPool != null && bufferPool.getSize() > 0
&& (!streamBufferFlushDelay ||
writtenDataLength - totalDataFlushedLength >= streamBufferSize)) {
Expand Down Expand Up @@ -543,7 +543,7 @@ private void handleFlush(boolean close)

@Override
public void close() throws IOException {
if (xceiverClientManager != null && xceiverClient != null
if (xceiverClientFactory != null && xceiverClient != null
&& bufferPool != null && bufferPool.getSize() > 0) {
try {
handleFlush(true);
Expand Down Expand Up @@ -604,10 +604,10 @@ private void setIoException(Exception e) {
}

public void cleanup(boolean invalidateClient) {
if (xceiverClientManager != null) {
xceiverClientManager.releaseClient(xceiverClient, invalidateClient);
if (xceiverClientFactory != null) {
xceiverClientFactory.releaseClient(xceiverClient, invalidateClient);
}
xceiverClientManager = null;
xceiverClientFactory = null;
xceiverClient = null;
commitWatcher.cleanup();
if (bufferList != null) {
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -19,22 +19,21 @@

import java.io.IOException;
import java.io.OutputStream;
import java.util.Collection;
import java.util.Collections;

import com.google.common.annotations.VisibleForTesting;
import org.apache.hadoop.hdds.client.BlockID;
import org.apache.hadoop.hdds.protocol.DatanodeDetails;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos
.ChecksumType;
import org.apache.hadoop.hdds.scm.XceiverClientManager;
import org.apache.hadoop.hdds.protocol.datanode.proto.ContainerProtos.ChecksumType;
import org.apache.hadoop.hdds.scm.XceiverClientFactory;
import org.apache.hadoop.hdds.scm.pipeline.Pipeline;
import org.apache.hadoop.hdds.scm.storage.BlockOutputStream;
import org.apache.hadoop.hdds.scm.storage.BufferPool;
import org.apache.hadoop.hdds.security.token.OzoneBlockTokenIdentifier;
import org.apache.hadoop.security.UserGroupInformation;
import org.apache.hadoop.security.token.Token;

import java.util.Collection;
import java.util.Collections;
import com.google.common.annotations.VisibleForTesting;

/**
* Helper class used inside {@link BlockOutputStream}.
Expand All @@ -44,7 +43,7 @@ public final class BlockOutputStreamEntry extends OutputStream {
private OutputStream outputStream;
private BlockID blockID;
private final String key;
private final XceiverClientManager xceiverClientManager;
private final XceiverClientFactory xceiverClientManager;
private final Pipeline pipeline;
private final ChecksumType checksumType;
private final int bytesPerChecksum;
Expand All @@ -64,7 +63,7 @@ public final class BlockOutputStreamEntry extends OutputStream {

@SuppressWarnings({"parameternumber", "squid:S00107"})
private BlockOutputStreamEntry(BlockID blockID, String key,
XceiverClientManager xceiverClientManager,
XceiverClientFactory xceiverClientManager,
Pipeline pipeline, String requestId, int chunkSize,
long length, int streamBufferSize, long streamBufferFlushSize,
boolean streamBufferFlushDelay, long streamBufferMaxSize,
Expand Down Expand Up @@ -215,7 +214,7 @@ public static class Builder {

private BlockID blockID;
private String key;
private XceiverClientManager xceiverClientManager;
private XceiverClientFactory xceiverClientManager;
private Pipeline pipeline;
private String requestId;
private int chunkSize;
Expand Down Expand Up @@ -250,7 +249,8 @@ public Builder setKey(String keys) {
return this;
}

public Builder setXceiverClientManager(XceiverClientManager
public Builder setXceiverClientManager(
XceiverClientFactory
xClientManager) {
this.xceiverClientManager = xClientManager;
return this;
Expand Down Expand Up @@ -333,7 +333,7 @@ public String getKey() {
return key;
}

public XceiverClientManager getXceiverClientManager() {
public XceiverClientFactory getXceiverClientManager() {
return xceiverClientManager;
}

Expand Down
Loading