HDFS-13702. Remove HTrace hooks from DFSClient to reduce CPU usage. Contributed by Todd Lipcon.
This commit is contained in:
parent
6ba9974108
commit
5d748bd056
@ -3070,25 +3070,6 @@ TraceScope newSrcDstTraceScope(String description, String src, String dst) {
|
||||
return scope;
|
||||
}
|
||||
|
||||
/**
|
||||
* Full detailed tracing for read requests: path, position in the file,
|
||||
* and length.
|
||||
*
|
||||
* @param reqLen requested length
|
||||
*/
|
||||
TraceScope newReaderTraceScope(String description, String path, long pos,
|
||||
int reqLen) {
|
||||
TraceScope scope = newPathTraceScope(description, path);
|
||||
scope.addKVAnnotation("pos", Long.toString(pos));
|
||||
scope.addKVAnnotation("reqLen", Integer.toString(reqLen));
|
||||
return scope;
|
||||
}
|
||||
|
||||
/** Add the returned length info to the scope. */
|
||||
void addRetLenToReaderScope(TraceScope scope, int retLen) {
|
||||
scope.addKVAnnotation("retLen", Integer.toString(retLen));
|
||||
}
|
||||
|
||||
/**
|
||||
* Get the erasure coding policy information for the specified path
|
||||
*
|
||||
|
@ -85,8 +85,6 @@
|
||||
import org.apache.hadoop.util.StopWatch;
|
||||
import org.apache.hadoop.util.StringUtils;
|
||||
import org.apache.htrace.core.SpanId;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
@ -641,7 +639,6 @@ protected BlockReader getBlockReader(LocatedBlock targetBlock,
|
||||
setClientCacheContext(dfsClient.getClientContext()).
|
||||
setUserGroupInformation(dfsClient.ugi).
|
||||
setConfiguration(dfsClient.getConfiguration()).
|
||||
setTracer(dfsClient.getTracer()).
|
||||
build();
|
||||
}
|
||||
|
||||
@ -821,31 +818,14 @@ public synchronized int read(@Nonnull final byte buf[], int off, int len)
|
||||
}
|
||||
ReaderStrategy byteArrayReader =
|
||||
new ByteArrayStrategy(buf, off, len, readStatistics, dfsClient);
|
||||
try (TraceScope scope =
|
||||
dfsClient.newReaderTraceScope("DFSInputStream#byteArrayRead",
|
||||
src, getPos(), len)) {
|
||||
int retLen = readWithStrategy(byteArrayReader);
|
||||
if (retLen < len) {
|
||||
dfsClient.addRetLenToReaderScope(scope, retLen);
|
||||
}
|
||||
return retLen;
|
||||
}
|
||||
return readWithStrategy(byteArrayReader);
|
||||
}
|
||||
|
||||
@Override
|
||||
public synchronized int read(final ByteBuffer buf) throws IOException {
|
||||
ReaderStrategy byteBufferReader =
|
||||
new ByteBufferStrategy(buf, readStatistics, dfsClient);
|
||||
int reqLen = buf.remaining();
|
||||
try (TraceScope scope =
|
||||
dfsClient.newReaderTraceScope("DFSInputStream#byteBufferRead",
|
||||
src, getPos(), reqLen)){
|
||||
int retLen = readWithStrategy(byteBufferReader);
|
||||
if (retLen < reqLen) {
|
||||
dfsClient.addRetLenToReaderScope(scope, retLen);
|
||||
}
|
||||
return retLen;
|
||||
}
|
||||
return readWithStrategy(byteBufferReader);
|
||||
}
|
||||
|
||||
private DNAddrPair chooseDataNode(LocatedBlock block,
|
||||
@ -1026,17 +1006,13 @@ private Callable<ByteBuffer> getFromOneDataNode(final DNAddrPair datanode,
|
||||
final ByteBuffer bb,
|
||||
final CorruptedBlocks corruptedBlocks,
|
||||
final int hedgedReadId) {
|
||||
final SpanId parentSpanId = Tracer.getCurrentSpanId();
|
||||
return new Callable<ByteBuffer>() {
|
||||
@Override
|
||||
public ByteBuffer call() throws Exception {
|
||||
DFSClientFaultInjector.get().sleepBeforeHedgedGet();
|
||||
try (TraceScope ignored = dfsClient.getTracer().
|
||||
newScope("hedgedRead" + hedgedReadId, parentSpanId)) {
|
||||
actualGetFromOneDataNode(datanode, start, end, bb, corruptedBlocks);
|
||||
return bb;
|
||||
}
|
||||
}
|
||||
};
|
||||
}
|
||||
|
||||
@ -1336,16 +1312,8 @@ public int read(long position, byte[] buffer, int offset, int length)
|
||||
if (length == 0) {
|
||||
return 0;
|
||||
}
|
||||
try (TraceScope scope = dfsClient.
|
||||
newReaderTraceScope("DFSInputStream#byteArrayPread",
|
||||
src, position, length)) {
|
||||
ByteBuffer bb = ByteBuffer.wrap(buffer, offset, length);
|
||||
int retLen = pread(position, bb);
|
||||
if (retLen < length) {
|
||||
dfsClient.addRetLenToReaderScope(scope, retLen);
|
||||
}
|
||||
return retLen;
|
||||
}
|
||||
return pread(position, bb);
|
||||
}
|
||||
|
||||
private int pread(long position, ByteBuffer buffer)
|
||||
|
@ -75,7 +75,6 @@
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
import com.google.common.base.Preconditions;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
@ -189,11 +188,6 @@ public boolean getSupportsReceiptVerification() {
|
||||
*/
|
||||
private Configuration configuration;
|
||||
|
||||
/**
|
||||
* The HTrace tracer to use.
|
||||
*/
|
||||
private Tracer tracer;
|
||||
|
||||
/**
|
||||
* Information about the domain socket path we should use to connect to the
|
||||
* local peer-- or null if we haven't examined the local domain socket.
|
||||
@ -298,11 +292,6 @@ public BlockReaderFactory setConfiguration(
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderFactory setTracer(Tracer tracer) {
|
||||
this.tracer = tracer;
|
||||
return this;
|
||||
}
|
||||
|
||||
@VisibleForTesting
|
||||
public static void setFailureInjectorForTesting(FailureInjector injector) {
|
||||
failureInjector = injector;
|
||||
@ -451,7 +440,7 @@ private BlockReader getLegacyBlockReaderLocal() throws IOException {
|
||||
try {
|
||||
return BlockReaderLocalLegacy.newBlockReader(conf,
|
||||
userGroupInformation, configuration, fileName, block, token,
|
||||
datanode, startOffset, length, storageType, tracer);
|
||||
datanode, startOffset, length, storageType);
|
||||
} catch (RemoteException remoteException) {
|
||||
ioe = remoteException.unwrapRemoteException(
|
||||
InvalidToken.class, AccessControlException.class);
|
||||
@ -509,7 +498,6 @@ private BlockReader getBlockReaderLocal() throws IOException {
|
||||
setVerifyChecksum(verifyChecksum).
|
||||
setCachingStrategy(cachingStrategy).
|
||||
setStorageType(storageType).
|
||||
setTracer(tracer).
|
||||
build();
|
||||
}
|
||||
|
||||
@ -860,7 +848,7 @@ private BlockReader getRemoteBlockReader(Peer peer) throws IOException {
|
||||
return BlockReaderRemote.newBlockReader(
|
||||
fileName, block, token, startOffset, length,
|
||||
verifyChecksum, clientName, peer, datanode,
|
||||
clientContext.getPeerCache(), cachingStrategy, tracer,
|
||||
clientContext.getPeerCache(), cachingStrategy,
|
||||
networkDistance);
|
||||
}
|
||||
|
||||
|
@ -35,8 +35,6 @@
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.hadoop.util.DirectBufferPool;
|
||||
import org.apache.hadoop.util.Timer;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -83,7 +81,6 @@ public static class Builder {
|
||||
private long dataPos;
|
||||
private ExtendedBlock block;
|
||||
private StorageType storageType;
|
||||
private Tracer tracer;
|
||||
private ShortCircuitConf shortCircuitConf;
|
||||
|
||||
public Builder(ShortCircuitConf conf) {
|
||||
@ -131,11 +128,6 @@ public Builder setStorageType(StorageType storageType) {
|
||||
return this;
|
||||
}
|
||||
|
||||
public Builder setTracer(Tracer tracer) {
|
||||
this.tracer = tracer;
|
||||
return this;
|
||||
}
|
||||
|
||||
public BlockReaderLocal build() {
|
||||
Preconditions.checkNotNull(replica);
|
||||
return new BlockReaderLocal(this);
|
||||
@ -244,11 +236,6 @@ public BlockReaderLocal build() {
|
||||
*/
|
||||
private StorageType storageType;
|
||||
|
||||
/**
|
||||
* The Tracer to use.
|
||||
*/
|
||||
private final Tracer tracer;
|
||||
|
||||
private BlockReaderLocal(Builder builder) {
|
||||
this.replica = builder.replica;
|
||||
this.dataIn = replica.getDataStream().getChannel();
|
||||
@ -278,7 +265,6 @@ private BlockReaderLocal(Builder builder) {
|
||||
}
|
||||
this.maxReadaheadLength = maxReadaheadChunks * bytesPerChecksum;
|
||||
this.storageType = builder.storageType;
|
||||
this.tracer = builder.tracer;
|
||||
|
||||
if (builder.shortCircuitConf.isScrMetricsEnabled()) {
|
||||
metricsInitializationLock.lock();
|
||||
@ -360,8 +346,6 @@ private synchronized int drainDataBuf(ByteBuffer buf) {
|
||||
*/
|
||||
private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
|
||||
throws IOException {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"BlockReaderLocal#fillBuffer(" + block.getBlockId() + ")")) {
|
||||
int total = 0;
|
||||
long startDataPos = dataPos;
|
||||
int startBufPos = buf.position();
|
||||
@ -406,7 +390,6 @@ private synchronized int fillBuffer(ByteBuffer buf, boolean canSkipChecksum)
|
||||
}
|
||||
return total;
|
||||
}
|
||||
}
|
||||
|
||||
private boolean createNoChecksumContext() {
|
||||
return !verifyChecksum ||
|
||||
|
@ -51,8 +51,6 @@
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.hadoop.util.DirectBufferPool;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
import org.apache.htrace.core.Tracer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -184,7 +182,6 @@ private void removeBlockLocalPathInfo(ExtendedBlock b) {
|
||||
private long startOffset;
|
||||
private final String filename;
|
||||
private long blockId;
|
||||
private final Tracer tracer;
|
||||
|
||||
/**
|
||||
* The only way this object can be instantiated.
|
||||
@ -193,8 +190,8 @@ static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
|
||||
UserGroupInformation userGroupInformation,
|
||||
Configuration configuration, String file, ExtendedBlock blk,
|
||||
Token<BlockTokenIdentifier> token, DatanodeInfo node,
|
||||
long startOffset, long length, StorageType storageType,
|
||||
Tracer tracer) throws IOException {
|
||||
long startOffset, long length, StorageType storageType)
|
||||
throws IOException {
|
||||
final ShortCircuitConf scConf = conf.getShortCircuitConf();
|
||||
LocalDatanodeInfo localDatanodeInfo = getLocalDatanodeInfo(node
|
||||
.getIpcPort());
|
||||
@ -239,11 +236,10 @@ static BlockReaderLocalLegacy newBlockReader(DfsClientConf conf,
|
||||
long firstChunkOffset = startOffset
|
||||
- (startOffset % checksum.getBytesPerChecksum());
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
||||
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn,
|
||||
tracer);
|
||||
startOffset, checksum, true, dataIn, firstChunkOffset, checksumIn);
|
||||
} else {
|
||||
localBlockReader = new BlockReaderLocalLegacy(scConf, file, blk,
|
||||
startOffset, dataIn, tracer);
|
||||
startOffset, dataIn);
|
||||
}
|
||||
} catch (IOException e) {
|
||||
// remove from cache
|
||||
@ -320,17 +316,17 @@ private static int getSlowReadBufferNumChunks(int bufferSizeBytes,
|
||||
}
|
||||
|
||||
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
ExtendedBlock block, long startOffset, FileInputStream dataIn,
|
||||
Tracer tracer) throws IOException {
|
||||
ExtendedBlock block, long startOffset, FileInputStream dataIn)
|
||||
throws IOException {
|
||||
this(conf, hdfsfile, block, startOffset,
|
||||
DataChecksum.newDataChecksum(DataChecksum.Type.NULL, 4), false,
|
||||
dataIn, startOffset, null, tracer);
|
||||
dataIn, startOffset, null);
|
||||
}
|
||||
|
||||
private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
ExtendedBlock block, long startOffset, DataChecksum checksum,
|
||||
boolean verifyChecksum, FileInputStream dataIn, long firstChunkOffset,
|
||||
FileInputStream checksumIn, Tracer tracer) throws IOException {
|
||||
FileInputStream checksumIn) throws IOException {
|
||||
this.filename = hdfsfile;
|
||||
this.checksum = checksum;
|
||||
this.verifyChecksum = verifyChecksum;
|
||||
@ -369,7 +365,6 @@ private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
bufferPool.returnBuffer(checksumBuff);
|
||||
}
|
||||
}
|
||||
this.tracer = tracer;
|
||||
}
|
||||
|
||||
/**
|
||||
@ -377,8 +372,6 @@ private BlockReaderLocalLegacy(ShortCircuitConf conf, String hdfsfile,
|
||||
*/
|
||||
private int fillBuffer(FileInputStream stream, ByteBuffer buf)
|
||||
throws IOException {
|
||||
try (TraceScope ignored = tracer.
|
||||
newScope("BlockReaderLocalLegacy#fillBuffer(" + blockId + ")")) {
|
||||
int bytesRead = stream.getChannel().read(buf);
|
||||
if (bytesRead < 0) {
|
||||
//EOF
|
||||
@ -394,7 +387,6 @@ private int fillBuffer(FileInputStream stream, ByteBuffer buf)
|
||||
}
|
||||
return bytesRead;
|
||||
}
|
||||
}
|
||||
|
||||
/**
|
||||
* Utility method used by read(ByteBuffer) to partially copy a ByteBuffer into
|
||||
|
@ -49,11 +49,9 @@
|
||||
import org.apache.hadoop.hdfs.shortcircuit.ClientMmap;
|
||||
import org.apache.hadoop.security.token.Token;
|
||||
import org.apache.hadoop.util.DataChecksum;
|
||||
import org.apache.htrace.core.TraceScope;
|
||||
|
||||
import com.google.common.annotations.VisibleForTesting;
|
||||
|
||||
import org.apache.htrace.core.Tracer;
|
||||
import org.slf4j.Logger;
|
||||
import org.slf4j.LoggerFactory;
|
||||
|
||||
@ -121,8 +119,6 @@ public class BlockReaderRemote implements BlockReader {
|
||||
|
||||
private boolean sentStatusCode = false;
|
||||
|
||||
private final Tracer tracer;
|
||||
|
||||
private final int networkDistance;
|
||||
|
||||
@VisibleForTesting
|
||||
@ -139,11 +135,8 @@ public synchronized int read(byte[] buf, int off, int len)
|
||||
|
||||
if (curDataSlice == null ||
|
||||
curDataSlice.remaining() == 0 && bytesNeededToFinish > 0) {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
|
||||
readNextPacket();
|
||||
}
|
||||
}
|
||||
|
||||
LOG.trace("Finishing read #{}", randomId);
|
||||
|
||||
@ -163,11 +156,8 @@ public synchronized int read(byte[] buf, int off, int len)
|
||||
public synchronized int read(ByteBuffer buf) throws IOException {
|
||||
if (curDataSlice == null ||
|
||||
(curDataSlice.remaining() == 0 && bytesNeededToFinish > 0)) {
|
||||
try (TraceScope ignored = tracer.newScope(
|
||||
"BlockReaderRemote2#readNextPacket(" + blockId + ")")) {
|
||||
readNextPacket();
|
||||
}
|
||||
}
|
||||
if (curDataSlice.remaining() == 0) {
|
||||
// we're at EOF now
|
||||
return -1;
|
||||
@ -280,7 +270,6 @@ protected BlockReaderRemote(String file, long blockId,
|
||||
long startOffset, long firstChunkOffset,
|
||||
long bytesToRead, Peer peer,
|
||||
DatanodeID datanodeID, PeerCache peerCache,
|
||||
Tracer tracer,
|
||||
int networkDistance) {
|
||||
// Path is used only for printing block and file information in debug
|
||||
this.peer = peer;
|
||||
@ -300,7 +289,6 @@ protected BlockReaderRemote(String file, long blockId,
|
||||
this.bytesNeededToFinish = bytesToRead + (startOffset - firstChunkOffset);
|
||||
bytesPerChecksum = this.checksum.getBytesPerChecksum();
|
||||
checksumSize = this.checksum.getChecksumSize();
|
||||
this.tracer = tracer;
|
||||
this.networkDistance = networkDistance;
|
||||
}
|
||||
|
||||
@ -397,7 +385,6 @@ public static BlockReader newBlockReader(String file,
|
||||
Peer peer, DatanodeID datanodeID,
|
||||
PeerCache peerCache,
|
||||
CachingStrategy cachingStrategy,
|
||||
Tracer tracer,
|
||||
int networkDistance) throws IOException {
|
||||
// in and out will be closed when sock is closed (by the caller)
|
||||
final DataOutputStream out = new DataOutputStream(new BufferedOutputStream(
|
||||
@ -431,7 +418,7 @@ public static BlockReader newBlockReader(String file,
|
||||
|
||||
return new BlockReaderRemote(file, block.getBlockId(), checksum,
|
||||
verifyChecksum, startOffset, firstChunkOffset, len, peer, datanodeID,
|
||||
peerCache, tracer, networkDistance);
|
||||
peerCache, networkDistance);
|
||||
}
|
||||
|
||||
static void checkSuccess(
|
||||
|
@ -128,7 +128,7 @@ private BlockReader createBlockReader(long offsetInBlock) {
|
||||
return BlockReaderRemote.newBlockReader(
|
||||
"dummy", block, blockToken, offsetInBlock,
|
||||
block.getNumBytes() - offsetInBlock, true, "", peer, source,
|
||||
null, stripedReader.getCachingStrategy(), datanode.getTracer(), -1);
|
||||
null, stripedReader.getCachingStrategy(), -1);
|
||||
} catch (IOException e) {
|
||||
LOG.info("Exception while creating remote block reader, datanode {}",
|
||||
source, e);
|
||||
|
@ -1032,7 +1032,6 @@ private void copyBlock(final DFSClient dfs, LocatedBlock lblock,
|
||||
setCachingStrategy(CachingStrategy.newDropBehind()).
|
||||
setClientCacheContext(dfs.getClientContext()).
|
||||
setConfiguration(namenode.getConf()).
|
||||
setTracer(tracer).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr,
|
||||
|
@ -30,7 +30,6 @@
|
||||
|
||||
import org.apache.commons.io.IOUtils;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FsTracer;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
@ -206,7 +205,6 @@ public static BlockReader getBlockReader(final DistributedFileSystem fs,
|
||||
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
|
||||
setConfiguration(fs.getConf()).
|
||||
setAllowShortCircuitLocalReads(true).
|
||||
setTracer(FsTracer.get(fs.getConf())).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr,
|
||||
|
@ -52,7 +52,6 @@
|
||||
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitReplica;
|
||||
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm;
|
||||
import org.apache.hadoop.hdfs.shortcircuit.ShortCircuitShm.ShmId;
|
||||
import org.apache.hadoop.fs.FsTracer;
|
||||
import org.apache.hadoop.io.IOUtils;
|
||||
import org.apache.hadoop.net.NetUtils;
|
||||
import org.apache.hadoop.net.unix.DomainSocket;
|
||||
@ -208,7 +207,6 @@ public void runBlockReaderLocalTest(BlockReaderLocalTest test,
|
||||
setShortCircuitReplica(replica).
|
||||
setCachingStrategy(new CachingStrategy(false, readahead)).
|
||||
setVerifyChecksum(checksum).
|
||||
setTracer(FsTracer.get(conf)).
|
||||
build();
|
||||
dataIn = null;
|
||||
metaIn = null;
|
||||
|
@ -33,7 +33,6 @@
|
||||
import org.apache.hadoop.fs.FSDataInputStream;
|
||||
import org.apache.hadoop.fs.FSDataOutputStream;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FsTracer;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.client.impl.BlockReaderFactory;
|
||||
@ -167,7 +166,6 @@ protected void tryRead(final Configuration conf, LocatedBlock lblock,
|
||||
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
|
||||
setClientCacheContext(ClientContext.getFromConf(conf)).
|
||||
setConfiguration(conf).
|
||||
setTracer(FsTracer.get(conf)).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr,
|
||||
|
@ -40,7 +40,6 @@
|
||||
import org.apache.hadoop.conf.Configuration;
|
||||
import org.apache.hadoop.fs.FileSystem;
|
||||
import org.apache.hadoop.fs.FileUtil;
|
||||
import org.apache.hadoop.fs.FsTracer;
|
||||
import org.apache.hadoop.fs.Path;
|
||||
import org.apache.hadoop.hdfs.BlockReader;
|
||||
import org.apache.hadoop.hdfs.ClientContext;
|
||||
@ -655,7 +654,6 @@ private void accessBlock(DatanodeInfo datanode, LocatedBlock lblock)
|
||||
setCachingStrategy(CachingStrategy.newDefaultStrategy()).
|
||||
setClientCacheContext(ClientContext.getFromConf(conf)).
|
||||
setConfiguration(conf).
|
||||
setTracer(FsTracer.get(conf)).
|
||||
setRemotePeerFactory(new RemotePeerFactory() {
|
||||
@Override
|
||||
public Peer newConnectedPeer(InetSocketAddress addr,
|
||||
|
Loading…
Reference in New Issue
Block a user