[SBN read] Addendum: Misc fix of issues, javadoc, whitespace unused backported fields etc
This commit is contained in:
parent
12dc7ff566
commit
097710131a
|
@ -76,5 +76,15 @@ public class MultipleIOException extends IOException {
|
|||
public IOException build() {
|
||||
return createIOException(exceptions);
|
||||
}
|
||||
|
||||
/**
|
||||
* @return whether any exception was added.
|
||||
*/
|
||||
public boolean isEmpty() {
|
||||
if (exceptions == null) {
|
||||
return true;
|
||||
}
|
||||
return exceptions.isEmpty();
|
||||
}
|
||||
}
|
||||
}
|
||||
|
|
|
@ -140,10 +140,6 @@ public abstract class Server {
|
|||
private ExceptionsHandler exceptionsHandler = new ExceptionsHandler();
|
||||
private Tracer tracer;
|
||||
private AlignmentContext alignmentContext;
|
||||
/**
|
||||
* Logical name of the server used in metrics and monitor.
|
||||
*/
|
||||
private final String serverName;
|
||||
|
||||
/**
|
||||
* Add exception classes for which server won't log stack traces.
|
||||
|
@ -2814,7 +2810,6 @@ public abstract class Server {
|
|||
this.rpcRequestClass = rpcRequestClass;
|
||||
this.handlerCount = handlerCount;
|
||||
this.socketSendBufferSize = 0;
|
||||
this.serverName = serverName;
|
||||
this.maxDataLength = conf.getInt(CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH,
|
||||
CommonConfigurationKeys.IPC_MAXIMUM_DATA_LENGTH_DEFAULT);
|
||||
if (queueSizePerHandler != -1) {
|
||||
|
|
|
@ -21,9 +21,11 @@ package org.apache.hadoop.io;
|
|||
import static org.junit.Assert.assertEquals;
|
||||
import static org.junit.Assert.fail;
|
||||
|
||||
import java.io.BufferedOutputStream;
|
||||
import java.io.ByteArrayInputStream;
|
||||
import java.io.EOFException;
|
||||
import java.io.File;
|
||||
import java.io.FileOutputStream;
|
||||
import java.io.FilenameFilter;
|
||||
import java.io.IOException;
|
||||
import java.io.InputStream;
|
||||
|
@ -289,4 +291,35 @@ public class TestIOUtils {
|
|||
FileUtils.deleteDirectory(dir);
|
||||
}
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testCloseStreams() throws IOException {
|
||||
File tmpFile = null;
|
||||
FileOutputStream fos;
|
||||
BufferedOutputStream bos;
|
||||
FileOutputStream nullStream = null;
|
||||
|
||||
try {
|
||||
tmpFile = new File(GenericTestUtils.getTestDir(), "testCloseStreams.txt");
|
||||
fos = new FileOutputStream(tmpFile) {
|
||||
@Override
|
||||
public void close() throws IOException {
|
||||
throw new IOException();
|
||||
}
|
||||
};
|
||||
bos = new BufferedOutputStream(
|
||||
new FileOutputStream(tmpFile)) {
|
||||
@Override
|
||||
public void close() {
|
||||
throw new NullPointerException();
|
||||
}
|
||||
};
|
||||
|
||||
IOUtils.closeStreams(fos, bos, nullStream);
|
||||
IOUtils.closeStreams();
|
||||
} finally {
|
||||
FileUtils.deleteQuietly(tmpFile);
|
||||
}
|
||||
|
||||
}
|
||||
}
|
||||
|
|
|
@ -129,7 +129,6 @@ public class NameNodeProxiesClient {
|
|||
* @return an object containing both the proxy and the associated
|
||||
* delegation token service it corresponds to
|
||||
* @throws IOException if there is an error creating the proxy
|
||||
* @see {@link NameNodeProxies#createProxy(Configuration, URI, Class)}.
|
||||
*/
|
||||
public static ProxyAndInfo<ClientProtocol> createProxyWithClientProtocol(
|
||||
Configuration conf, URI nameNodeUri, AtomicBoolean fallbackToSimpleAuth)
|
||||
|
|
|
@ -198,9 +198,6 @@ public interface HdfsClientConfigKeys {
|
|||
"dfs.namenode.snapshot.capture.openfiles";
|
||||
boolean DFS_NAMENODE_SNAPSHOT_CAPTURE_OPENFILES_DEFAULT = false;
|
||||
|
||||
String DFS_CLIENT_FAILOVER_IPFAILOVER_VIRTUAL_ADDRESS =
|
||||
Failover.PREFIX + "ipfailover.virtual-address";
|
||||
|
||||
/**
|
||||
* These are deprecated config keys to client code.
|
||||
*/
|
||||
|
|
|
@ -45,27 +45,19 @@ public class TestReadOnly {
|
|||
"getFileInfo",
|
||||
"isFileClosed",
|
||||
"getFileLinkInfo",
|
||||
"getLocatedFileInfo",
|
||||
"getContentSummary",
|
||||
"getLinkTarget",
|
||||
"getSnapshotDiffReport",
|
||||
"getSnapshotDiffReportListing",
|
||||
"listCacheDirectives",
|
||||
"listCachePools",
|
||||
"getAclStatus",
|
||||
"getEZForPath",
|
||||
"listEncryptionZones",
|
||||
"listReencryptionStatus",
|
||||
"getXAttrs",
|
||||
"listXAttrs",
|
||||
"checkAccess",
|
||||
"getErasureCodingPolicies",
|
||||
"getErasureCodingCodecs",
|
||||
"getErasureCodingPolicy",
|
||||
"listOpenFiles",
|
||||
"getStats",
|
||||
"getReplicatedBlockStats",
|
||||
"getECBlockGroupStats",
|
||||
"getDatanodeReport",
|
||||
"getDatanodeStorageReport",
|
||||
"getDataEncryptionKey",
|
||||
|
|
|
@ -1592,7 +1592,7 @@ public class ClientNamenodeProtocolServerSideTranslatorPB implements
|
|||
throw new ServiceException(e);
|
||||
}
|
||||
}
|
||||
|
||||
|
||||
@Override
|
||||
public MsyncResponseProto msync(RpcController controller,
|
||||
MsyncRequestProto req) throws ServiceException {
|
||||
|
|
|
@ -142,14 +142,13 @@ public interface QJournalProtocol {
|
|||
* {@value DFSConfigKeys#DFS_HA_TAILEDITS_INPROGRESS_KEY} configuration key.
|
||||
*
|
||||
* @param jid The ID of the journal from which to fetch edits.
|
||||
* @param nameServiceId The ID of the namespace for which to fetch edits.
|
||||
* @param sinceTxId Fetch edits starting at this transaction ID
|
||||
* @param maxTxns Request at most this many transactions to be returned
|
||||
* @throws IOException If there was an issue encountered while fetching edits
|
||||
* from the cache, including a cache miss (cache does not contain the
|
||||
* requested edits). The caller should then attempt to fetch the edits via
|
||||
* the streaming mechanism (starting with
|
||||
* {@link #getEditLogManifest(String, String, long, boolean)}).
|
||||
* {@link #getEditLogManifest(String, long, boolean)}).
|
||||
* @return Response containing serialized edits to be loaded
|
||||
* @see org.apache.hadoop.hdfs.qjournal.server.JournaledEditsCache
|
||||
*/
|
||||
|
|
|
@ -727,7 +727,7 @@ public class Journal implements Closeable {
|
|||
}
|
||||
|
||||
/**
|
||||
* @see QJournalProtocol#getJournaledEdits(String, String, long, int)
|
||||
* @see QJournalProtocol#getJournaledEdits(String, long, int)
|
||||
*/
|
||||
public GetJournaledEditsResponseProto getJournaledEdits(long sinceTxId,
|
||||
int maxTxns) throws IOException {
|
||||
|
|
|
@ -279,7 +279,6 @@ message GetJournaledEditsRequestProto {
|
|||
required JournalIdProto jid = 1;
|
||||
required uint64 sinceTxId = 2;
|
||||
required uint32 maxTxns = 3;
|
||||
optional string nameServiceId = 4;
|
||||
}
|
||||
|
||||
message GetJournaledEditsResponseProto {
|
||||
|
|
|
@ -35,6 +35,7 @@ import java.util.Arrays;
|
|||
import java.util.List;
|
||||
import java.util.concurrent.TimeUnit;
|
||||
import java.util.concurrent.TimeoutException;
|
||||
import java.util.concurrent.atomic.AtomicLong;
|
||||
import java.util.concurrent.atomic.LongAccumulator;
|
||||
|
||||
import com.google.common.base.Function;
|
||||
|
@ -214,12 +215,15 @@ public abstract class HATestUtil {
|
|||
|
||||
conf.setBoolean(DFS_HA_TAILEDITS_INPROGRESS_KEY, fastTailing);
|
||||
if(fastTailing) {
|
||||
conf.setTimeDuration(
|
||||
DFS_HA_TAILEDITS_PERIOD_KEY, 100, TimeUnit.MILLISECONDS);
|
||||
// TODO: In trunk, the below setting is 100ms. But
|
||||
// without backporting HDFS-9847, specifying time unit is not supported
|
||||
conf.setInt(DFS_HA_TAILEDITS_PERIOD_KEY, 0);
|
||||
} else {
|
||||
// disable fast tailing so that coordination takes time.
|
||||
conf.setTimeDuration(DFS_HA_LOGROLL_PERIOD_KEY, 300, TimeUnit.SECONDS);
|
||||
conf.setTimeDuration(DFS_HA_TAILEDITS_PERIOD_KEY, 200, TimeUnit.SECONDS);
|
||||
// TODO: In trunk, the below setting is 300s and 200s. But
|
||||
// without backporting HDFS-9847, specifying time unit is not supported
|
||||
conf.setInt(DFS_HA_LOGROLL_PERIOD_KEY, 300);
|
||||
conf.setInt(DFS_HA_TAILEDITS_PERIOD_KEY, 200);
|
||||
}
|
||||
|
||||
MiniQJMHACluster.Builder qjmBuilder = new MiniQJMHACluster.Builder(conf)
|
||||
|
@ -362,9 +366,9 @@ public abstract class HATestUtil {
|
|||
ClientGSIContext ac = (ClientGSIContext)(provider.getAlignmentContext());
|
||||
Field f = ac.getClass().getDeclaredField("lastSeenStateId");
|
||||
f.setAccessible(true);
|
||||
LongAccumulator lastSeenStateId = (LongAccumulator)f.get(ac);
|
||||
long currentStateId = lastSeenStateId.getThenReset();
|
||||
lastSeenStateId.accumulate(stateId);
|
||||
AtomicLong lastSeenStateId = (AtomicLong)f.get(ac);
|
||||
long currentStateId = lastSeenStateId.get();
|
||||
lastSeenStateId.set(stateId);
|
||||
return currentStateId;
|
||||
}
|
||||
}
|
||||
|
|
|
@ -162,10 +162,10 @@ public class TestConsistentReadsObserver {
|
|||
assertEquals(1, readStatus.get());
|
||||
}
|
||||
|
||||
private void testMsync(boolean autoMsync, long autoMsyncPeriodMs)
|
||||
private void testMsync(final boolean autoMsync, final long autoMsyncPeriodMs)
|
||||
throws Exception {
|
||||
// 0 == not completed, 1 == succeeded, -1 == failed
|
||||
AtomicInteger readStatus = new AtomicInteger(0);
|
||||
final AtomicInteger readStatus = new AtomicInteger(0);
|
||||
Configuration conf2 = new Configuration(conf);
|
||||
|
||||
// Disable FS cache so two different DFS clients will be used.
|
||||
|
@ -176,7 +176,8 @@ public class TestConsistentReadsObserver {
|
|||
+ "." + dfs.getUri().getHost(),
|
||||
autoMsyncPeriodMs, TimeUnit.MILLISECONDS);
|
||||
}
|
||||
DistributedFileSystem dfs2 = (DistributedFileSystem) FileSystem.get(conf2);
|
||||
final DistributedFileSystem dfs2 =
|
||||
(DistributedFileSystem) FileSystem.get(conf2);
|
||||
|
||||
// Initialize the proxies for Observer Node.
|
||||
dfs.getClient().getHAServiceState();
|
||||
|
@ -191,25 +192,28 @@ public class TestConsistentReadsObserver {
|
|||
dfs.mkdir(testPath, FsPermission.getDefault());
|
||||
assertSentTo(0);
|
||||
|
||||
Thread reader = new Thread(() -> {
|
||||
try {
|
||||
// After msync, client should have the latest state ID from active.
|
||||
// Therefore, the subsequent getFileStatus call should succeed.
|
||||
if (!autoMsync) {
|
||||
// If not testing auto-msync, perform an explicit one here
|
||||
dfs2.getClient().msync();
|
||||
} else if (autoMsyncPeriodMs > 0) {
|
||||
Thread.sleep(autoMsyncPeriodMs);
|
||||
}
|
||||
dfs2.getFileStatus(testPath);
|
||||
if (HATestUtil.isSentToAnyOfNameNodes(dfs2, dfsCluster, 2)) {
|
||||
readStatus.set(1);
|
||||
} else {
|
||||
Thread reader = new Thread(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
// After msync, client should have the latest state ID from active.
|
||||
// Therefore, the subsequent getFileStatus call should succeed.
|
||||
if (!autoMsync) {
|
||||
// If not testing auto-msync, perform an explicit one here
|
||||
dfs2.getClient().msync();
|
||||
} else if (autoMsyncPeriodMs > 0) {
|
||||
Thread.sleep(autoMsyncPeriodMs);
|
||||
}
|
||||
dfs2.getFileStatus(testPath);
|
||||
if (HATestUtil.isSentToAnyOfNameNodes(dfs2, dfsCluster, 2)) {
|
||||
readStatus.set(1);
|
||||
} else {
|
||||
readStatus.set(-1);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
readStatus.set(-1);
|
||||
}
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
readStatus.set(-1);
|
||||
}
|
||||
});
|
||||
|
||||
|
@ -220,7 +224,12 @@ public class TestConsistentReadsObserver {
|
|||
|
||||
dfsCluster.rollEditLogAndTail(0);
|
||||
|
||||
GenericTestUtils.waitFor(() -> readStatus.get() != 0, 100, 3000);
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
return readStatus.get() != 0;
|
||||
}
|
||||
}, 100, 3000);
|
||||
assertEquals(1, readStatus.get());
|
||||
}
|
||||
|
||||
|
@ -258,7 +267,7 @@ public class TestConsistentReadsObserver {
|
|||
dfsCluster.transitionToActive(2);
|
||||
try {
|
||||
// 0 == not completed, 1 == succeeded, -1 == failed
|
||||
AtomicInteger readStatus = new AtomicInteger(0);
|
||||
final AtomicInteger readStatus = new AtomicInteger(0);
|
||||
|
||||
// Initialize the proxies for Observer Node.
|
||||
dfs.getClient().getHAServiceState();
|
||||
|
@ -276,17 +285,20 @@ public class TestConsistentReadsObserver {
|
|||
|
||||
// Disable FS cache so two different DFS clients will be used.
|
||||
conf2.setBoolean("fs.hdfs.impl.disable.cache", true);
|
||||
DistributedFileSystem dfs2 =
|
||||
final DistributedFileSystem dfs2 =
|
||||
(DistributedFileSystem) FileSystem.get(conf2);
|
||||
dfs2.getClient().getHAServiceState();
|
||||
|
||||
Thread reader = new Thread(() -> {
|
||||
try {
|
||||
dfs2.getFileStatus(testPath);
|
||||
readStatus.set(1);
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
readStatus.set(-1);
|
||||
Thread reader = new Thread(new Runnable() {
|
||||
@Override
|
||||
public void run() {
|
||||
try {
|
||||
dfs2.getFileStatus(testPath);
|
||||
readStatus.set(1);
|
||||
} catch (Exception e) {
|
||||
e.printStackTrace();
|
||||
readStatus.set(-1);
|
||||
}
|
||||
}
|
||||
});
|
||||
|
||||
|
@ -299,7 +311,12 @@ public class TestConsistentReadsObserver {
|
|||
dfsCluster.getNameNode(0)
|
||||
.getNamesystem().getEditLogTailer().doTailEdits();
|
||||
|
||||
GenericTestUtils.waitFor(() -> readStatus.get() != 0, 100, 10000);
|
||||
GenericTestUtils.waitFor(new Supplier<Boolean>() {
|
||||
@Override
|
||||
public Boolean get() {
|
||||
return readStatus.get() != 0;
|
||||
}
|
||||
}, 100, 10000);
|
||||
assertEquals(1, readStatus.get());
|
||||
} finally {
|
||||
// Put the cluster back the way it was when the test started
|
||||
|
|
|
@ -137,8 +137,12 @@ public class TestDelegationTokensWithHA {
|
|||
getDelegationToken(dfs, ugi.getShortUserName());
|
||||
ugi.addToken(token);
|
||||
// Recreate the DFS, this time authenticating using a DT
|
||||
dfs = ugi.doAs((PrivilegedExceptionAction<DistributedFileSystem>)
|
||||
() -> (DistributedFileSystem) FileSystem.get(conf));
|
||||
dfs = ugi.doAs(new PrivilegedExceptionAction<DistributedFileSystem>() {
|
||||
@Override
|
||||
public DistributedFileSystem run() throws Exception {
|
||||
return (DistributedFileSystem) FileSystem.get(conf);
|
||||
}
|
||||
});
|
||||
|
||||
GenericTestUtils.setLogLevel(ObserverReadProxyProvider.LOG, Level.DEBUG);
|
||||
GenericTestUtils.LogCapturer logCapture = GenericTestUtils.LogCapturer
|
||||
|
|
|
@ -302,24 +302,6 @@ public class TestStandbyInProgressTail {
|
|||
waitForFileInfo(nn1, "/test", "/test2", "/test3");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testNonUniformConfig() throws Exception {
|
||||
// Test case where some NNs (in this case the active NN) in the cluster
|
||||
// do not have in-progress tailing enabled.
|
||||
Configuration newConf = cluster.getNameNode(0).getConf();
|
||||
newConf.setBoolean(
|
||||
DFSConfigKeys.DFS_HA_TAILEDITS_INPROGRESS_KEY,
|
||||
false);
|
||||
cluster.restartNameNode(0);
|
||||
cluster.transitionToActive(0);
|
||||
|
||||
cluster.getNameNode(0).getRpcServer().mkdirs("/test",
|
||||
FsPermission.createImmutable((short) 0755), true);
|
||||
cluster.getNameNode(0).getRpcServer().rollEdits();
|
||||
|
||||
waitForFileInfo(nn1, "/test");
|
||||
}
|
||||
|
||||
@Test
|
||||
public void testEditsServedViaCache() throws Exception {
|
||||
cluster.transitionToActive(0);
|
||||
|
|
Loading…
Reference in New Issue